From 666557615a888ce55083c49c57b0b16cbbc235b6 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Tue, 22 Aug 2023 20:24:50 +0530 Subject: [PATCH 01/37] Supports array string in BigQuery CDC (#339) --- flow/connectors/bigquery/bigquery.go | 3 ++- flow/e2e/peer_flow_test.go | 5 +++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/flow/connectors/bigquery/bigquery.go b/flow/connectors/bigquery/bigquery.go index e822b6cb62..e64ce92a69 100644 --- a/flow/connectors/bigquery/bigquery.go +++ b/flow/connectors/bigquery/bigquery.go @@ -1227,7 +1227,8 @@ func (m *MergeStmtGenerator) generateFlattenedCTE() string { case qvalue.QValueKindBytes, qvalue.QValueKindBit: castStmt = fmt.Sprintf("FROM_BASE64(JSON_EXTRACT_SCALAR(_peerdb_data, '$.%s')) AS %s", colName, colName) - case qvalue.QValueKindArrayFloat32, qvalue.QValueKindArrayFloat64, qvalue.QValueKindArrayInt32, qvalue.QValueKindArrayInt64: + case qvalue.QValueKindArrayFloat32, qvalue.QValueKindArrayFloat64, + qvalue.QValueKindArrayInt32, qvalue.QValueKindArrayInt64, qvalue.QValueKindArrayString: castStmt = fmt.Sprintf("ARRAY(SELECT CAST(element AS %s) FROM "+ "UNNEST(CAST(JSON_EXTRACT_ARRAY(_peerdb_data, '$.%s') AS ARRAY)) AS element) AS %s", bqType, colName, colName) diff --git a/flow/e2e/peer_flow_test.go b/flow/e2e/peer_flow_test.go index 103031a18f..4df215cd3f 100644 --- a/flow/e2e/peer_flow_test.go +++ b/flow/e2e/peer_flow_test.go @@ -916,7 +916,7 @@ func (s *E2EPeerFlowTestSuite) Test_Types_Avro_BQ() { c14 INET,c15 INTEGER,c16 INTERVAL,c17 JSON,c18 JSONB,c21 MACADDR,c22 MONEY, c23 NUMERIC,c24 OID,c28 REAL,c29 SMALLINT,c30 SMALLSERIAL,c31 SERIAL,c32 TEXT, c33 TIMESTAMP,c34 TIMESTAMPTZ,c35 TIME, c36 TIMETZ,c37 TSQUERY,c38 TSVECTOR, - c39 TXID_SNAPSHOT,c40 UUID,c41 XML, c42 INT[], c43 FLOAT[]); + c39 TXID_SNAPSHOT,c40 UUID,c41 XML, c42 INT[], c43 FLOAT[], c44 TEXT[]); CREATE OR REPLACE FUNCTION random_bytea(bytea_length integer) RETURNS bytea AS $body$ SELECT decode(string_agg(lpad(to_hex(width_bucket(random(), 0, 1, 256)-1),2,'0') ,''), 'hex') @@ -962,7 +962,8 @@ func (s *E2EPeerFlowTestSuite) Test_Types_Avro_BQ() { txid_current_snapshot(), '66073c38-b8df-4bdb-bbca-1c97596b8940'::uuid,xmlcomment('hello'), ARRAY[9301,239827], - ARRAY[0.0003, 1039.0034]; + ARRAY[0.0003, 1039.0034], + ARRAY['hello','bye']; `) s.NoError(err) fmt.Println("Executed an insert with all types") From d0198af0c6f707f98329e53166fff568a9d1260e Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Tue, 22 Aug 2023 22:40:38 +0530 Subject: [PATCH 02/37] Better Go and Activity Logs (#340) - Adds stream splitting for logrus - Adds heartbeat messages at many places in BigQuery Avro sync functions --- flow/connectors/bigquery/qrep_avro_sync.go | 51 +++++++++++++++++++--- flow/connectors/utils/logrus.go | 22 ++++++++++ 2 files changed, 67 insertions(+), 6 deletions(-) create mode 100644 flow/connectors/utils/logrus.go diff --git a/flow/connectors/bigquery/qrep_avro_sync.go b/flow/connectors/bigquery/qrep_avro_sync.go index 4f36dfbac6..ce90de6c14 100644 --- a/flow/connectors/bigquery/qrep_avro_sync.go +++ b/flow/connectors/bigquery/qrep_avro_sync.go @@ -9,12 +9,14 @@ import ( "time" "cloud.google.com/go/bigquery" + "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/connectors/utils/metrics" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/linkedin/goavro/v2" log "github.com/sirupsen/logrus" + "go.temporal.io/sdk/activity" ) type QRepAvroSyncMethod struct { @@ -37,7 +39,11 @@ func (s *QRepAvroSyncMethod) SyncRecords( syncBatchID int64, stream *model.QRecordStream, ) (int, error) { - + activity.RecordHeartbeat(s.connector.ctx, time.Minute, + fmt.Sprintf("Flow job %s: Obtaining Avro schema"+ + " for destination table %s and sync batch ID %d", + flowJobName, dstTableName, syncBatchID), + ) // You will need to define your Avro schema as a string avroSchema, nullable, err := DefineAvroSchema(dstTableName, dstTableMetadata) if err != nil { @@ -58,6 +64,13 @@ func (s *QRepAvroSyncMethod) SyncRecords( if err != nil { return -1, fmt.Errorf("failed to update metadata: %v", err) } + + activity.RecordHeartbeat(s.connector.ctx, time.Minute, + fmt.Sprintf("Flow job %s: performing insert and update transaction"+ + " for destination table %s and sync batch ID %d", + flowJobName, dstTableName, syncBatchID), + ) + // execute the statements in a transaction stmts := []string{} stmts = append(stmts, "BEGIN TRANSACTION;") @@ -97,14 +110,17 @@ func (s *QRepAvroSyncMethod) SyncQRepRecords( } fmt.Printf("Avro schema: %s\n", avroSchema) - // create a staging table name with partitionID replace hyphens with underscores stagingTable := fmt.Sprintf("%s_%s_staging", dstTableName, strings.ReplaceAll(partition.PartitionId, "-", "_")) numRecords, err := s.writeToStage(partition.PartitionId, flowJobName, avroSchema, stagingTable, stream, nullable) if err != nil { return -1, fmt.Errorf("failed to push to avro stage: %v", err) } - + activity.RecordHeartbeat(s.connector.ctx, fmt.Sprintf( + "Flow job %s: running insert-into-select transaction for"+ + " destination table %s and partition ID %s", + flowJobName, dstTableName, partition.PartitionId), + ) bqClient := s.connector.client datasetID := s.connector.datasetID // Start a transaction @@ -282,6 +298,15 @@ func (s *QRepAvroSyncMethod) writeToStage( stream *model.QRecordStream, nullable map[string]bool, ) (int, error) { + shutdown := utils.HeartbeatRoutine(s.connector.ctx, time.Minute, + func() string { + return fmt.Sprintf("writing to avro stage for objectFolder %s and staging table %s", + objectFolder, stagingTable) + }, + ) + defer func() { + shutdown <- true + }() ctx := context.Background() bucket := s.connector.storageClient.Bucket(s.gcsBucket) gcsObjectName := fmt.Sprintf("%s/%s.avro", objectFolder, syncID) @@ -298,15 +323,26 @@ func (s *QRepAvroSyncMethod) writeToStage( if err != nil { return 0, fmt.Errorf("failed to create OCF writer: %w", err) } + schema, err := stream.Schema() if err != nil { log.Errorf("failed to get schema from stream: %v", err) return 0, fmt.Errorf("failed to get schema from stream: %w", err) } - numRecords := 0 + activity.RecordHeartbeat(s.connector.ctx, fmt.Sprintf( + "Obtained staging bucket %s and schema of rows. Now writing records to OCF file.", + gcsObjectName), + ) + numRecords := 0 // Write each QRecord to the OCF file for qRecordOrErr := range stream.Records { + if numRecords > 0 && numRecords%10000 == 0 { + activity.RecordHeartbeat(s.connector.ctx, fmt.Sprintf( + "Written %d records to OCF file for staging bucket %s.", + numRecords, gcsObjectName), + ) + } if qRecordOrErr.Err != nil { log.Errorf("[bq_avro] failed to get record from stream: %v", qRecordOrErr.Err) return 0, fmt.Errorf("[bq_avro] failed to get record from stream: %w", qRecordOrErr.Err) @@ -329,8 +365,12 @@ 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", + syncID), + ) // Write OCF contents to GCS if _, err = w.Write(ocfFileContents.Bytes()); err != nil { return 0, fmt.Errorf("failed to write OCF file to GCS: %w", err) @@ -343,7 +383,6 @@ func (s *QRepAvroSyncMethod) writeToStage( // write this file to bigquery gcsRef := bigquery.NewGCSReference(fmt.Sprintf("gs://%s/%s", s.gcsBucket, gcsObjectName)) gcsRef.SourceFormat = bigquery.Avro - bqClient := s.connector.client datasetID := s.connector.datasetID loader := bqClient.Dataset(datasetID).Table(stagingTable).LoaderFrom(gcsRef) diff --git a/flow/connectors/utils/logrus.go b/flow/connectors/utils/logrus.go new file mode 100644 index 0000000000..dc6551cff3 --- /dev/null +++ b/flow/connectors/utils/logrus.go @@ -0,0 +1,22 @@ +package utils + +import ( + "bytes" + "os" + + "github.com/sirupsen/logrus" +) + +// we do this as logrus does not separate output into stderr and stdout +type LogRouter struct{} + +func (router *LogRouter) Write(p []byte) (n int, err error) { + if bytes.Contains(p, []byte("level=error")) { + return os.Stderr.Write(p) + } + return os.Stdout.Write(p) +} + +func init() { + logrus.SetOutput(&LogRouter{}) +} From ca360a15a9d0eb6ecaee4683ad79863e31769e08 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Wed, 23 Aug 2023 22:24:34 +0530 Subject: [PATCH 03/37] Fix statements in normalize flow (#343) - Backticks for columns to account for keywords - Unique temporary table name --- flow/connectors/bigquery/bigquery.go | 37 ++++++++++--------- .../bigquery/merge_stmt_generator_test.go | 32 +++++++++++----- flow/utils/random.go | 13 +++++++ 3 files changed, 55 insertions(+), 27 deletions(-) diff --git a/flow/connectors/bigquery/bigquery.go b/flow/connectors/bigquery/bigquery.go index e64ce92a69..3aa228ab97 100644 --- a/flow/connectors/bigquery/bigquery.go +++ b/flow/connectors/bigquery/bigquery.go @@ -17,6 +17,7 @@ import ( "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" + util "github.com/PeerDB-io/peer-flow/utils" "github.com/google/uuid" log "github.com/sirupsen/logrus" "google.golang.org/api/iterator" @@ -1192,15 +1193,15 @@ func (m *MergeStmtGenerator) GenerateMergeStmts() []string { // return an empty array for now flattenedCTE := m.generateFlattenedCTE() deDupedCTE := m.generateDeDupedCTE() - + tempTable := fmt.Sprintf("_peerdb_de_duplicated_data_%s", util.RandomString(5)) // create temp table stmt createTempTableStmt := fmt.Sprintf( - "CREATE TEMP TABLE _peerdb_de_duplicated_data AS (%s, %s);", - flattenedCTE, deDupedCTE) + "CREATE TEMP TABLE %s AS (%s, %s);", + tempTable, flattenedCTE, deDupedCTE) - mergeStmt := m.generateMergeStmt() + mergeStmt := m.generateMergeStmt(tempTable) - dropTempTableStmt := "DROP TABLE _peerdb_de_duplicated_data;" + dropTempTableStmt := fmt.Sprintf("DROP TABLE %s;", tempTable) return []string{createTempTableStmt, mergeStmt, dropTempTableStmt} } @@ -1221,16 +1222,16 @@ func (m *MergeStmtGenerator) generateFlattenedCTE() string { switch qvalue.QValueKind(colType) { case qvalue.QValueKindJSON: //if the type is JSON, then just extract JSON - castStmt = fmt.Sprintf("CAST(JSON_EXTRACT(_peerdb_data, '$.%s') AS %s) AS %s", + castStmt = fmt.Sprintf("CAST(JSON_EXTRACT(_peerdb_data, '$.%s') AS %s) AS `%s`", colName, bqType, colName) // expecting data in BASE64 format case qvalue.QValueKindBytes, qvalue.QValueKindBit: - castStmt = fmt.Sprintf("FROM_BASE64(JSON_EXTRACT_SCALAR(_peerdb_data, '$.%s')) AS %s", + castStmt = fmt.Sprintf("FROM_BASE64(JSON_EXTRACT_SCALAR(_peerdb_data, '$.%s')) AS `%s`", colName, colName) case qvalue.QValueKindArrayFloat32, qvalue.QValueKindArrayFloat64, qvalue.QValueKindArrayInt32, qvalue.QValueKindArrayInt64, qvalue.QValueKindArrayString: castStmt = fmt.Sprintf("ARRAY(SELECT CAST(element AS %s) FROM "+ - "UNNEST(CAST(JSON_EXTRACT_ARRAY(_peerdb_data, '$.%s') AS ARRAY)) AS element) AS %s", + "UNNEST(CAST(JSON_EXTRACT_ARRAY(_peerdb_data, '$.%s') AS ARRAY)) AS element) AS `%s`", bqType, colName, colName) // MAKE_INTERVAL(years INT64, months INT64, days INT64, hours INT64, minutes INT64, seconds INT64) // Expecting interval to be in the format of {"Microseconds":2000000,"Days":0,"Months":0,"Valid":true} @@ -1248,7 +1249,7 @@ func (m *MergeStmtGenerator) generateFlattenedCTE() string { // " AS int64))) AS %s", // colName, colName) default: - castStmt = fmt.Sprintf("CAST(JSON_EXTRACT_SCALAR(_peerdb_data, '$.%s') AS %s) AS %s", + castStmt = fmt.Sprintf("CAST(JSON_EXTRACT_SCALAR(_peerdb_data, '$.%s') AS %s) AS `%s`", colName, bqType, colName) } flattenedProjs = append(flattenedProjs, castStmt) @@ -1282,28 +1283,30 @@ func (m *MergeStmtGenerator) generateDeDupedCTE() string { } // generateMergeStmt generates a merge statement. -func (m *MergeStmtGenerator) generateMergeStmt() string { +func (m *MergeStmtGenerator) generateMergeStmt(tempTable string) string { pkey := m.NormalizedTableSchema.PrimaryKeyColumn // comma separated list of column names - colNames := make([]string, 0) + backtickColNames := make([]string, 0) + pureColNames := make([]string, 0) for colName := range m.NormalizedTableSchema.Columns { - colNames = append(colNames, colName) + backtickColNames = append(backtickColNames, fmt.Sprintf("`%s`", colName)) + pureColNames = append(pureColNames, colName) } - csep := strings.Join(colNames, ", ") + csep := strings.Join(backtickColNames, ", ") - udateStatementsforToastCols := m.generateUpdateStatement(colNames, m.UnchangedToastColumns) + udateStatementsforToastCols := m.generateUpdateStatement(pureColNames, m.UnchangedToastColumns) updateStringToastCols := strings.Join(udateStatementsforToastCols, " ") return fmt.Sprintf(` - MERGE %s.%s _peerdb_target USING _peerdb_de_duplicated_data _peerdb_deduped + MERGE %s.%s _peerdb_target USING %s _peerdb_deduped ON _peerdb_target.%s = _peerdb_deduped.%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, pkey, pkey, csep, csep, updateStringToastCols) + `, m.Dataset, m.NormalizedTable, tempTable, pkey, pkey, csep, csep, updateStringToastCols) } /* @@ -1329,7 +1332,7 @@ func (m *MergeStmtGenerator) generateUpdateStatement(allCols []string, unchanged otherCols := utils.ArrayMinus(allCols, unchangedColsArray) tmpArray := make([]string, 0) for _, colName := range otherCols { - tmpArray = append(tmpArray, fmt.Sprintf("%s = _peerdb_deduped.%s", colName, colName)) + tmpArray = append(tmpArray, fmt.Sprintf("`%s` = _peerdb_deduped.%s", colName, colName)) } ssep := strings.Join(tmpArray, ", ") updateStmt := fmt.Sprintf(`WHEN MATCHED AND diff --git a/flow/connectors/bigquery/merge_stmt_generator_test.go b/flow/connectors/bigquery/merge_stmt_generator_test.go index ea8c9245c6..7320639680 100644 --- a/flow/connectors/bigquery/merge_stmt_generator_test.go +++ b/flow/connectors/bigquery/merge_stmt_generator_test.go @@ -12,14 +12,22 @@ func TestGenerateUpdateStatement_WithUnchangedToastCols(t *testing.T) { unchangedToastCols := []string{"", "col2, col3", "col2", "col3"} expected := []string{ - `WHEN MATCHED AND (_peerdb_deduped._peerdb_record_type != 2) AND _peerdb_unchanged_toast_columns='' - THEN UPDATE SET col1 = _peerdb_deduped.col1, col2 = _peerdb_deduped.col2, col3 = _peerdb_deduped.col3`, - `WHEN MATCHED AND (_peerdb_deduped._peerdb_record_type != 2) AND _peerdb_unchanged_toast_columns='col2, col3' - THEN UPDATE SET col1 = _peerdb_deduped.col1`, - `WHEN MATCHED AND (_peerdb_deduped._peerdb_record_type != 2) AND _peerdb_unchanged_toast_columns='col2' - THEN UPDATE SET col1 = _peerdb_deduped.col1, col3 = _peerdb_deduped.col3`, - `WHEN MATCHED AND (_peerdb_deduped._peerdb_record_type != 2) AND _peerdb_unchanged_toast_columns='col3' - THEN UPDATE SET col1 = _peerdb_deduped.col1, col2 = _peerdb_deduped.col2`, + "WHEN MATCHED AND (_peerdb_deduped._peerdb_record_type != 2)" + + " AND _peerdb_unchanged_toast_columns='' " + + "THEN UPDATE SET `col1` = _peerdb_deduped.col1," + + " `col2` = _peerdb_deduped.col2," + + " `col3` = _peerdb_deduped.col3", + "WHEN MATCHED AND (_peerdb_deduped._peerdb_record_type != 2)" + + " AND _peerdb_unchanged_toast_columns='col2, col3' " + + "THEN UPDATE SET `col1` = _peerdb_deduped.col1", + "WHEN MATCHED AND (_peerdb_deduped._peerdb_record_type != 2)" + + " AND _peerdb_unchanged_toast_columns='col2'" + + "THEN UPDATE SET `col1` = _peerdb_deduped.col1," + + " `col3` = _peerdb_deduped.col3", + "WHEN MATCHED AND (_peerdb_deduped._peerdb_record_type != 2)" + + " AND _peerdb_unchanged_toast_columns='col3'" + + "THEN UPDATE SET `col1` = _peerdb_deduped.col1," + + " `col2` = _peerdb_deduped.col2", } result := m.generateUpdateStatement(allCols, unchangedToastCols) @@ -40,8 +48,12 @@ func TestGenerateUpdateStatement_NoUnchangedToastCols(t *testing.T) { unchangedToastCols := []string{""} expected := []string{ - `WHEN MATCHED AND (_peerdb_deduped._peerdb_record_type != 2) AND _peerdb_unchanged_toast_columns='' - THEN UPDATE SET col1 = _peerdb_deduped.col1, col2 = _peerdb_deduped.col2, col3 = _peerdb_deduped.col3`, + "WHEN MATCHED AND (_peerdb_deduped._peerdb_record_type != 2) " + + "AND _peerdb_unchanged_toast_columns=''" + + "THEN UPDATE SET " + + "`col1` = _peerdb_deduped.col1," + + " `col2` = _peerdb_deduped.col2," + + " `col3` = _peerdb_deduped.col3", } result := m.generateUpdateStatement(allCols, unchangedToastCols) diff --git a/flow/utils/random.go b/flow/utils/random.go index a3c22330ca..7222a23f94 100644 --- a/flow/utils/random.go +++ b/flow/utils/random.go @@ -27,3 +27,16 @@ func RandomUInt64() (uint64, error) { // Convert bytes to uint64 return binary.LittleEndian.Uint64(b), nil } + +func RandomString(n int) string { + const alphanum = "0123456789ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz" + var bytes = make([]byte, n) + _, err := rand.Read(bytes) + if err != nil { + return "temp" + } + for i, b := range bytes { + bytes[i] = alphanum[b%byte(len(alphanum))] + } + return string(bytes) +} From 2f987e4d38f87f51145737b76c5244a329bb6047 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Thu, 24 Aug 2023 02:44:02 +0530 Subject: [PATCH 04/37] Support for Arrays PG to PG (#344) --- flow/connectors/postgres/client.go | 10 ++++++++-- flow/connectors/postgres/qvalue_convert.go | 10 ++++++++++ 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/flow/connectors/postgres/client.go b/flow/connectors/postgres/client.go index 0ed55d976d..03fb7cf092 100644 --- a/flow/connectors/postgres/client.go +++ b/flow/connectors/postgres/client.go @@ -495,8 +495,14 @@ func (c *PostgresConnector) generateMergeStatement(destinationTableIdentifier st var primaryKeyColumnCast string for columnName, genericColumnType := range normalizedTableSchema.Columns { pgType := qValueKindToPostgresType(genericColumnType) - flattenedCastsSQLArray = append(flattenedCastsSQLArray, fmt.Sprintf("(_peerdb_data->>'%s')::%s AS %s", - columnName, pgType, columnName)) + if strings.Contains(genericColumnType, "array") { + flattenedCastsSQLArray = append(flattenedCastsSQLArray, + fmt.Sprintf("ARRAY(SELECT * FROM JSON_ARRAY_ELEMENTS_TEXT((_peerdb_data->>'%s')::JSON))::%s AS \"%s\"", + columnName, pgType, columnName)) + } else { + 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) } diff --git a/flow/connectors/postgres/qvalue_convert.go b/flow/connectors/postgres/qvalue_convert.go index 62465bd4cc..b9c7dcc904 100644 --- a/flow/connectors/postgres/qvalue_convert.go +++ b/flow/connectors/postgres/qvalue_convert.go @@ -121,6 +121,16 @@ func qValueKindToPostgresType(qvalueKind string) string { return "NUMERIC" case qvalue.QValueKindBit: return "BIT" + case qvalue.QValueKindArrayInt32: + return "INTEGER[]" + case qvalue.QValueKindArrayInt64: + return "BIGINT[]" + case qvalue.QValueKindArrayFloat32: + return "REAL[]" + case qvalue.QValueKindArrayFloat64: + return "DOUBLE PRECISION[]" + case qvalue.QValueKindArrayString: + return "TEXT[]" default: return "TEXT" } From 6613652b43f7af4570ba3095248f858e7085e4b0 Mon Sep 17 00:00:00 2001 From: Kevin K Biju <52661649+heavycrystal@users.noreply.github.com> Date: Thu, 24 Aug 2023 17:03:33 +0530 Subject: [PATCH 05/37] generating unique workflow IDs in snapshot flow via side-effect (#346) --- flow/workflows/snapshot_flow.go | 26 ++++++++++++++++++-------- 1 file changed, 18 insertions(+), 8 deletions(-) diff --git a/flow/workflows/snapshot_flow.go b/flow/workflows/snapshot_flow.go index 71ea63160c..e441c6a877 100644 --- a/flow/workflows/snapshot_flow.go +++ b/flow/workflows/snapshot_flow.go @@ -8,6 +8,7 @@ import ( "github.com/PeerDB-io/peer-flow/concurrency" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/shared" + "github.com/google/uuid" "go.temporal.io/sdk/log" "go.temporal.io/sdk/temporal" "go.temporal.io/sdk/workflow" @@ -74,12 +75,19 @@ func (s *SnapshotFlowExecution) cloneTable( snapshotName string, sourceTable string, destinationTableName string, -) workflow.Future { +) (workflow.Future, error) { flowName := s.config.FlowJobName - childWorkflowID := fmt.Sprintf("clone_%s_%s", flowName, destinationTableName) - reg := regexp.MustCompile("[^a-zA-Z0-9]+") - childWorkflowID = reg.ReplaceAllString(childWorkflowID, "_") + childWorkflowIDSideEffect := workflow.SideEffect(childCtx, func(ctx workflow.Context) interface{} { + childWorkflowID := fmt.Sprintf("clone_%s_%s_%s", flowName, destinationTableName, uuid.New().String()) + reg := regexp.MustCompile("[^a-zA-Z0-9]+") + return reg.ReplaceAllString(childWorkflowID, "_") + }) + + var childWorkflowID string + if err := childWorkflowIDSideEffect.Get(&childWorkflowID); err != nil { + return nil, fmt.Errorf("failed to get child workflow ID: %w", err) + } childCtx = workflow.WithChildOptions(childCtx, workflow.ChildWorkflowOptions{ WorkflowID: childWorkflowID, @@ -134,7 +142,7 @@ func (s *SnapshotFlowExecution) cloneTable( numPartitionsProcessed, ) - return qrepFuture + return qrepFuture, nil } // startChildQrepWorkflow starts a child workflow for query based replication. @@ -149,7 +157,11 @@ func (s *SnapshotFlowExecution) cloneTables( source := srcTbl snapshotName := slotInfo.SnapshotName - future := s.cloneTable(ctx, snapshotName, source, dstTbl) + future, err := s.cloneTable(ctx, snapshotName, source, dstTbl) + if err != nil { + s.logger.Error("failed to start clone child workflow: ", err) + continue + } boundSelector.AddFuture(future, func(f workflow.Future) error { if err := f.Get(ctx, nil); err != nil { s.logger.Error("failed to clone table", "table", source, "error", err) @@ -166,8 +178,6 @@ func (s *SnapshotFlowExecution) cloneTables( } s.logger.Info("finished cloning tables") - - return } func SnapshotFlowWorkflow(ctx workflow.Context, config *protos.FlowConnectionConfigs) error { From ee41a7b63e066e56be0225d8c4a94f14f20984e2 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Thu, 24 Aug 2023 17:46:38 +0530 Subject: [PATCH 06/37] Uses peerdb prefix in alias for BQ and PG Merge (#345) --- flow/connectors/bigquery/bigquery.go | 4 ++-- flow/connectors/postgres/client.go | 12 ++++++------ 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/flow/connectors/bigquery/bigquery.go b/flow/connectors/bigquery/bigquery.go index 3aa228ab97..f3e0d84863 100644 --- a/flow/connectors/bigquery/bigquery.go +++ b/flow/connectors/bigquery/bigquery.go @@ -1274,9 +1274,9 @@ func (m *MergeStmtGenerator) generateDeDupedCTE() string { FROM ( SELECT RANK() OVER ( PARTITION BY %s ORDER BY _peerdb_timestamp_nanos DESC - ) as rank, * FROM _peerdb_flattened + ) as _peerdb_rank, * FROM _peerdb_flattened ) _peerdb_ranked - WHERE rank = 1 + WHERE _peerdb_rank = 1 ) SELECT * FROM _peerdb_de_duplicated_data_res` pkey := m.NormalizedTableSchema.PrimaryKeyColumn return fmt.Sprintf(cte, pkey) diff --git a/flow/connectors/postgres/client.go b/flow/connectors/postgres/client.go index 03fb7cf092..a6ea5996e6 100644 --- a/flow/connectors/postgres/client.go +++ b/flow/connectors/postgres/client.go @@ -43,11 +43,11 @@ const ( srcTableName = "src" mergeStatementSQL = `WITH src_rank AS ( SELECT _peerdb_data,_peerdb_record_type,_peerdb_unchanged_toast_columns, - RANK() OVER (PARTITION BY %s ORDER BY _peerdb_timestamp DESC) AS rank + 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 ) MERGE INTO %s dst - USING (SELECT %s,_peerdb_record_type,_peerdb_unchanged_toast_columns FROM src_rank WHERE rank=1) src + USING (SELECT %s,_peerdb_record_type,_peerdb_unchanged_toast_columns FROM src_rank WHERE _peerdb_rank=1) src ON dst.%s=src.%s WHEN NOT MATCHED AND src._peerdb_record_type!=2 THEN INSERT (%s) VALUES (%s) @@ -56,17 +56,17 @@ const ( DELETE` fallbackUpsertStatementSQL = `WITH src_rank AS ( SELECT _peerdb_data,_peerdb_record_type,_peerdb_unchanged_toast_columns, - RANK() OVER (PARTITION BY %s ORDER BY _peerdb_timestamp DESC) AS rank + 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 ) - INSERT INTO %s (%s) SELECT %s FROM src_rank WHERE rank=1 AND _peerdb_record_type!=2 + INSERT INTO %s (%s) SELECT %s FROM src_rank WHERE _peerdb_rank=1 AND _peerdb_record_type!=2 ON CONFLICT (%s) DO UPDATE SET %s` fallbackDeleteStatementSQL = `WITH src_rank AS ( SELECT _peerdb_data,_peerdb_record_type,_peerdb_unchanged_toast_columns, - RANK() OVER (PARTITION BY %s ORDER BY _peerdb_timestamp DESC) AS rank + 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.rank=1 AND src_rank._peerdb_record_type=2` + DELETE FROM %s USING src_rank WHERE %s.%s=%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" From 2d5368edfde439cde9664eaf5ebf9d292affae8a Mon Sep 17 00:00:00 2001 From: Kevin K Biju <52661649+heavycrystal@users.noreply.github.com> Date: Thu, 24 Aug 2023 18:54:50 +0530 Subject: [PATCH 07/37] added heartbeats (#330) --- flow/connectors/postgres/postgres.go | 16 ++++++++++++++++ flow/workflows/setup_flow.go | 1 + 2 files changed, 17 insertions(+) diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index 3d6dad42f3..5900a93295 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -20,6 +20,7 @@ import ( "github.com/jackc/pgx/v5/pgconn" "github.com/jackc/pgx/v5/pgxpool" log "github.com/sirupsen/logrus" + "go.temporal.io/sdk/activity" "golang.org/x/exp/maps" ) @@ -511,6 +512,7 @@ func (c *PostgresConnector) GetTableSchema( return nil, err } res[tableName] = tableSchema + c.recordHeartbeatWithRecover(fmt.Sprintf("fetched schema for table %s", tableName)) } return &protos.GetTableSchemaBatchOutput{ @@ -601,6 +603,7 @@ func (c *PostgresConnector) SetupNormalizedTables(req *protos.SetupNormalizedTab tableExistsMapping[tableIdentifier] = false log.Printf("created table %s", tableIdentifier) + c.recordHeartbeatWithRecover(fmt.Sprintf("created table %s", tableIdentifier)) } err = createNormalizedTablesTx.Commit(c.ctx) @@ -642,6 +645,7 @@ func (c *PostgresConnector) EnsurePullability(req *protos.EnsurePullabilityBatch RelId: relID}, }, } + c.recordHeartbeatWithRecover(fmt.Sprintf("ensured pullability table %s", tableName)) } return &protos.EnsurePullabilityBatchOutput{TableIdentifierMapping: tableIdentifierMapping}, nil @@ -754,3 +758,15 @@ func parseSchemaTable(tableName string) (*SchemaTable, error) { Table: parts[1], }, nil } + +// if the functions are being called outside the context of a Temporal workflow, +// activity.RecordHeartbeat panics, this is a bandaid for that. +func (c *PostgresConnector) recordHeartbeatWithRecover(details ...interface{}) { + defer func() { + if r := recover(); r != nil { + log.Warnln("ignoring panic from activity.RecordHeartbeat") + log.Warnln("this can happen when function is invoked outside of a Temporal workflow") + } + }() + activity.RecordHeartbeat(c.ctx, details...) +} diff --git a/flow/workflows/setup_flow.go b/flow/workflows/setup_flow.go index 1919c8fdaa..bc55d2ca6b 100644 --- a/flow/workflows/setup_flow.go +++ b/flow/workflows/setup_flow.go @@ -160,6 +160,7 @@ func (s *SetupFlowExecution) fetchTableSchemaAndSetupNormalizedTables( ctx = workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ StartToCloseTimeout: 1 * time.Hour, + HeartbeatTimeout: 5 * time.Minute, }) sourceTables := make([]string, 0) From 719dc35353f959c75378e7ed77bf6345e4881452 Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Thu, 24 Aug 2023 09:58:56 -0700 Subject: [PATCH 08/37] Truncate progress (#349) --- flow/workflows/peer_flow.go | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/flow/workflows/peer_flow.go b/flow/workflows/peer_flow.go index d0f1a99e3e..8b49f5bd9a 100644 --- a/flow/workflows/peer_flow.go +++ b/flow/workflows/peer_flow.go @@ -82,6 +82,19 @@ func NewStartedPeerFlowState() *PeerFlowState { } } +// truncate the progress and other arrays to a max of 10 elements +func (s *PeerFlowState) TruncateProgress() { + if len(s.Progress) > 10 { + s.Progress = s.Progress[len(s.Progress)-10:] + } + if len(s.SyncFlowStatuses) > 10 { + s.SyncFlowStatuses = s.SyncFlowStatuses[len(s.SyncFlowStatuses)-10:] + } + if len(s.NormalizeFlowStatuses) > 10 { + s.NormalizeFlowStatuses = s.NormalizeFlowStatuses[len(s.NormalizeFlowStatuses)-10:] + } +} + // PeerFlowWorkflowExecution represents the state for execution of a peer flow. type PeerFlowWorkflowExecution struct { flowExecutionID string @@ -362,5 +375,6 @@ func PeerFlowWorkflowWithConfig( }) selector.Select(ctx) + state.TruncateProgress() return nil, workflow.NewContinueAsNewError(ctx, PeerFlowWorkflowWithConfig, cfg, limits, state) } From a325fdf9ceae4b2c1147e31827b9df0406754f1a Mon Sep 17 00:00:00 2001 From: Kevin K Biju <52661649+heavycrystal@users.noreply.github.com> Date: Fri, 25 Aug 2023 00:01:46 +0530 Subject: [PATCH 09/37] override for replication slot name (#350) --- flow/activities/flowable.go | 17 +- flow/connectors/postgres/cdc.go | 10 +- flow/generated/protos/flow.pb.go | 776 +++++++++++++++--------------- flow/model/model.go | 2 + nexus/analyzer/src/lib.rs | 8 + nexus/flow-rs/src/grpc.rs | 2 + nexus/pt/src/flow_model.rs | 1 + nexus/pt/src/peerdb_flow.rs | 3 + nexus/pt/src/peerdb_flow.serde.rs | 18 + protos/flow.proto | 2 + 10 files changed, 447 insertions(+), 392 deletions(-) diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index 7cfc3beb3b..1a638f6a0a 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -186,14 +186,15 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, input *protos.StartFlo startTime := time.Now() records, err := src.PullRecords(&model.PullRecordsRequest{ - FlowJobName: input.FlowConnectionConfigs.FlowJobName, - SrcTableIDNameMapping: input.FlowConnectionConfigs.SrcTableIdNameMapping, - TableNameMapping: input.FlowConnectionConfigs.TableNameMapping, - LastSyncState: input.LastSyncState, - MaxBatchSize: uint32(input.SyncFlowOptions.BatchSize), - IdleTimeout: 10 * time.Second, - TableNameSchemaMapping: input.FlowConnectionConfigs.TableNameSchemaMapping, - OverridePublicationName: input.FlowConnectionConfigs.PublicationName, + FlowJobName: input.FlowConnectionConfigs.FlowJobName, + SrcTableIDNameMapping: input.FlowConnectionConfigs.SrcTableIdNameMapping, + TableNameMapping: input.FlowConnectionConfigs.TableNameMapping, + LastSyncState: input.LastSyncState, + MaxBatchSize: uint32(input.SyncFlowOptions.BatchSize), + IdleTimeout: 10 * time.Second, + TableNameSchemaMapping: input.FlowConnectionConfigs.TableNameSchemaMapping, + OverridePublicationName: input.FlowConnectionConfigs.PublicationName, + OverrideReplicationSlotName: input.FlowConnectionConfigs.ReplicationSlotName, }) if err != nil { return nil, fmt.Errorf("failed to pull records: %w", err) diff --git a/flow/connectors/postgres/cdc.go b/flow/connectors/postgres/cdc.go index 96d73a44ce..1418611add 100644 --- a/flow/connectors/postgres/cdc.go +++ b/flow/connectors/postgres/cdc.go @@ -68,6 +68,12 @@ func (p *PostgresCDCSource) PullRecords(req *model.PullRecordsRequest) (*model.R } replicationOpts := pglogrepl.StartReplicationOptions{PluginArgs: pluginArguments} + var replicationSlot string + if p.slot != "" && req.OverrideReplicationSlotName == "" { + replicationSlot = p.slot + } else { + replicationSlot = req.OverrideReplicationSlotName + } // create replication connection replicationConn, err := p.replPool.Acquire(p.ctx) @@ -94,11 +100,11 @@ func (p *PostgresCDCSource) PullRecords(req *model.PullRecordsRequest) (*model.R p.startLSN = pglogrepl.LSN(req.LastSyncState.Checkpoint + 1) } - err = pglogrepl.StartReplication(p.ctx, pgConn, p.slot, p.startLSN, replicationOpts) + err = pglogrepl.StartReplication(p.ctx, pgConn, replicationSlot, p.startLSN, replicationOpts) if err != nil { return nil, fmt.Errorf("error starting replication at startLsn - %d: %w", p.startLSN, err) } - log.Infof("started replication on slot %s at startLSN: %d", p.slot, p.startLSN) + log.Infof("started replication on slot %s at startLSN: %d", replicationSlot, p.startLSN) return p.consumeStream(pgConn, req, p.startLSN) } diff --git a/flow/generated/protos/flow.pb.go b/flow/generated/protos/flow.pb.go index 6e8dbda166..d82cb3f87e 100644 --- a/flow/generated/protos/flow.pb.go +++ b/flow/generated/protos/flow.pb.go @@ -195,7 +195,9 @@ type FlowConnectionConfigs struct { CdcSyncMode QRepSyncMode `protobuf:"varint,16,opt,name=cdc_sync_mode,json=cdcSyncMode,proto3,enum=peerdb_flow.QRepSyncMode" json:"cdc_sync_mode,omitempty"` SnapshotStagingPath string `protobuf:"bytes,17,opt,name=snapshot_staging_path,json=snapshotStagingPath,proto3" json:"snapshot_staging_path,omitempty"` CdcStagingPath string `protobuf:"bytes,18,opt,name=cdc_staging_path,json=cdcStagingPath,proto3" json:"cdc_staging_path,omitempty"` - SoftDelete bool `protobuf:"varint,19,opt,name=soft_delete,json=softDelete,proto3" json:"soft_delete,omitempty"` + // currently only works for snowflake + SoftDelete bool `protobuf:"varint,19,opt,name=soft_delete,json=softDelete,proto3" json:"soft_delete,omitempty"` + ReplicationSlotName string `protobuf:"bytes,20,opt,name=replication_slot_name,json=replicationSlotName,proto3" json:"replication_slot_name,omitempty"` } func (x *FlowConnectionConfigs) Reset() { @@ -363,6 +365,13 @@ func (x *FlowConnectionConfigs) GetSoftDelete() bool { return false } +func (x *FlowConnectionConfigs) GetReplicationSlotName() string { + if x != nil { + return x.ReplicationSlotName + } + return "" +} + type SyncFlowOptions struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -2372,7 +2381,7 @@ var file_flow_proto_rawDesc = []byte{ 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x34, 0x0a, 0x16, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x14, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xb2, 0x0b, + 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xe6, 0x0b, 0x0a, 0x15, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x12, 0x2a, 0x0a, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, @@ -2448,251 +2457,219 @@ var file_flow_proto_rawDesc = []byte{ 0x6e, 0x67, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x12, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x64, 0x63, 0x53, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x50, 0x61, 0x74, 0x68, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x6f, 0x66, 0x74, 0x5f, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x18, 0x13, 0x20, 0x01, - 0x28, 0x08, 0x52, 0x0a, 0x73, 0x6f, 0x66, 0x74, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x1a, 0x43, - 0x0a, 0x15, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, - 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, - 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, - 0x02, 0x38, 0x01, 0x1a, 0x48, 0x0a, 0x1a, 0x53, 0x72, 0x63, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, - 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, - 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x03, - 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x63, 0x0a, - 0x1b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, - 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, - 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2e, - 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, - 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, - 0x38, 0x01, 0x22, 0x30, 0x0a, 0x0f, 0x53, 0x79, 0x6e, 0x63, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, - 0x69, 0x7a, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x62, 0x61, 0x74, 0x63, 0x68, - 0x53, 0x69, 0x7a, 0x65, 0x22, 0x35, 0x0a, 0x14, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, - 0x65, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1d, 0x0a, 0x0a, - 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, - 0x52, 0x09, 0x62, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x22, 0x71, 0x0a, 0x0d, 0x4c, - 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1e, 0x0a, 0x0a, - 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, - 0x52, 0x0a, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x40, 0x0a, 0x0e, - 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x73, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, - 0x52, 0x0c, 0x6c, 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x41, 0x74, 0x22, 0xfa, - 0x01, 0x0a, 0x0e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x46, 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x70, 0x75, - 0x74, 0x12, 0x42, 0x0a, 0x0f, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x73, - 0x74, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, - 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x4c, 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, - 0x63, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x0d, 0x6c, 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, - 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x5a, 0x0a, 0x17, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x63, 0x6f, - 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, - 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x52, 0x15, 0x66, 0x6c, 0x6f, 0x77, - 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, - 0x73, 0x12, 0x48, 0x0a, 0x11, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6f, - 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, - 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x79, 0x6e, 0x63, 0x46, - 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x0f, 0x73, 0x79, 0x6e, 0x63, - 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x71, 0x0a, 0x13, 0x53, - 0x74, 0x61, 0x72, 0x74, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x49, 0x6e, 0x70, - 0x75, 0x74, 0x12, 0x5a, 0x0a, 0x17, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, - 0x77, 0x2e, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x52, 0x15, 0x66, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, - 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x22, 0x84, - 0x01, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x65, 0x64, - 0x49, 0x44, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, - 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, - 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, - 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, - 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, - 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xbe, 0x01, 0x0a, 0x16, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, - 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x49, 0x6e, 0x70, 0x75, 0x74, - 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, - 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, - 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x36, - 0x0a, 0x17, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, - 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x15, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, - 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x22, 0xc5, 0x01, 0x0a, 0x1b, 0x45, 0x6e, 0x73, 0x75, 0x72, - 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x42, 0x61, 0x74, 0x63, - 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, - 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, - 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, - 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, - 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, - 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, - 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x38, 0x0a, 0x18, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, - 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x16, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, 0x22, 0x30, - 0x0a, 0x17, 0x50, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, - 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x15, 0x0a, 0x06, 0x72, 0x65, 0x6c, - 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x72, 0x65, 0x6c, 0x49, 0x64, - 0x22, 0x89, 0x01, 0x0a, 0x0f, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, - 0x66, 0x69, 0x65, 0x72, 0x12, 0x62, 0x0a, 0x19, 0x70, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, - 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, - 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, - 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x50, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x48, 0x00, 0x52, - 0x17, 0x70, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, - 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x42, 0x12, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x22, 0x62, 0x0a, 0x17, - 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, - 0x79, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x47, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, - 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, - 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, - 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, - 0x22, 0x88, 0x02, 0x0a, 0x1c, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, - 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, - 0x74, 0x12, 0x7f, 0x0a, 0x18, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, - 0x69, 0x66, 0x69, 0x65, 0x72, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, - 0x03, 0x28, 0x0b, 0x32, 0x45, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, - 0x77, 0x2e, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, 0x6c, - 0x69, 0x74, 0x79, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x2e, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x4d, 0x61, - 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x16, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x4d, 0x61, 0x70, 0x70, 0x69, - 0x6e, 0x67, 0x1a, 0x67, 0x0a, 0x1b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, - 0x69, 0x66, 0x69, 0x65, 0x72, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, - 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, - 0x6b, 0x65, 0x79, 0x12, 0x32, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, - 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, - 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x99, 0x03, 0x0a, 0x15, - 0x53, 0x65, 0x74, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, - 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, - 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, - 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, - 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, - 0x61, 0x6d, 0x65, 0x12, 0x66, 0x0a, 0x12, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, - 0x65, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, - 0x38, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x65, - 0x74, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, - 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, - 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, - 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x12, 0x3d, 0x0a, 0x10, 0x64, - 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, - 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, - 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0f, 0x64, 0x65, 0x73, 0x74, 0x69, - 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x65, 0x65, 0x72, 0x12, 0x26, 0x0a, 0x0f, 0x64, 0x6f, - 0x5f, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x63, 0x6f, 0x70, 0x79, 0x18, 0x05, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x0d, 0x64, 0x6f, 0x49, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x43, 0x6f, - 0x70, 0x79, 0x1a, 0x43, 0x0a, 0x15, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, + 0x28, 0x08, 0x52, 0x0a, 0x73, 0x6f, 0x66, 0x74, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x12, 0x32, + 0x0a, 0x15, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x6c, + 0x6f, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x14, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x72, + 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x6c, 0x6f, 0x74, 0x4e, 0x61, + 0x6d, 0x65, 0x1a, 0x43, 0x0a, 0x15, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x5a, 0x0a, 0x16, 0x53, 0x65, 0x74, 0x75, 0x70, - 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x75, 0x74, 0x70, 0x75, - 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x6c, 0x6f, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x73, 0x6c, 0x6f, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x23, - 0x0a, 0x0d, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x4e, - 0x61, 0x6d, 0x65, 0x22, 0xed, 0x02, 0x0a, 0x13, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x61, - 0x77, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, + 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x48, 0x0a, 0x1a, 0x53, 0x72, 0x63, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x49, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0d, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, + 0x01, 0x1a, 0x63, 0x0a, 0x1b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, + 0x65, 0x79, 0x12, 0x2e, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x30, 0x0a, 0x0f, 0x53, 0x79, 0x6e, 0x63, 0x46, 0x6c, + 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x62, 0x61, 0x74, + 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x62, + 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x22, 0x35, 0x0a, 0x14, 0x4e, 0x6f, 0x72, 0x6d, + 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x12, 0x1d, 0x0a, 0x0a, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x62, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x22, + 0x71, 0x0a, 0x0d, 0x4c, 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, + 0x12, 0x40, 0x0a, 0x0e, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x73, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x5f, + 0x61, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, + 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0c, 0x6c, 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x65, 0x64, + 0x41, 0x74, 0x22, 0xfa, 0x01, 0x0a, 0x0e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x46, 0x6c, 0x6f, 0x77, + 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x42, 0x0a, 0x0f, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x73, 0x79, + 0x6e, 0x63, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, + 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x4c, 0x61, 0x73, + 0x74, 0x53, 0x79, 0x6e, 0x63, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x0d, 0x6c, 0x61, 0x73, 0x74, + 0x53, 0x79, 0x6e, 0x63, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x5a, 0x0a, 0x17, 0x66, 0x6c, 0x6f, + 0x77, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x65, 0x65, + 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, + 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x52, 0x15, + 0x66, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x73, 0x12, 0x48, 0x0a, 0x11, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x66, 0x6c, + 0x6f, 0x77, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, + 0x79, 0x6e, 0x63, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x0f, + 0x73, 0x79, 0x6e, 0x63, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, + 0x71, 0x0a, 0x13, 0x53, 0x74, 0x61, 0x72, 0x74, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, + 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x5a, 0x0a, 0x17, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x63, + 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, + 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x52, 0x15, 0x66, 0x6c, 0x6f, + 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x73, 0x22, 0x84, 0x01, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x53, 0x79, + 0x6e, 0x63, 0x65, 0x64, 0x49, 0x44, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, - 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x64, 0x0a, 0x12, 0x74, 0x61, 0x62, - 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, - 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, - 0x6c, 0x6f, 0x77, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x61, 0x77, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, - 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x10, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x12, - 0x3d, 0x0a, 0x0d, 0x63, 0x64, 0x63, 0x5f, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6d, 0x6f, 0x64, 0x65, - 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, - 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, - 0x65, 0x52, 0x0b, 0x63, 0x64, 0x63, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x1a, 0x43, - 0x0a, 0x15, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, - 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, - 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, - 0x02, 0x38, 0x01, 0x22, 0x41, 0x0a, 0x14, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x61, 0x77, - 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x29, 0x0a, 0x10, 0x74, + 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xbe, 0x01, 0x0a, 0x16, 0x45, 0x6e, + 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x49, + 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, + 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, + 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, + 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, + 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, + 0x6d, 0x65, 0x12, 0x36, 0x0a, 0x17, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x15, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x22, 0xc5, 0x01, 0x0a, 0x1b, 0x45, + 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, + 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, + 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, + 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, + 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, + 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, + 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x38, 0x0a, 0x18, 0x73, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, + 0x69, 0x65, 0x72, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x16, 0x73, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, + 0x72, 0x73, 0x22, 0x30, 0x0a, 0x17, 0x50, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x15, 0x0a, + 0x06, 0x72, 0x65, 0x6c, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x72, + 0x65, 0x6c, 0x49, 0x64, 0x22, 0x89, 0x01, 0x0a, 0x0f, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, + 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x62, 0x0a, 0x19, 0x70, 0x6f, 0x73, 0x74, + 0x67, 0x72, 0x65, 0x73, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, + 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x70, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x50, 0x6f, 0x73, 0x74, 0x67, 0x72, + 0x65, 0x73, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, + 0x72, 0x48, 0x00, 0x52, 0x17, 0x70, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x42, 0x12, 0x0a, 0x10, + 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, + 0x22, 0x62, 0x0a, 0x17, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, + 0x69, 0x6c, 0x69, 0x74, 0x79, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x47, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, - 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x22, 0xe3, 0x01, 0x0a, 0x0b, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x29, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, - 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, - 0x72, 0x12, 0x3f, 0x0a, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, - 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x43, 0x6f, 0x6c, - 0x75, 0x6d, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, - 0x6e, 0x73, 0x12, 0x2c, 0x0a, 0x12, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x5f, 0x6b, 0x65, - 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, - 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x4b, 0x65, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, - 0x1a, 0x3a, 0x0a, 0x0c, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, - 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, - 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x91, 0x01, 0x0a, - 0x18, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, - 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, + 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, + 0x69, 0x65, 0x72, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, + 0x66, 0x69, 0x65, 0x72, 0x22, 0x88, 0x02, 0x0a, 0x1c, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, + 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, + 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x7f, 0x0a, 0x18, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, + 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, + 0x67, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x45, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, + 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, + 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, + 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, + 0x65, 0x72, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x16, + 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x4d, + 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x67, 0x0a, 0x1b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, + 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x32, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, + 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, + 0x66, 0x69, 0x65, 0x72, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, + 0x99, 0x03, 0x0a, 0x15, 0x53, 0x65, 0x74, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, - 0x66, 0x69, 0x67, 0x12, 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, + 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, + 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, + 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x66, 0x0a, 0x12, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x03, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, + 0x77, 0x2e, 0x53, 0x65, 0x74, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, + 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x10, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x12, + 0x3d, 0x0a, 0x10, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, + 0x65, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, + 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0f, 0x64, + 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x65, 0x65, 0x72, 0x12, 0x26, + 0x0a, 0x0f, 0x64, 0x6f, 0x5f, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x63, 0x6f, 0x70, + 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x64, 0x6f, 0x49, 0x6e, 0x69, 0x74, 0x69, + 0x61, 0x6c, 0x43, 0x6f, 0x70, 0x79, 0x1a, 0x43, 0x0a, 0x15, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, + 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, + 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, + 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x5a, 0x0a, 0x16, 0x53, + 0x65, 0x74, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4f, + 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x6c, 0x6f, 0x74, 0x5f, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x73, 0x6c, 0x6f, 0x74, 0x4e, 0x61, + 0x6d, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x6e, 0x61, 0x70, 0x73, + 0x68, 0x6f, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xed, 0x02, 0x0a, 0x13, 0x43, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x52, 0x61, 0x77, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, + 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, + 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, + 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x64, 0x0a, + 0x12, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x6d, 0x61, 0x70, 0x70, + 0x69, 0x6e, 0x67, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x70, 0x65, 0x65, 0x72, + 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x61, + 0x77, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x52, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, + 0x69, 0x6e, 0x67, 0x12, 0x3d, 0x0a, 0x0d, 0x63, 0x64, 0x63, 0x5f, 0x73, 0x79, 0x6e, 0x63, 0x5f, + 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x65, 0x65, + 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x53, 0x79, 0x6e, + 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x0b, 0x63, 0x64, 0x63, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, + 0x64, 0x65, 0x1a, 0x43, 0x0a, 0x15, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, + 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, + 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x41, 0x0a, 0x14, 0x43, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x52, 0x61, 0x77, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, + 0x29, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, + 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x22, 0xe3, 0x01, 0x0a, 0x0b, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x29, 0x0a, 0x10, 0x74, 0x61, + 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, - 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 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, + 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x3f, 0x0a, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, + 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, + 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x07, 0x63, + 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x2c, 0x0a, 0x12, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, + 0x79, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x10, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x4b, 0x65, 0x79, 0x43, 0x6f, + 0x6c, 0x75, 0x6d, 0x6e, 0x1a, 0x3a, 0x0a, 0x0c, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, + 0x22, 0x91, 0x01, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, + 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, + 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, + 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x2b, 0x0a, 0x11, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, 0x18, 0x02, 0x20, 0x03, + 0x28, 0x09, 0x52, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, + 0x69, 0x65, 0x72, 0x73, 0x22, 0xff, 0x01, 0x0a, 0x19, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, + 0x75, 0x74, 0x12, 0x7d, 0x0a, 0x19, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, + 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x42, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, + 0x6c, 0x6f, 0x77, 0x2e, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x16, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, @@ -2702,158 +2679,193 @@ var file_flow_proto_rawDesc = []byte{ 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, + 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, 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, 0x12, 0x76, 0x0a, 0x14, 0x74, 0x61, - 0x62, 0x6c, 0x65, 0x5f, 0x65, 0x78, 0x69, 0x73, 0x74, 0x73, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, - 0x6e, 0x67, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x44, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, - 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, - 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, 0x73, - 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x12, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, - 0x6e, 0x67, 0x1a, 0x45, 0x0a, 0x17, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, 0x73, 0x74, - 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, - 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, - 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, - 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x3b, 0x0a, 0x11, 0x49, 0x6e, 0x74, - 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x14, - 0x0a, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x73, - 0x74, 0x61, 0x72, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x03, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0x79, 0x0a, 0x17, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, - 0x61, 0x6d, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, - 0x65, 0x12, 0x30, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, - 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x05, 0x73, 0x74, - 0x61, 0x72, 0x74, 0x12, 0x2c, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, - 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x03, 0x65, 0x6e, - 0x64, 0x22, 0x4d, 0x0a, 0x03, 0x54, 0x49, 0x44, 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x6c, 0x6f, 0x63, - 0x6b, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0b, - 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x6f, - 0x66, 0x66, 0x73, 0x65, 0x74, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x0d, 0x52, 0x0c, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, - 0x22, 0x5f, 0x0a, 0x11, 0x54, 0x49, 0x44, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x26, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, - 0x6f, 0x77, 0x2e, 0x54, 0x49, 0x44, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x22, 0x0a, - 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x70, 0x65, 0x65, - 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x49, 0x44, 0x52, 0x03, 0x65, 0x6e, - 0x64, 0x22, 0xe8, 0x01, 0x0a, 0x0e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, - 0x61, 0x6e, 0x67, 0x65, 0x12, 0x3d, 0x0a, 0x09, 0x69, 0x6e, 0x74, 0x5f, 0x72, 0x61, 0x6e, 0x67, - 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, - 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x49, 0x6e, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x48, 0x00, 0x52, 0x08, 0x69, 0x6e, 0x74, 0x52, 0x61, - 0x6e, 0x67, 0x65, 0x12, 0x4f, 0x0a, 0x0f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, - 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x70, - 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, - 0x74, 0x61, 0x6d, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, - 0x67, 0x65, 0x48, 0x00, 0x52, 0x0e, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, - 0x61, 0x6e, 0x67, 0x65, 0x12, 0x3d, 0x0a, 0x09, 0x74, 0x69, 0x64, 0x5f, 0x72, 0x61, 0x6e, 0x67, - 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, - 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x49, 0x44, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x48, 0x00, 0x52, 0x08, 0x74, 0x69, 0x64, 0x52, 0x61, - 0x6e, 0x67, 0x65, 0x42, 0x07, 0x0a, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x22, 0x78, 0x0a, 0x0d, - 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x39, 0x0a, - 0x0a, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0e, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, - 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x77, - 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x2c, 0x0a, 0x12, 0x75, 0x70, 0x73, 0x65, - 0x72, 0x74, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x02, - 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x4b, 0x65, 0x79, 0x43, - 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x22, 0x96, 0x06, 0x0a, 0x0a, 0x51, 0x52, 0x65, 0x70, 0x43, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, - 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, - 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x33, 0x0a, 0x0b, 0x73, 0x6f, 0x75, - 0x72, 0x63, 0x65, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, - 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, - 0x65, 0x72, 0x52, 0x0a, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x65, 0x65, 0x72, 0x12, 0x3d, - 0x0a, 0x10, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x65, - 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0f, 0x64, 0x65, - 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x65, 0x65, 0x72, 0x12, 0x40, 0x0a, - 0x1c, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x61, 0x62, - 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x04, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x1a, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, - 0x14, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, - 0x71, 0x75, 0x65, 0x72, 0x79, 0x12, 0x27, 0x0a, 0x0f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, - 0x72, 0x6b, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, - 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x29, - 0x0a, 0x10, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x63, 0x6f, 0x6c, 0x75, - 0x6d, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, - 0x61, 0x72, 0x6b, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x2a, 0x0a, 0x11, 0x69, 0x6e, 0x69, - 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x63, 0x6f, 0x70, 0x79, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x08, - 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x43, 0x6f, 0x70, - 0x79, 0x4f, 0x6e, 0x6c, 0x79, 0x12, 0x36, 0x0a, 0x09, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6d, 0x6f, - 0x64, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x53, 0x79, 0x6e, 0x63, 0x4d, - 0x6f, 0x64, 0x65, 0x52, 0x08, 0x73, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x24, 0x0a, - 0x0e, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, 0x69, 0x6e, 0x74, 0x18, - 0x0a, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0c, 0x62, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, - 0x49, 0x6e, 0x74, 0x12, 0x34, 0x0a, 0x16, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x64, 0x75, 0x72, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x0b, 0x20, - 0x01, 0x28, 0x0d, 0x52, 0x14, 0x62, 0x61, 0x74, 0x63, 0x68, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x30, 0x0a, 0x14, 0x6d, 0x61, 0x78, - 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, - 0x73, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x12, 0x6d, 0x61, 0x78, 0x50, 0x61, 0x72, 0x61, - 0x6c, 0x6c, 0x65, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73, 0x12, 0x3f, 0x0a, 0x1c, 0x77, - 0x61, 0x69, 0x74, 0x5f, 0x62, 0x65, 0x74, 0x77, 0x65, 0x65, 0x6e, 0x5f, 0x62, 0x61, 0x74, 0x63, - 0x68, 0x65, 0x73, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, - 0x0d, 0x52, 0x19, 0x77, 0x61, 0x69, 0x74, 0x42, 0x65, 0x74, 0x77, 0x65, 0x65, 0x6e, 0x42, 0x61, - 0x74, 0x63, 0x68, 0x65, 0x73, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x39, 0x0a, 0x0a, - 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, - 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x09, 0x77, 0x72, - 0x69, 0x74, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x74, 0x61, 0x67, 0x69, - 0x6e, 0x67, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, - 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x50, 0x61, 0x74, 0x68, 0x12, 0x33, 0x0a, 0x16, 0x6e, 0x75, - 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x70, 0x65, 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x13, 0x6e, 0x75, 0x6d, 0x52, - 0x6f, 0x77, 0x73, 0x50, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, - 0x97, 0x01, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, - 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x31, 0x0a, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, - 0x77, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, - 0x52, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x30, 0x0a, 0x14, 0x66, 0x75, 0x6c, 0x6c, 0x5f, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, - 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x66, 0x75, 0x6c, 0x6c, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 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, 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, 0x47, 0x0a, 0x0d, 0x51, - 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1a, 0x0a, 0x16, + 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x52, 0x12, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x4d, + 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x45, 0x0a, 0x17, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x45, + 0x78, 0x69, 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, + 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x3b, 0x0a, + 0x11, 0x49, 0x6e, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, + 0x67, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0x79, 0x0a, 0x17, 0x54, 0x69, + 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x30, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, + 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x2c, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, + 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0x4d, 0x0a, 0x03, 0x54, 0x49, 0x44, 0x12, 0x21, 0x0a, 0x0c, + 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0d, 0x52, 0x0b, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, + 0x23, 0x0a, 0x0d, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0c, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x4e, 0x75, + 0x6d, 0x62, 0x65, 0x72, 0x22, 0x5f, 0x0a, 0x11, 0x54, 0x49, 0x44, 0x50, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x26, 0x0a, 0x05, 0x73, 0x74, 0x61, + 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, + 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x49, 0x44, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, + 0x74, 0x12, 0x22, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, + 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x49, 0x44, + 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0xe8, 0x01, 0x0a, 0x0e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x3d, 0x0a, 0x09, 0x69, 0x6e, 0x74, 0x5f, + 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x49, 0x6e, 0x74, 0x50, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x48, 0x00, 0x52, 0x08, 0x69, + 0x6e, 0x74, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x4f, 0x0a, 0x0f, 0x74, 0x69, 0x6d, 0x65, 0x73, + 0x74, 0x61, 0x6d, 0x70, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x24, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, + 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x48, 0x00, 0x52, 0x0e, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, + 0x61, 0x6d, 0x70, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x3d, 0x0a, 0x09, 0x74, 0x69, 0x64, 0x5f, + 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x49, 0x44, 0x50, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x48, 0x00, 0x52, 0x08, 0x74, + 0x69, 0x64, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x42, 0x07, 0x0a, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, + 0x22, 0x78, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4d, 0x6f, 0x64, + 0x65, 0x12, 0x39, 0x0a, 0x0a, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, + 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, + 0x65, 0x52, 0x09, 0x77, 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x2c, 0x0a, 0x12, + 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, + 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, + 0x4b, 0x65, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x22, 0x96, 0x06, 0x0a, 0x0a, 0x51, + 0x52, 0x65, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, + 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x33, 0x0a, + 0x0b, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, + 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0a, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x65, + 0x65, 0x72, 0x12, 0x3d, 0x0a, 0x10, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, + 0x52, 0x0f, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x65, 0x65, + 0x72, 0x12, 0x40, 0x0a, 0x1c, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, + 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1a, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, + 0x69, 0x65, 0x72, 0x12, 0x14, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x12, 0x27, 0x0a, 0x0f, 0x77, 0x61, 0x74, + 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0e, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x5f, + 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x77, 0x61, + 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x2a, 0x0a, + 0x11, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x63, 0x6f, 0x70, 0x79, 0x5f, 0x6f, 0x6e, + 0x6c, 0x79, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, + 0x6c, 0x43, 0x6f, 0x70, 0x79, 0x4f, 0x6e, 0x6c, 0x79, 0x12, 0x36, 0x0a, 0x09, 0x73, 0x79, 0x6e, + 0x63, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x53, + 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x08, 0x73, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, + 0x65, 0x12, 0x24, 0x0a, 0x0e, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, + 0x69, 0x6e, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0c, 0x62, 0x61, 0x74, 0x63, 0x68, + 0x53, 0x69, 0x7a, 0x65, 0x49, 0x6e, 0x74, 0x12, 0x34, 0x0a, 0x16, 0x62, 0x61, 0x74, 0x63, 0x68, + 0x5f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, + 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x14, 0x62, 0x61, 0x74, 0x63, 0x68, 0x44, 0x75, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x30, 0x0a, + 0x14, 0x6d, 0x61, 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x5f, 0x77, 0x6f, + 0x72, 0x6b, 0x65, 0x72, 0x73, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x12, 0x6d, 0x61, 0x78, + 0x50, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73, 0x12, + 0x3f, 0x0a, 0x1c, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x62, 0x65, 0x74, 0x77, 0x65, 0x65, 0x6e, 0x5f, + 0x62, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, + 0x0d, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x19, 0x77, 0x61, 0x69, 0x74, 0x42, 0x65, 0x74, 0x77, 0x65, + 0x65, 0x6e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, + 0x12, 0x39, 0x0a, 0x0a, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x0e, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, + 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4d, 0x6f, 0x64, 0x65, + 0x52, 0x09, 0x77, 0x72, 0x69, 0x74, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x73, + 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x0f, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0b, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x50, 0x61, 0x74, 0x68, 0x12, 0x33, + 0x0a, 0x16, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x70, 0x65, 0x72, 0x5f, 0x70, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x13, + 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x50, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x22, 0x97, 0x01, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x70, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x31, 0x0a, 0x05, 0x72, 0x61, 0x6e, 0x67, + 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, + 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x61, 0x6e, 0x67, 0x65, 0x52, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x30, 0x0a, 0x14, 0x66, + 0x75, 0x6c, 0x6c, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x66, 0x75, 0x6c, 0x6c, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 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, 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, + 0x47, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, + 0x12, 0x1a, 0x0a, 0x16, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x5f, 0x4d, + 0x4f, 0x44, 0x45, 0x5f, 0x41, 0x50, 0x50, 0x45, 0x4e, 0x44, 0x10, 0x00, 0x12, 0x1a, 0x0a, 0x16, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, - 0x41, 0x50, 0x50, 0x45, 0x4e, 0x44, 0x10, 0x00, 0x12, 0x1a, 0x0a, 0x16, 0x51, 0x52, 0x45, 0x50, - 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x55, 0x50, 0x53, 0x45, - 0x52, 0x54, 0x10, 0x01, 0x42, 0x76, 0x0a, 0x0f, 0x63, 0x6f, 0x6d, 0x2e, 0x70, 0x65, 0x65, 0x72, - 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x42, 0x09, 0x46, 0x6c, 0x6f, 0x77, 0x50, 0x72, 0x6f, - 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x10, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x64, 0x2f, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x73, 0xa2, 0x02, 0x03, 0x50, 0x58, 0x58, 0xaa, 0x02, 0x0a, 0x50, - 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0xca, 0x02, 0x0a, 0x50, 0x65, 0x65, 0x72, - 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0xe2, 0x02, 0x16, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, - 0x6c, 0x6f, 0x77, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, - 0x02, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0x62, 0x06, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x33, + 0x55, 0x50, 0x53, 0x45, 0x52, 0x54, 0x10, 0x01, 0x42, 0x76, 0x0a, 0x0f, 0x63, 0x6f, 0x6d, 0x2e, + 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x42, 0x09, 0x46, 0x6c, 0x6f, + 0x77, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x10, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, + 0x74, 0x65, 0x64, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x73, 0xa2, 0x02, 0x03, 0x50, 0x58, 0x58, + 0xaa, 0x02, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0xca, 0x02, 0x0a, + 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0xe2, 0x02, 0x16, 0x50, 0x65, 0x65, + 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, + 0x61, 0x74, 0x61, 0xea, 0x02, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, + 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( diff --git a/flow/model/model.go b/flow/model/model.go index ca5fae696d..d87314ceef 100644 --- a/flow/model/model.go +++ b/flow/model/model.go @@ -26,6 +26,8 @@ type PullRecordsRequest struct { TableNameSchemaMapping map[string]*protos.TableSchema // override publication name OverridePublicationName string + // override replication slot name + OverrideReplicationSlotName string } type Record interface { diff --git a/nexus/analyzer/src/lib.rs b/nexus/analyzer/src/lib.rs index dd5c11e68f..70dae94f9d 100644 --- a/nexus/analyzer/src/lib.rs +++ b/nexus/analyzer/src/lib.rs @@ -170,6 +170,13 @@ impl StatementAnalyzer for PeerDDLAnalyzer { _ => None, }; + let replication_slot_name: Option = match raw_options + .remove("replication_slot_name") + { + Some(sqlparser::ast::Value::SingleQuotedString(s)) => Some(s.clone()), + _ => None, + }; + let snapshot_num_rows_per_partition: Option = match raw_options .remove("snapshot_num_rows_per_partition") { @@ -239,6 +246,7 @@ impl StatementAnalyzer for PeerDDLAnalyzer { cdc_sync_mode, cdc_staging_path, soft_delete, + replication_slot_name }; // Error reporting diff --git a/nexus/flow-rs/src/grpc.rs b/nexus/flow-rs/src/grpc.rs index c5bd7fde45..a749b22cb7 100644 --- a/nexus/flow-rs/src/grpc.rs +++ b/nexus/flow-rs/src/grpc.rs @@ -133,6 +133,7 @@ impl FlowGrpcClient { let do_initial_copy = job.do_initial_copy; let publication_name = job.publication_name.clone(); + let replication_slot_name = job.replication_slot_name.clone(); let snapshot_num_rows_per_partition = job.snapshot_num_rows_per_partition; let snapshot_max_parallel_workers = job.snapshot_max_parallel_workers; let snapshot_num_tables_in_parallel = job.snapshot_num_tables_in_parallel; @@ -160,6 +161,7 @@ impl FlowGrpcClient { .unwrap_or(0), cdc_staging_path: job.cdc_staging_path.clone().unwrap_or_default(), soft_delete: job.soft_delete, + replication_slot_name: replication_slot_name.unwrap_or_default(), ..Default::default() }; diff --git a/nexus/pt/src/flow_model.rs b/nexus/pt/src/flow_model.rs index 7fa5651cc9..74c2234824 100644 --- a/nexus/pt/src/flow_model.rs +++ b/nexus/pt/src/flow_model.rs @@ -72,6 +72,7 @@ pub struct FlowJob { pub cdc_sync_mode: Option, pub cdc_staging_path: Option, pub soft_delete: bool, + pub replication_slot_name: Option } #[derive(Debug, PartialEq, Eq, Serialize, Deserialize, Clone)] diff --git a/nexus/pt/src/peerdb_flow.rs b/nexus/pt/src/peerdb_flow.rs index 399850b3ce..2afeb242b3 100644 --- a/nexus/pt/src/peerdb_flow.rs +++ b/nexus/pt/src/peerdb_flow.rs @@ -49,8 +49,11 @@ pub struct FlowConnectionConfigs { pub snapshot_staging_path: ::prost::alloc::string::String, #[prost(string, tag="18")] pub cdc_staging_path: ::prost::alloc::string::String, + /// currently only works for snowflake #[prost(bool, tag="19")] pub soft_delete: bool, + #[prost(string, tag="20")] + pub replication_slot_name: ::prost::alloc::string::String, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] diff --git a/nexus/pt/src/peerdb_flow.serde.rs b/nexus/pt/src/peerdb_flow.serde.rs index 621a866654..0665a74023 100644 --- a/nexus/pt/src/peerdb_flow.serde.rs +++ b/nexus/pt/src/peerdb_flow.serde.rs @@ -868,6 +868,9 @@ impl serde::Serialize for FlowConnectionConfigs { if self.soft_delete { len += 1; } + if !self.replication_slot_name.is_empty() { + len += 1; + } let mut struct_ser = serializer.serialize_struct("peerdb_flow.FlowConnectionConfigs", len)?; if let Some(v) = self.source.as_ref() { struct_ser.serialize_field("source", v)?; @@ -930,6 +933,9 @@ impl serde::Serialize for FlowConnectionConfigs { if self.soft_delete { struct_ser.serialize_field("softDelete", &self.soft_delete)?; } + if !self.replication_slot_name.is_empty() { + struct_ser.serialize_field("replicationSlotName", &self.replication_slot_name)?; + } struct_ser.end() } } @@ -976,6 +982,8 @@ impl<'de> serde::Deserialize<'de> for FlowConnectionConfigs { "cdcStagingPath", "soft_delete", "softDelete", + "replication_slot_name", + "replicationSlotName", ]; #[allow(clippy::enum_variant_names)] @@ -999,6 +1007,7 @@ impl<'de> serde::Deserialize<'de> for FlowConnectionConfigs { SnapshotStagingPath, CdcStagingPath, SoftDelete, + ReplicationSlotName, __SkipField__, } impl<'de> serde::Deserialize<'de> for GeneratedField { @@ -1040,6 +1049,7 @@ impl<'de> serde::Deserialize<'de> for FlowConnectionConfigs { "snapshotStagingPath" | "snapshot_staging_path" => Ok(GeneratedField::SnapshotStagingPath), "cdcStagingPath" | "cdc_staging_path" => Ok(GeneratedField::CdcStagingPath), "softDelete" | "soft_delete" => Ok(GeneratedField::SoftDelete), + "replicationSlotName" | "replication_slot_name" => Ok(GeneratedField::ReplicationSlotName), _ => Ok(GeneratedField::__SkipField__), } } @@ -1078,6 +1088,7 @@ impl<'de> serde::Deserialize<'de> for FlowConnectionConfigs { let mut snapshot_staging_path__ = None; let mut cdc_staging_path__ = None; let mut soft_delete__ = None; + let mut replication_slot_name__ = None; while let Some(k) = map.next_key()? { match k { GeneratedField::Source => { @@ -1209,6 +1220,12 @@ impl<'de> serde::Deserialize<'de> for FlowConnectionConfigs { } soft_delete__ = Some(map.next_value()?); } + GeneratedField::ReplicationSlotName => { + if replication_slot_name__.is_some() { + return Err(serde::de::Error::duplicate_field("replicationSlotName")); + } + replication_slot_name__ = Some(map.next_value()?); + } GeneratedField::__SkipField__ => { let _ = map.next_value::()?; } @@ -1234,6 +1251,7 @@ impl<'de> serde::Deserialize<'de> for FlowConnectionConfigs { snapshot_staging_path: snapshot_staging_path__.unwrap_or_default(), cdc_staging_path: cdc_staging_path__.unwrap_or_default(), soft_delete: soft_delete__.unwrap_or_default(), + replication_slot_name: replication_slot_name__.unwrap_or_default(), }) } } diff --git a/protos/flow.proto b/protos/flow.proto index 75ea687fa5..1066fe4fe0 100644 --- a/protos/flow.proto +++ b/protos/flow.proto @@ -38,6 +38,8 @@ message FlowConnectionConfigs { // currently only works for snowflake bool soft_delete = 19; + + string replication_slot_name = 20; } message SyncFlowOptions { int32 batch_size = 1; } From 0bca9e3d3bb4df6bc7027b010247a729eaa7f538 Mon Sep 17 00:00:00 2001 From: Kevin K Biju <52661649+heavycrystal@users.noreply.github.com> Date: Fri, 25 Aug 2023 01:24:34 +0530 Subject: [PATCH 10/37] added monitoring of QRep mirror status, dumped to catalog (#342) --- flow/activities/flowable.go | 50 +++++++- flow/connectors/postgres/qrep.go | 14 +-- .../connectors/utils/monitoring/monitoring.go | 117 +++++++++++++++++- flow/workflows/qrep_flow.go | 21 ++-- .../migrations/V6__monitoring_refine_1.sql | 5 + 5 files changed, 185 insertions(+), 22 deletions(-) create mode 100644 nexus/catalog/migrations/V6__monitoring_refine_1.sql diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index 1a638f6a0a..776b43c9fd 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -328,6 +328,7 @@ func (a *FlowableActivity) SetupQRepMetadataTables(ctx context.Context, config * func (a *FlowableActivity) GetQRepPartitions(ctx context.Context, config *protos.QRepConfig, last *protos.QRepPartition, + runUUID string, ) (*protos.QRepParitionResult, error) { conn, err := connectors.GetConnector(ctx, config.SourcePeer) if err != nil { @@ -343,10 +344,18 @@ func (a *FlowableActivity) GetQRepPartitions(ctx context.Context, shutdown <- true }() + startTime := time.Now() partitions, err := conn.GetQRepPartitions(config, last) if err != nil { return nil, fmt.Errorf("failed to get partitions from source: %w", err) } + if len(partitions) > 0 { + err = a.CatalogMirrorMonitor.InitializeQRepRun(ctx, config.FlowJobName, + runUUID, startTime) + if err != nil { + return nil, err + } + } return &protos.QRepParitionResult{ Partitions: partitions, @@ -357,6 +366,7 @@ func (a *FlowableActivity) GetQRepPartitions(ctx context.Context, func (a *FlowableActivity) ReplicateQRepPartition(ctx context.Context, config *protos.QRepConfig, partition *protos.QRepPartition, + runUUID string, ) error { ctx = context.WithValue(ctx, shared.EnableMetricsKey, a.EnableMetrics) srcConn, err := connectors.GetConnector(ctx, config.SourcePeer) @@ -376,18 +386,30 @@ func (a *FlowableActivity) ReplicateQRepPartition(ctx context.Context, var stream *model.QRecordStream bufferSize := shared.FetchAndChannelSize var wg sync.WaitGroup + var numRecords int64 + + err = a.CatalogMirrorMonitor.AddPartitionToQRepRun(ctx, config.FlowJobName, runUUID, partition) + if err != nil { + return err + } + var goroutineErr error = nil if config.SourcePeer.Type == protos.DBType_POSTGRES { stream = model.NewQRecordStream(bufferSize) wg.Add(1) pullPgRecords := func() { pgConn := srcConn.(*connpostgres.PostgresConnector) - err = pgConn.PullQRepRecordStream(config, partition, stream) + tmp, err := pgConn.PullQRepRecordStream(config, partition, stream) + numRecords = int64(tmp) if err != nil { log.Errorf("failed to pull records: %v", err) - return + goroutineErr = err + } + err = a.CatalogMirrorMonitor.UpdatePullEndTimeAndRowsForPartition(ctx, runUUID, partition, numRecords) + if err != nil { + log.Errorf("%v", err) + goroutineErr = err } - wg.Done() } @@ -397,9 +419,14 @@ func (a *FlowableActivity) ReplicateQRepPartition(ctx context.Context, if err != nil { return fmt.Errorf("failed to pull records: %w", err) } - + numRecords = int64(recordBatch.NumRecords) log.Printf("pulled %d records\n", len(recordBatch.Records)) + err = a.CatalogMirrorMonitor.UpdatePullEndTimeAndRowsForPartition(ctx, runUUID, partition, numRecords) + if err != nil { + return err + } + stream, err = recordBatch.ToQRecordStream(bufferSize) if err != nil { return fmt.Errorf("failed to convert to qrecord stream: %w", err) @@ -425,11 +452,20 @@ func (a *FlowableActivity) ReplicateQRepPartition(ctx context.Context, } wg.Wait() + if goroutineErr != nil { + return goroutineErr + } log.Printf("pushed %d records\n", res) + err = a.CatalogMirrorMonitor.UpdateEndTimeForPartition(ctx, runUUID, partition) + if err != nil { + return err + } + return nil } -func (a *FlowableActivity) ConsolidateQRepPartitions(ctx context.Context, config *protos.QRepConfig) error { +func (a *FlowableActivity) ConsolidateQRepPartitions(ctx context.Context, config *protos.QRepConfig, + runUUID string) error { ctx = context.WithValue(ctx, shared.EnableMetricsKey, a.EnableMetrics) dst, err := connectors.GetConnector(ctx, config.DestinationPeer) if err != nil { @@ -445,6 +481,10 @@ func (a *FlowableActivity) ConsolidateQRepPartitions(ctx context.Context, config }() err = dst.ConsolidateQRepPartitions(config) + if err != nil { + return err + } + err = a.CatalogMirrorMonitor.UpdateEndTimeForQRepRun(ctx, runUUID) return err } diff --git a/flow/connectors/postgres/qrep.go b/flow/connectors/postgres/qrep.go index b1e877dead..5c11f7fac7 100644 --- a/flow/connectors/postgres/qrep.go +++ b/flow/connectors/postgres/qrep.go @@ -329,13 +329,13 @@ func (c *PostgresConnector) PullQRepRecordStream( config *protos.QRepConfig, partition *protos.QRepPartition, stream *model.QRecordStream, -) error { +) (int, error) { if partition.FullTablePartition { log.Infof("pulling full table partition for flow job %s", config.FlowJobName) executor := NewQRepQueryExecutorSnapshot(c.pool, c.ctx, c.config.TransactionSnapshot) query := config.Query _, err := executor.ExecuteAndProcessQueryStream(stream, query) - return err + return 0, err } var rangeStart interface{} @@ -361,29 +361,29 @@ func (c *PostgresConnector) PullQRepRecordStream( Valid: true, } default: - return fmt.Errorf("unknown range type: %v", x) + return 0, fmt.Errorf("unknown range type: %v", x) } // Build the query to pull records within the range from the source table // Be sure to order the results by the watermark column to ensure consistency across pulls query, err := BuildQuery(config.Query) if err != nil { - return err + return 0, err } executor := NewQRepQueryExecutorSnapshot(c.pool, c.ctx, c.config.TransactionSnapshot) numRecords, err := executor.ExecuteAndProcessQueryStream(stream, query, rangeStart, rangeEnd) if err != nil { - return err + return 0, err } totalRecordsAtSource, err := c.getApproxTableCounts([]string{config.WatermarkTable}) if err != nil { - return err + return 0, err } metrics.LogQRepPullMetrics(c.ctx, config.FlowJobName, numRecords, totalRecordsAtSource) log.Infof("pulled %d records for flow job %s", numRecords, config.FlowJobName) - return nil + return numRecords, nil } func (c *PostgresConnector) SyncQRepRecords( diff --git a/flow/connectors/utils/monitoring/monitoring.go b/flow/connectors/utils/monitoring/monitoring.go index b806a7861f..f297e7cee0 100644 --- a/flow/connectors/utils/monitoring/monitoring.go +++ b/flow/connectors/utils/monitoring/monitoring.go @@ -5,8 +5,10 @@ import ( "fmt" "time" + "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/jackc/pglogrepl" "github.com/jackc/pgx/v5" + "github.com/jackc/pgx/v5/pgtype" "github.com/jackc/pgx/v5/pgxpool" log "github.com/sirupsen/logrus" ) @@ -92,7 +94,8 @@ func (c *CatalogMirrorMonitor) AddCDCBatchForFlow(ctx context.Context, flowJobNa _, err := c.catalogConn.Exec(ctx, `INSERT INTO peerdb_stats.cdc_batches(flow_name,batch_id,rows_in_batch,batch_start_lsn,batch_end_lsn, - start_time) VALUES($1,$2,$3,$4,$5,$6)`, flowJobName, batchInfo.BatchID, batchInfo.RowsInBatch, + start_time) VALUES($1,$2,$3,$4,$5,$6) ON CONFLICT DO NOTHING`, + flowJobName, batchInfo.BatchID, batchInfo.RowsInBatch, uint64(batchInfo.BatchStartLSN), uint64(batchInfo.BatchEndlSN), batchInfo.StartTime) if err != nil { return fmt.Errorf("error while inserting batch into cdc_batch: %w", err) @@ -135,7 +138,7 @@ func (c *CatalogMirrorMonitor) AddCDCBatchTablesForFlow(ctx context.Context, flo for destinationTableName, numRows := range tableNameRowsMapping { _, err = insertBatchTablesTx.Exec(ctx, `INSERT INTO peerdb_stats.cdc_batch_table(flow_name,batch_id,destination_table_name,num_rows) - VALUES($1,$2,$3,$4)`, + VALUES($1,$2,$3,$4) ON CONFLICT DO NOTHING`, flowJobName, batchID, destinationTableName, numRows) if err != nil { return fmt.Errorf("error while inserting statistics into cdc_batch_table: %w", err) @@ -148,3 +151,113 @@ func (c *CatalogMirrorMonitor) AddCDCBatchTablesForFlow(ctx context.Context, flo } return nil } + +func (c *CatalogMirrorMonitor) InitializeQRepRun(ctx context.Context, flowJobName string, runUUID string, + startTime time.Time) error { + if c == nil || c.catalogConn == nil { + return nil + } + + _, err := c.catalogConn.Exec(ctx, + "INSERT INTO peerdb_stats.qrep_runs(flow_name,run_uuid,start_time) VALUES($1,$2,$3) ON CONFLICT DO NOTHING", + flowJobName, runUUID, startTime) + if err != nil { + return fmt.Errorf("error while inserting qrep run in qrep_runs: %w", err) + } + + return nil +} + +func (c *CatalogMirrorMonitor) UpdateEndTimeForQRepRun(ctx context.Context, runUUID string) error { + if c == nil || c.catalogConn == nil { + return nil + } + + _, err := c.catalogConn.Exec(ctx, + "UPDATE peerdb_stats.qrep_runs SET end_time=$1 WHERE run_uuid=$2", + time.Now(), runUUID) + if err != nil { + return fmt.Errorf("error while updating num_rows_to_sync for run_uuid %s in qrep_runs: %w", runUUID, err) + } + + return nil +} + +func (c *CatalogMirrorMonitor) AddPartitionToQRepRun(ctx context.Context, flowJobName string, + runUUID string, partition *protos.QRepPartition) error { + if c == nil || c.catalogConn == nil { + return nil + } + + var rangeStart, rangeEnd string + switch x := partition.Range.Range.(type) { + case *protos.PartitionRange_IntRange: + rangeStart = fmt.Sprint(x.IntRange.Start) + rangeEnd = fmt.Sprint(x.IntRange.End) + case *protos.PartitionRange_TimestampRange: + rangeStart = x.TimestampRange.Start.AsTime().String() + rangeEnd = x.TimestampRange.End.AsTime().String() + case *protos.PartitionRange_TidRange: + rangeStartValue, err := pgtype.TID{ + BlockNumber: x.TidRange.Start.BlockNumber, + OffsetNumber: uint16(x.TidRange.Start.OffsetNumber), + Valid: true, + }.Value() + if err != nil { + return fmt.Errorf("unable to encode TID as string: %w", err) + } + rangeStart = rangeStartValue.(string) + + rangeEndValue, err := pgtype.TID{ + BlockNumber: x.TidRange.End.BlockNumber, + OffsetNumber: uint16(x.TidRange.End.OffsetNumber), + Valid: true, + }.Value() + if err != nil { + return fmt.Errorf("unable to encode TID as string: %w", err) + } + rangeEnd = rangeEndValue.(string) + default: + return fmt.Errorf("unknown range type: %v", x) + } + + _, err := c.catalogConn.Exec(ctx, + `INSERT INTO peerdb_stats.qrep_partitions + (flow_name,run_uuid,partition_uuid,partition_start,partition_end,start_time,restart_count) + VALUES($1,$2,$3,$4,$5,$6,$7) ON CONFLICT(run_uuid,partition_uuid) DO UPDATE SET + restart_count=qrep_partitions.restart_count+1`, + flowJobName, runUUID, partition.PartitionId, rangeStart, rangeEnd, time.Now(), 0) + if err != nil { + return fmt.Errorf("error while inserting qrep partition in qrep_partitions: %w", err) + } + + return nil +} + +func (c *CatalogMirrorMonitor) UpdatePullEndTimeAndRowsForPartition(ctx context.Context, runUUID string, + partition *protos.QRepPartition, rowsInPartition int64) error { + if c == nil || c.catalogConn == nil { + return nil + } + + _, err := c.catalogConn.Exec(ctx, `UPDATE peerdb_stats.qrep_partitions SET pull_end_time=$1,rows_in_partition=$2 + WHERE run_uuid=$3 AND partition_uuid=$4`, time.Now(), rowsInPartition, runUUID, partition.PartitionId) + if err != nil { + return fmt.Errorf("error while updating qrep partition in qrep_partitions: %w", err) + } + return nil +} + +func (c *CatalogMirrorMonitor) UpdateEndTimeForPartition(ctx context.Context, runUUID string, + partition *protos.QRepPartition) error { + if c == nil || c.catalogConn == nil { + return nil + } + + _, err := c.catalogConn.Exec(ctx, `UPDATE peerdb_stats.qrep_partitions SET end_time=$1 + WHERE run_uuid=$2 AND partition_uuid=$3`, time.Now(), runUUID, partition.PartitionId) + if err != nil { + return fmt.Errorf("error while updating qrep partition in qrep_partitions: %w", err) + } + return nil +} diff --git a/flow/workflows/qrep_flow.go b/flow/workflows/qrep_flow.go index f22c4b2ce6..d5fbcf8745 100644 --- a/flow/workflows/qrep_flow.go +++ b/flow/workflows/qrep_flow.go @@ -19,14 +19,16 @@ type QRepFlowExecution struct { config *protos.QRepConfig flowExecutionID string logger log.Logger + runUUID string } // NewQRepFlowExecution creates a new instance of QRepFlowExecution. -func NewQRepFlowExecution(ctx workflow.Context, config *protos.QRepConfig) *QRepFlowExecution { +func NewQRepFlowExecution(ctx workflow.Context, config *protos.QRepConfig, runUUID string) *QRepFlowExecution { return &QRepFlowExecution{ config: config, flowExecutionID: workflow.GetInfo(ctx).WorkflowExecution.ID, logger: workflow.GetLogger(ctx), + runUUID: runUUID, } } @@ -58,7 +60,7 @@ func (q *QRepFlowExecution) GetPartitions( HeartbeatTimeout: 5 * time.Minute, }) - partitionsFuture := workflow.ExecuteActivity(ctx, flowable.GetQRepPartitions, q.config, last) + partitionsFuture := workflow.ExecuteActivity(ctx, flowable.GetQRepPartitions, q.config, last, q.runUUID) partitions := &protos.QRepParitionResult{} if err := partitionsFuture.Get(ctx, &partitions); err != nil { return nil, fmt.Errorf("failed to fetch partitions to replicate: %w", err) @@ -81,7 +83,7 @@ func (q *QRepFlowExecution) ReplicatePartition(ctx workflow.Context, partition * }) if err := workflow.ExecuteActivity(ctx, - flowable.ReplicateQRepPartition, q.config, partition).Get(ctx, nil); err != nil { + flowable.ReplicateQRepPartition, q.config, partition, q.runUUID).Get(ctx, nil); err != nil { return fmt.Errorf("failed to replicate partition: %w", err) } @@ -119,7 +121,8 @@ func (q *QRepFlowExecution) startChildWorkflow( }, }) - return workflow.ExecuteChildWorkflow(partFlowCtx, QRepPartitionWorkflow, q.config, partition), nil + return workflow.ExecuteChildWorkflow(partFlowCtx, QRepPartitionWorkflow, q.config, partition, + q.runUUID), nil } // processPartitions handles the logic for processing the partitions. @@ -166,7 +169,8 @@ func (q *QRepFlowExecution) consolidatePartitions(ctx workflow.Context) error { HeartbeatTimeout: 10 * time.Minute, }) - if err := workflow.ExecuteActivity(ctx, flowable.ConsolidateQRepPartitions, q.config).Get(ctx, nil); err != nil { + if err := workflow.ExecuteActivity(ctx, flowable.ConsolidateQRepPartitions, q.config, + q.runUUID).Get(ctx, nil); err != nil { return fmt.Errorf("failed to consolidate partitions: %w", err) } @@ -234,7 +238,7 @@ func QRepFlowWorkflow( return fmt.Errorf("failed to register query handler: %w", err) } - q := NewQRepFlowExecution(ctx, config) + q := NewQRepFlowExecution(ctx, config, uuid.New().String()) err = q.SetupMetadataTables(ctx) if err != nil { @@ -303,7 +307,8 @@ func QRepFlowWorkflow( } // QRepPartitionWorkflow replicate a single partition. -func QRepPartitionWorkflow(ctx workflow.Context, config *protos.QRepConfig, partition *protos.QRepPartition) error { - q := NewQRepFlowExecution(ctx, config) +func QRepPartitionWorkflow(ctx workflow.Context, config *protos.QRepConfig, partition *protos.QRepPartition, + runUUID string) error { + q := NewQRepFlowExecution(ctx, config, runUUID) return q.ReplicatePartition(ctx, partition) } diff --git a/nexus/catalog/migrations/V6__monitoring_refine_1.sql b/nexus/catalog/migrations/V6__monitoring_refine_1.sql new file mode 100644 index 0000000000..723f677a50 --- /dev/null +++ b/nexus/catalog/migrations/V6__monitoring_refine_1.sql @@ -0,0 +1,5 @@ +ALTER TABLE peerdb_stats.qrep_runs DROP COLUMN num_rows_to_sync; + +ALTER TABLE peerdb_stats.qrep_partitions ALTER COLUMN rows_in_partition DROP NOT NULL; + +ALTER TABLE peerdb_stats.qrep_partitions ADD UNIQUE(run_uuid,partition_uuid); \ No newline at end of file From 21546b62db9fbc7908471a1876293b8336549737 Mon Sep 17 00:00:00 2001 From: Kevin K Biju <52661649+heavycrystal@users.noreply.github.com> Date: Fri, 25 Aug 2023 02:04:49 +0530 Subject: [PATCH 11/37] duplicate publication and slot are not created when specified (#352) --- flow/connectors/postgres/postgres.go | 6 + flow/generated/protos/flow.pb.go | 544 ++++++++++++++------------- flow/workflows/snapshot_flow.go | 10 +- nexus/pt/src/peerdb_flow.rs | 4 + nexus/pt/src/peerdb_flow.serde.rs | 36 ++ protos/flow.proto | 2 + 6 files changed, 338 insertions(+), 264 deletions(-) diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index 5900a93295..ecd4cd1ac9 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -661,9 +661,15 @@ func (c *PostgresConnector) SetupReplication(signal *SlotSignal, req *protos.Set // Slotname would be the job name prefixed with "peerflow_slot_" slotName := fmt.Sprintf("peerflow_slot_%s", req.FlowJobName) + if req.ExistingReplicationSlotName != "" { + slotName = req.ExistingReplicationSlotName + } // Publication name would be the job name prefixed with "peerflow_pub_" publicationName := fmt.Sprintf("peerflow_pub_%s", req.FlowJobName) + if req.ExistingPublicationName != "" { + publicationName = req.ExistingPublicationName + } // Check if the replication slot and publication exist exists, err := c.checkSlotAndPublication(slotName, publicationName) diff --git a/flow/generated/protos/flow.pb.go b/flow/generated/protos/flow.pb.go index d82cb3f87e..be7e7ae680 100644 --- a/flow/generated/protos/flow.pb.go +++ b/flow/generated/protos/flow.pb.go @@ -1029,8 +1029,10 @@ type SetupReplicationInput struct { FlowJobName string `protobuf:"bytes,2,opt,name=flow_job_name,json=flowJobName,proto3" json:"flow_job_name,omitempty"` TableNameMapping map[string]string `protobuf:"bytes,3,rep,name=table_name_mapping,json=tableNameMapping,proto3" json:"table_name_mapping,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // replicate to destination using ctid - DestinationPeer *Peer `protobuf:"bytes,4,opt,name=destination_peer,json=destinationPeer,proto3" json:"destination_peer,omitempty"` - DoInitialCopy bool `protobuf:"varint,5,opt,name=do_initial_copy,json=doInitialCopy,proto3" json:"do_initial_copy,omitempty"` + DestinationPeer *Peer `protobuf:"bytes,4,opt,name=destination_peer,json=destinationPeer,proto3" json:"destination_peer,omitempty"` + DoInitialCopy bool `protobuf:"varint,5,opt,name=do_initial_copy,json=doInitialCopy,proto3" json:"do_initial_copy,omitempty"` + ExistingPublicationName string `protobuf:"bytes,6,opt,name=existing_publication_name,json=existingPublicationName,proto3" json:"existing_publication_name,omitempty"` + ExistingReplicationSlotName string `protobuf:"bytes,7,opt,name=existing_replication_slot_name,json=existingReplicationSlotName,proto3" json:"existing_replication_slot_name,omitempty"` } func (x *SetupReplicationInput) Reset() { @@ -1100,6 +1102,20 @@ func (x *SetupReplicationInput) GetDoInitialCopy() bool { return false } +func (x *SetupReplicationInput) GetExistingPublicationName() string { + if x != nil { + return x.ExistingPublicationName + } + return "" +} + +func (x *SetupReplicationInput) GetExistingReplicationSlotName() string { + if x != nil { + return x.ExistingReplicationSlotName + } + return "" +} + type SetupReplicationOutput struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -2581,7 +2597,7 @@ var file_flow_proto_rawDesc = []byte{ 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, - 0x99, 0x03, 0x0a, 0x15, 0x53, 0x65, 0x74, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, + 0x9a, 0x04, 0x0a, 0x15, 0x53, 0x65, 0x74, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, @@ -2602,270 +2618,278 @@ var file_flow_proto_rawDesc = []byte{ 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x65, 0x65, 0x72, 0x12, 0x26, 0x0a, 0x0f, 0x64, 0x6f, 0x5f, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x63, 0x6f, 0x70, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x64, 0x6f, 0x49, 0x6e, 0x69, 0x74, 0x69, - 0x61, 0x6c, 0x43, 0x6f, 0x70, 0x79, 0x1a, 0x43, 0x0a, 0x15, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, - 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, - 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, - 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x5a, 0x0a, 0x16, 0x53, - 0x65, 0x74, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4f, - 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x6c, 0x6f, 0x74, 0x5f, 0x6e, 0x61, - 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x73, 0x6c, 0x6f, 0x74, 0x4e, 0x61, - 0x6d, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x5f, 0x6e, - 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x6e, 0x61, 0x70, 0x73, - 0x68, 0x6f, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xed, 0x02, 0x0a, 0x13, 0x43, 0x72, 0x65, 0x61, - 0x74, 0x65, 0x52, 0x61, 0x77, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, - 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, - 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, - 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x64, 0x0a, - 0x12, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x6d, 0x61, 0x70, 0x70, - 0x69, 0x6e, 0x67, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x70, 0x65, 0x65, 0x72, - 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x61, - 0x77, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, - 0x79, 0x52, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, - 0x69, 0x6e, 0x67, 0x12, 0x3d, 0x0a, 0x0d, 0x63, 0x64, 0x63, 0x5f, 0x73, 0x79, 0x6e, 0x63, 0x5f, - 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x65, 0x65, - 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x53, 0x79, 0x6e, - 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x0b, 0x63, 0x64, 0x63, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, - 0x64, 0x65, 0x1a, 0x43, 0x0a, 0x15, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, - 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, - 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x41, 0x0a, 0x14, 0x43, 0x72, 0x65, 0x61, 0x74, - 0x65, 0x52, 0x61, 0x77, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, + 0x61, 0x6c, 0x43, 0x6f, 0x70, 0x79, 0x12, 0x3a, 0x0a, 0x19, 0x65, 0x78, 0x69, 0x73, 0x74, 0x69, + 0x6e, 0x67, 0x5f, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x17, 0x65, 0x78, 0x69, 0x73, 0x74, + 0x69, 0x6e, 0x67, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, + 0x6d, 0x65, 0x12, 0x43, 0x0a, 0x1e, 0x65, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x72, + 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x5f, + 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1b, 0x65, 0x78, 0x69, 0x73, + 0x74, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, + 0x6c, 0x6f, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0x43, 0x0a, 0x15, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, + 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x5a, 0x0a, 0x16, + 0x53, 0x65, 0x74, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x6c, 0x6f, 0x74, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x73, 0x6c, 0x6f, 0x74, 0x4e, + 0x61, 0x6d, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x5f, + 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x6e, 0x61, 0x70, + 0x73, 0x68, 0x6f, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xed, 0x02, 0x0a, 0x13, 0x43, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x52, 0x61, 0x77, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, + 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, + 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, + 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x64, + 0x0a, 0x12, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x6d, 0x61, 0x70, + 0x70, 0x69, 0x6e, 0x67, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x70, 0x65, 0x65, + 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, + 0x61, 0x77, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x52, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, + 0x70, 0x69, 0x6e, 0x67, 0x12, 0x3d, 0x0a, 0x0d, 0x63, 0x64, 0x63, 0x5f, 0x73, 0x79, 0x6e, 0x63, + 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x53, 0x79, + 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x0b, 0x63, 0x64, 0x63, 0x53, 0x79, 0x6e, 0x63, 0x4d, + 0x6f, 0x64, 0x65, 0x1a, 0x43, 0x0a, 0x15, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, + 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, + 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, + 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x41, 0x0a, 0x14, 0x43, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x52, 0x61, 0x77, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, + 0x12, 0x29, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, + 0x66, 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x22, 0xe3, 0x01, 0x0a, 0x0b, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x29, 0x0a, 0x10, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, + 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x3f, 0x0a, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, + 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, + 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, + 0x61, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x07, + 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x2c, 0x0a, 0x12, 0x70, 0x72, 0x69, 0x6d, 0x61, + 0x72, 0x79, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x10, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x4b, 0x65, 0x79, 0x43, + 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x1a, 0x3a, 0x0a, 0x0c, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, + 0x01, 0x22, 0x91, 0x01, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, + 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, + 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, + 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 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, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, - 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x22, 0xe3, 0x01, 0x0a, 0x0b, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x29, 0x0a, 0x10, 0x74, 0x61, - 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, - 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x3f, 0x0a, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, - 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, - 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, - 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x07, 0x63, - 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x2c, 0x0a, 0x12, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, - 0x79, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x10, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x4b, 0x65, 0x79, 0x43, 0x6f, - 0x6c, 0x75, 0x6d, 0x6e, 0x1a, 0x3a, 0x0a, 0x0c, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x45, - 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, - 0x22, 0x91, 0x01, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, - 0x65, 0x6d, 0x61, 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, - 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, - 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, - 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x2b, 0x0a, 0x11, 0x74, 0x61, 0x62, 0x6c, 0x65, - 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, 0x18, 0x02, 0x20, 0x03, - 0x28, 0x09, 0x52, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, - 0x69, 0x65, 0x72, 0x73, 0x22, 0xff, 0x01, 0x0a, 0x19, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, - 0x75, 0x74, 0x12, 0x7d, 0x0a, 0x19, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 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, - 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, + 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, 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, + 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, 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, + 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, 0x52, 0x12, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x4d, - 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x45, 0x0a, 0x17, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x45, - 0x78, 0x69, 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, - 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, - 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x08, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x3b, 0x0a, - 0x11, 0x49, 0x6e, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, - 0x67, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x03, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0x79, 0x0a, 0x17, 0x54, 0x69, - 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x30, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, - 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x2c, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, - 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0x4d, 0x0a, 0x03, 0x54, 0x49, 0x44, 0x12, 0x21, 0x0a, 0x0c, - 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x0d, 0x52, 0x0b, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, - 0x23, 0x0a, 0x0d, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0c, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x4e, 0x75, - 0x6d, 0x62, 0x65, 0x72, 0x22, 0x5f, 0x0a, 0x11, 0x54, 0x49, 0x44, 0x50, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x26, 0x0a, 0x05, 0x73, 0x74, 0x61, - 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x49, 0x44, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, - 0x74, 0x12, 0x22, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, - 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x49, 0x44, - 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0xe8, 0x01, 0x0a, 0x0e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x3d, 0x0a, 0x09, 0x69, 0x6e, 0x74, 0x5f, - 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x65, - 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x49, 0x6e, 0x74, 0x50, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x48, 0x00, 0x52, 0x08, 0x69, - 0x6e, 0x74, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x4f, 0x0a, 0x0f, 0x74, 0x69, 0x6d, 0x65, 0x73, - 0x74, 0x61, 0x6d, 0x70, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x24, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, + 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, 0x48, 0x00, 0x52, 0x0e, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, - 0x61, 0x6d, 0x70, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x3d, 0x0a, 0x09, 0x74, 0x69, 0x64, 0x5f, - 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x65, - 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x49, 0x44, 0x50, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x48, 0x00, 0x52, 0x08, 0x74, - 0x69, 0x64, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x42, 0x07, 0x0a, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, - 0x22, 0x78, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4d, 0x6f, 0x64, - 0x65, 0x12, 0x39, 0x0a, 0x0a, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, - 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, - 0x65, 0x52, 0x09, 0x77, 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x2c, 0x0a, 0x12, - 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, - 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, - 0x4b, 0x65, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x22, 0x96, 0x06, 0x0a, 0x0a, 0x51, - 0x52, 0x65, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, - 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x33, 0x0a, - 0x0b, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, - 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0a, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x65, - 0x65, 0x72, 0x12, 0x3d, 0x0a, 0x10, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, - 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, - 0x52, 0x0f, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x65, 0x65, - 0x72, 0x12, 0x40, 0x0a, 0x1c, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, - 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1a, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, - 0x69, 0x65, 0x72, 0x12, 0x14, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x05, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x12, 0x27, 0x0a, 0x0f, 0x77, 0x61, 0x74, - 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x06, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x0e, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x5f, - 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x77, 0x61, - 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x2a, 0x0a, - 0x11, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x63, 0x6f, 0x70, 0x79, 0x5f, 0x6f, 0x6e, - 0x6c, 0x79, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, - 0x6c, 0x43, 0x6f, 0x70, 0x79, 0x4f, 0x6e, 0x6c, 0x79, 0x12, 0x36, 0x0a, 0x09, 0x73, 0x79, 0x6e, - 0x63, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, - 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x53, - 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x08, 0x73, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, - 0x65, 0x12, 0x24, 0x0a, 0x0e, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, - 0x69, 0x6e, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0c, 0x62, 0x61, 0x74, 0x63, 0x68, - 0x53, 0x69, 0x7a, 0x65, 0x49, 0x6e, 0x74, 0x12, 0x34, 0x0a, 0x16, 0x62, 0x61, 0x74, 0x63, 0x68, - 0x5f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, - 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x14, 0x62, 0x61, 0x74, 0x63, 0x68, 0x44, 0x75, - 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x30, 0x0a, - 0x14, 0x6d, 0x61, 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x5f, 0x77, 0x6f, - 0x72, 0x6b, 0x65, 0x72, 0x73, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x12, 0x6d, 0x61, 0x78, - 0x50, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73, 0x12, - 0x3f, 0x0a, 0x1c, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x62, 0x65, 0x74, 0x77, 0x65, 0x65, 0x6e, 0x5f, - 0x62, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, - 0x0d, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x19, 0x77, 0x61, 0x69, 0x74, 0x42, 0x65, 0x74, 0x77, 0x65, - 0x65, 0x6e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, - 0x12, 0x39, 0x0a, 0x0a, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x0e, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, - 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4d, 0x6f, 0x64, 0x65, - 0x52, 0x09, 0x77, 0x72, 0x69, 0x74, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x73, - 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x0f, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x0b, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x50, 0x61, 0x74, 0x68, 0x12, 0x33, - 0x0a, 0x16, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x70, 0x65, 0x72, 0x5f, 0x70, - 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x13, - 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x50, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x22, 0x97, 0x01, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x70, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x31, 0x0a, 0x05, 0x72, 0x61, 0x6e, 0x67, - 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, - 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, - 0x61, 0x6e, 0x67, 0x65, 0x52, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x30, 0x0a, 0x14, 0x66, - 0x75, 0x6c, 0x6c, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x66, 0x75, 0x6c, 0x6c, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 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, 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, + 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x30, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, + 0x70, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x2c, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, + 0x70, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0x4d, 0x0a, 0x03, 0x54, 0x49, 0x44, 0x12, 0x21, 0x0a, + 0x0c, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0d, 0x52, 0x0b, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, + 0x12, 0x23, 0x0a, 0x0d, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, + 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0c, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x4e, + 0x75, 0x6d, 0x62, 0x65, 0x72, 0x22, 0x5f, 0x0a, 0x11, 0x54, 0x49, 0x44, 0x50, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x26, 0x0a, 0x05, 0x73, 0x74, + 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x70, 0x65, 0x65, 0x72, + 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x49, 0x44, 0x52, 0x05, 0x73, 0x74, 0x61, + 0x72, 0x74, 0x12, 0x22, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x10, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x49, + 0x44, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0xe8, 0x01, 0x0a, 0x0e, 0x50, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x3d, 0x0a, 0x09, 0x69, 0x6e, 0x74, + 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x49, 0x6e, 0x74, 0x50, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x48, 0x00, 0x52, 0x08, + 0x69, 0x6e, 0x74, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x4f, 0x0a, 0x0f, 0x74, 0x69, 0x6d, 0x65, + 0x73, 0x74, 0x61, 0x6d, 0x70, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x24, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, + 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x48, 0x00, 0x52, 0x0e, 0x74, 0x69, 0x6d, 0x65, 0x73, + 0x74, 0x61, 0x6d, 0x70, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x3d, 0x0a, 0x09, 0x74, 0x69, 0x64, + 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x49, 0x44, 0x50, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x48, 0x00, 0x52, 0x08, + 0x74, 0x69, 0x64, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x42, 0x07, 0x0a, 0x05, 0x72, 0x61, 0x6e, 0x67, + 0x65, 0x22, 0x78, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4d, 0x6f, + 0x64, 0x65, 0x12, 0x39, 0x0a, 0x0a, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, + 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, + 0x70, 0x65, 0x52, 0x09, 0x77, 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x2c, 0x0a, + 0x12, 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, + 0x6d, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x75, 0x70, 0x73, 0x65, 0x72, + 0x74, 0x4b, 0x65, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x22, 0x96, 0x06, 0x0a, 0x0a, + 0x51, 0x52, 0x65, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, + 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x33, + 0x0a, 0x0b, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, + 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0a, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, + 0x65, 0x65, 0x72, 0x12, 0x3d, 0x0a, 0x10, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, + 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, + 0x72, 0x52, 0x0f, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x65, + 0x65, 0x72, 0x12, 0x40, 0x0a, 0x1c, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, + 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1a, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, + 0x66, 0x69, 0x65, 0x72, 0x12, 0x14, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x12, 0x27, 0x0a, 0x0f, 0x77, 0x61, + 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x06, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0e, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, + 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x77, + 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x2a, + 0x0a, 0x11, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x63, 0x6f, 0x70, 0x79, 0x5f, 0x6f, + 0x6e, 0x6c, 0x79, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x69, 0x6e, 0x69, 0x74, 0x69, + 0x61, 0x6c, 0x43, 0x6f, 0x70, 0x79, 0x4f, 0x6e, 0x6c, 0x79, 0x12, 0x36, 0x0a, 0x09, 0x73, 0x79, + 0x6e, 0x63, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, + 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, + 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x08, 0x73, 0x79, 0x6e, 0x63, 0x4d, 0x6f, + 0x64, 0x65, 0x12, 0x24, 0x0a, 0x0e, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, + 0x5f, 0x69, 0x6e, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0c, 0x62, 0x61, 0x74, 0x63, + 0x68, 0x53, 0x69, 0x7a, 0x65, 0x49, 0x6e, 0x74, 0x12, 0x34, 0x0a, 0x16, 0x62, 0x61, 0x74, 0x63, + 0x68, 0x5f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, + 0x64, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x14, 0x62, 0x61, 0x74, 0x63, 0x68, 0x44, + 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x30, + 0x0a, 0x14, 0x6d, 0x61, 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x5f, 0x77, + 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x12, 0x6d, 0x61, + 0x78, 0x50, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73, + 0x12, 0x3f, 0x0a, 0x1c, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x62, 0x65, 0x74, 0x77, 0x65, 0x65, 0x6e, + 0x5f, 0x62, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, + 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x19, 0x77, 0x61, 0x69, 0x74, 0x42, 0x65, 0x74, 0x77, + 0x65, 0x65, 0x6e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, + 0x73, 0x12, 0x39, 0x0a, 0x0a, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, + 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, + 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4d, 0x6f, 0x64, + 0x65, 0x52, 0x09, 0x77, 0x72, 0x69, 0x74, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x21, 0x0a, 0x0c, + 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x0f, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0b, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x50, 0x61, 0x74, 0x68, 0x12, + 0x33, 0x0a, 0x16, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x70, 0x65, 0x72, 0x5f, + 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0d, 0x52, + 0x13, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x50, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x97, 0x01, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x70, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x31, 0x0a, 0x05, 0x72, 0x61, 0x6e, + 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, + 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x30, 0x0a, 0x14, + 0x66, 0x75, 0x6c, 0x6c, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x66, 0x75, 0x6c, 0x6c, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 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, 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, 0x53, 0x54, 0x4f, 0x52, 0x41, 0x47, 0x45, 0x5f, 0x41, 0x56, 0x52, 0x4f, 0x10, 0x01, 0x2a, - 0x47, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, - 0x12, 0x1a, 0x0a, 0x16, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x5f, 0x4d, - 0x4f, 0x44, 0x45, 0x5f, 0x41, 0x50, 0x50, 0x45, 0x4e, 0x44, 0x10, 0x00, 0x12, 0x1a, 0x0a, 0x16, - 0x51, 0x52, 0x45, 0x50, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, - 0x55, 0x50, 0x53, 0x45, 0x52, 0x54, 0x10, 0x01, 0x42, 0x76, 0x0a, 0x0f, 0x63, 0x6f, 0x6d, 0x2e, - 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x42, 0x09, 0x46, 0x6c, 0x6f, - 0x77, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x10, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, - 0x74, 0x65, 0x64, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x73, 0xa2, 0x02, 0x03, 0x50, 0x58, 0x58, - 0xaa, 0x02, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0xca, 0x02, 0x0a, - 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0xe2, 0x02, 0x16, 0x50, 0x65, 0x65, - 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, - 0x61, 0x74, 0x61, 0xea, 0x02, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, - 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x5f, 0x4d, 0x55, 0x4c, 0x54, 0x49, 0x5f, 0x49, 0x4e, 0x53, 0x45, 0x52, 0x54, 0x10, 0x00, 0x12, + 0x1f, 0x0a, 0x1b, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x53, 0x59, 0x4e, 0x43, 0x5f, 0x4d, 0x4f, 0x44, + 0x45, 0x5f, 0x53, 0x54, 0x4f, 0x52, 0x41, 0x47, 0x45, 0x5f, 0x41, 0x56, 0x52, 0x4f, 0x10, 0x01, + 0x2a, 0x47, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, + 0x65, 0x12, 0x1a, 0x0a, 0x16, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x5f, + 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x41, 0x50, 0x50, 0x45, 0x4e, 0x44, 0x10, 0x00, 0x12, 0x1a, 0x0a, + 0x16, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x5f, 0x4d, 0x4f, 0x44, 0x45, + 0x5f, 0x55, 0x50, 0x53, 0x45, 0x52, 0x54, 0x10, 0x01, 0x42, 0x76, 0x0a, 0x0f, 0x63, 0x6f, 0x6d, + 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x42, 0x09, 0x46, 0x6c, + 0x6f, 0x77, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x10, 0x67, 0x65, 0x6e, 0x65, 0x72, + 0x61, 0x74, 0x65, 0x64, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x73, 0xa2, 0x02, 0x03, 0x50, 0x58, + 0x58, 0xaa, 0x02, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0xca, 0x02, + 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0xe2, 0x02, 0x16, 0x50, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, + 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, + 0x77, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( diff --git a/flow/workflows/snapshot_flow.go b/flow/workflows/snapshot_flow.go index e441c6a877..625981b674 100644 --- a/flow/workflows/snapshot_flow.go +++ b/flow/workflows/snapshot_flow.go @@ -34,10 +34,12 @@ func (s *SnapshotFlowExecution) setupReplication( }) setupReplicationInput := &protos.SetupReplicationInput{ - PeerConnectionConfig: s.config.Source, - FlowJobName: flowName, - TableNameMapping: s.config.TableNameMapping, - DoInitialCopy: s.config.DoInitialCopy, + PeerConnectionConfig: s.config.Source, + FlowJobName: flowName, + TableNameMapping: s.config.TableNameMapping, + DoInitialCopy: s.config.DoInitialCopy, + ExistingPublicationName: s.config.PublicationName, + ExistingReplicationSlotName: s.config.ReplicationSlotName, } res := &protos.SetupReplicationOutput{} diff --git a/nexus/pt/src/peerdb_flow.rs b/nexus/pt/src/peerdb_flow.rs index 2afeb242b3..3da275341f 100644 --- a/nexus/pt/src/peerdb_flow.rs +++ b/nexus/pt/src/peerdb_flow.rs @@ -166,6 +166,10 @@ pub struct SetupReplicationInput { pub destination_peer: ::core::option::Option, #[prost(bool, tag="5")] pub do_initial_copy: bool, + #[prost(string, tag="6")] + pub existing_publication_name: ::prost::alloc::string::String, + #[prost(string, tag="7")] + pub existing_replication_slot_name: ::prost::alloc::string::String, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] diff --git a/nexus/pt/src/peerdb_flow.serde.rs b/nexus/pt/src/peerdb_flow.serde.rs index 0665a74023..0ec540285a 100644 --- a/nexus/pt/src/peerdb_flow.serde.rs +++ b/nexus/pt/src/peerdb_flow.serde.rs @@ -3488,6 +3488,12 @@ impl serde::Serialize for SetupReplicationInput { if self.do_initial_copy { len += 1; } + if !self.existing_publication_name.is_empty() { + len += 1; + } + if !self.existing_replication_slot_name.is_empty() { + len += 1; + } let mut struct_ser = serializer.serialize_struct("peerdb_flow.SetupReplicationInput", len)?; if let Some(v) = self.peer_connection_config.as_ref() { struct_ser.serialize_field("peerConnectionConfig", v)?; @@ -3504,6 +3510,12 @@ impl serde::Serialize for SetupReplicationInput { if self.do_initial_copy { struct_ser.serialize_field("doInitialCopy", &self.do_initial_copy)?; } + if !self.existing_publication_name.is_empty() { + struct_ser.serialize_field("existingPublicationName", &self.existing_publication_name)?; + } + if !self.existing_replication_slot_name.is_empty() { + struct_ser.serialize_field("existingReplicationSlotName", &self.existing_replication_slot_name)?; + } struct_ser.end() } } @@ -3524,6 +3536,10 @@ impl<'de> serde::Deserialize<'de> for SetupReplicationInput { "destinationPeer", "do_initial_copy", "doInitialCopy", + "existing_publication_name", + "existingPublicationName", + "existing_replication_slot_name", + "existingReplicationSlotName", ]; #[allow(clippy::enum_variant_names)] @@ -3533,6 +3549,8 @@ impl<'de> serde::Deserialize<'de> for SetupReplicationInput { TableNameMapping, DestinationPeer, DoInitialCopy, + ExistingPublicationName, + ExistingReplicationSlotName, __SkipField__, } impl<'de> serde::Deserialize<'de> for GeneratedField { @@ -3560,6 +3578,8 @@ impl<'de> serde::Deserialize<'de> for SetupReplicationInput { "tableNameMapping" | "table_name_mapping" => Ok(GeneratedField::TableNameMapping), "destinationPeer" | "destination_peer" => Ok(GeneratedField::DestinationPeer), "doInitialCopy" | "do_initial_copy" => Ok(GeneratedField::DoInitialCopy), + "existingPublicationName" | "existing_publication_name" => Ok(GeneratedField::ExistingPublicationName), + "existingReplicationSlotName" | "existing_replication_slot_name" => Ok(GeneratedField::ExistingReplicationSlotName), _ => Ok(GeneratedField::__SkipField__), } } @@ -3584,6 +3604,8 @@ impl<'de> serde::Deserialize<'de> for SetupReplicationInput { let mut table_name_mapping__ = None; let mut destination_peer__ = None; let mut do_initial_copy__ = None; + let mut existing_publication_name__ = None; + let mut existing_replication_slot_name__ = None; while let Some(k) = map.next_key()? { match k { GeneratedField::PeerConnectionConfig => { @@ -3618,6 +3640,18 @@ impl<'de> serde::Deserialize<'de> for SetupReplicationInput { } do_initial_copy__ = Some(map.next_value()?); } + GeneratedField::ExistingPublicationName => { + if existing_publication_name__.is_some() { + return Err(serde::de::Error::duplicate_field("existingPublicationName")); + } + existing_publication_name__ = Some(map.next_value()?); + } + GeneratedField::ExistingReplicationSlotName => { + if existing_replication_slot_name__.is_some() { + return Err(serde::de::Error::duplicate_field("existingReplicationSlotName")); + } + existing_replication_slot_name__ = Some(map.next_value()?); + } GeneratedField::__SkipField__ => { let _ = map.next_value::()?; } @@ -3629,6 +3663,8 @@ impl<'de> serde::Deserialize<'de> for SetupReplicationInput { table_name_mapping: table_name_mapping__.unwrap_or_default(), destination_peer: destination_peer__, do_initial_copy: do_initial_copy__.unwrap_or_default(), + existing_publication_name: existing_publication_name__.unwrap_or_default(), + existing_replication_slot_name: existing_replication_slot_name__.unwrap_or_default(), }) } } diff --git a/protos/flow.proto b/protos/flow.proto index 1066fe4fe0..d76be176a7 100644 --- a/protos/flow.proto +++ b/protos/flow.proto @@ -105,6 +105,8 @@ message SetupReplicationInput { // replicate to destination using ctid peerdb_peers.Peer destination_peer = 4; bool do_initial_copy = 5; + string existing_publication_name = 6; + string existing_replication_slot_name = 7; } message SetupReplicationOutput { From 629ee4401d9c5476c0f7a94f2a3a9fe26ae6828f Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Thu, 24 Aug 2023 14:22:47 -0700 Subject: [PATCH 12/37] pull records fix (#353) --- flow/connectors/postgres/cdc.go | 12 ++---------- flow/connectors/postgres/postgres.go | 6 ++++++ 2 files changed, 8 insertions(+), 10 deletions(-) diff --git a/flow/connectors/postgres/cdc.go b/flow/connectors/postgres/cdc.go index 1418611add..92b1c94755 100644 --- a/flow/connectors/postgres/cdc.go +++ b/flow/connectors/postgres/cdc.go @@ -59,21 +59,13 @@ func (p *PostgresCDCSource) PullRecords(req *model.PullRecordsRequest) (*model.R "proto_version '1'", } - if p.publication != "" && req.OverridePublicationName == "" { + if p.publication != "" { pubOpt := fmt.Sprintf("publication_names '%s'", p.publication) pluginArguments = append(pluginArguments, pubOpt) - } else { - pubOpt := fmt.Sprintf("publication_names '%s'", req.OverridePublicationName) - pluginArguments = append(pluginArguments, pubOpt) } replicationOpts := pglogrepl.StartReplicationOptions{PluginArgs: pluginArguments} - var replicationSlot string - if p.slot != "" && req.OverrideReplicationSlotName == "" { - replicationSlot = p.slot - } else { - replicationSlot = req.OverrideReplicationSlotName - } + replicationSlot := p.slot // create replication connection replicationConn, err := p.replPool.Acquire(p.ctx) diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index ecd4cd1ac9..46de9889b6 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -176,9 +176,15 @@ func (c *PostgresConnector) GetLastOffset(jobName string) (*protos.LastSyncState func (c *PostgresConnector) PullRecords(req *model.PullRecordsRequest) (*model.RecordBatch, error) { // Slotname would be the job name prefixed with "peerflow_slot_" slotName := fmt.Sprintf("peerflow_slot_%s", req.FlowJobName) + if req.OverrideReplicationSlotName != "" { + slotName = req.OverrideReplicationSlotName + } // Publication name would be the job name prefixed with "peerflow_pub_" publicationName := fmt.Sprintf("peerflow_pub_%s", req.FlowJobName) + if req.OverridePublicationName != "" { + publicationName = req.OverridePublicationName + } // Check if the replication slot and publication exist exists, err := c.checkSlotAndPublication(slotName, publicationName) From 79ee15a051e417710717f082fdbd5f00812c77fb Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Thu, 24 Aug 2023 14:59:57 -0700 Subject: [PATCH 13/37] make normalize run after every sync (#354) --- flow/workflows/peer_flow.go | 57 ++++++++++++++++++------------------- 1 file changed, 28 insertions(+), 29 deletions(-) diff --git a/flow/workflows/peer_flow.go b/flow/workflows/peer_flow.go index 8b49f5bd9a..e6cde4378c 100644 --- a/flow/workflows/peer_flow.go +++ b/flow/workflows/peer_flow.go @@ -342,38 +342,37 @@ func PeerFlowWorkflowWithConfig( } }) selector.Select(ctx) - } - normalizeFlowID, err := GetChildWorkflowID(ctx, "normalize-flow", cfg.FlowJobName) - if err != nil { - return state, err - } + normalizeFlowID, err := GetChildWorkflowID(ctx, "normalize-flow", cfg.FlowJobName) + if err != nil { + return state, err + } - // execute the normalize flow as a child workflow - childNormalizeFlowOpts := workflow.ChildWorkflowOptions{ - WorkflowID: normalizeFlowID, - ParentClosePolicy: enums.PARENT_CLOSE_POLICY_REQUEST_CANCEL, - RetryPolicy: &temporal.RetryPolicy{ - MaximumAttempts: 20, - }, - } - ctx = workflow.WithChildOptions(ctx, childNormalizeFlowOpts) - childNormalizeFlowFuture := workflow.ExecuteChildWorkflow( - ctx, - NormalizeFlowWorkflow, - cfg, - ) - - selector.AddFuture(childNormalizeFlowFuture, func(f workflow.Future) { - var childNormalizeFlowRes *model.NormalizeResponse - if err := f.Get(ctx, &childNormalizeFlowRes); err != nil { - w.logger.Error("failed to execute normalize flow: ", err) - state.NormalizeFlowErrors = multierror.Append(state.NormalizeFlowErrors, err) - } else { - state.NormalizeFlowStatuses = append(state.NormalizeFlowStatuses, childNormalizeFlowRes) + childNormalizeFlowOpts := workflow.ChildWorkflowOptions{ + WorkflowID: normalizeFlowID, + ParentClosePolicy: enums.PARENT_CLOSE_POLICY_REQUEST_CANCEL, + RetryPolicy: &temporal.RetryPolicy{ + MaximumAttempts: 20, + }, } - }) - selector.Select(ctx) + ctx = workflow.WithChildOptions(ctx, childNormalizeFlowOpts) + childNormalizeFlowFuture := workflow.ExecuteChildWorkflow( + ctx, + NormalizeFlowWorkflow, + cfg, + ) + + selector.AddFuture(childNormalizeFlowFuture, func(f workflow.Future) { + var childNormalizeFlowRes *model.NormalizeResponse + if err := f.Get(ctx, &childNormalizeFlowRes); err != nil { + w.logger.Error("failed to execute normalize flow: ", err) + state.NormalizeFlowErrors = multierror.Append(state.NormalizeFlowErrors, err) + } else { + state.NormalizeFlowStatuses = append(state.NormalizeFlowStatuses, childNormalizeFlowRes) + } + }) + selector.Select(ctx) + } state.TruncateProgress() return nil, workflow.NewContinueAsNewError(ctx, PeerFlowWorkflowWithConfig, cfg, limits, state) From f80095a92519add8c04234e9c51417195f3672d3 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Fri, 25 Aug 2023 14:58:34 +0530 Subject: [PATCH 14/37] Fixes for Snapshot Flow (#351) - Destination name as variable in `cloneTables()` - Fixes normalize statement in Postgres CDC - Logs in `snapshot_flow.go` --- flow/connectors/postgres/client.go | 25 ++++++++++++++++--------- flow/workflows/snapshot_flow.go | 23 +++++++++++++++++++++-- 2 files changed, 37 insertions(+), 11 deletions(-) diff --git a/flow/connectors/postgres/client.go b/flow/connectors/postgres/client.go index a6ea5996e6..0799917565 100644 --- a/flow/connectors/postgres/client.go +++ b/flow/connectors/postgres/client.go @@ -456,13 +456,12 @@ func (c *PostgresConnector) generateFallbackStatements(destinationTableIdentifie rawTableIdentifier string) []string { normalizedTableSchema := c.tableSchemaMapping[destinationTableIdentifier] columnNames := make([]string, 0, len(normalizedTableSchema.Columns)) - flattenedCastsSQLArray := make([]string, 0, len(normalizedTableSchema.Columns)) var primaryKeyColumnCast string for columnName, genericColumnType := range normalizedTableSchema.Columns { - columnNames = append(columnNames, columnName) + columnNames = append(columnNames, fmt.Sprintf("\"%s\"", columnName)) pgType := qValueKindToPostgresType(genericColumnType) - flattenedCastsSQLArray = append(flattenedCastsSQLArray, fmt.Sprintf("(_peerdb_data->>'%s')::%s AS %s", + 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) @@ -490,6 +489,9 @@ func (c *PostgresConnector) generateMergeStatement(destinationTableIdentifier st rawTableIdentifier string) string { normalizedTableSchema := c.tableSchemaMapping[destinationTableIdentifier] columnNames := maps.Keys(normalizedTableSchema.Columns) + for i, columnName := range columnNames { + columnNames[i] = fmt.Sprintf("\"%s\"", columnName) + } flattenedCastsSQLArray := make([]string, 0, len(normalizedTableSchema.Columns)) var primaryKeyColumnCast string @@ -497,14 +499,14 @@ func (c *PostgresConnector) generateMergeStatement(destinationTableIdentifier st pgType := qValueKindToPostgresType(genericColumnType) if strings.Contains(genericColumnType, "array") { flattenedCastsSQLArray = append(flattenedCastsSQLArray, - fmt.Sprintf("ARRAY(SELECT * FROM JSON_ARRAY_ELEMENTS_TEXT((_peerdb_data->>'%s')::JSON))::%s AS \"%s\"", - columnName, pgType, columnName)) + fmt.Sprintf("ARRAY(SELECT * FROM JSON_ARRAY_ELEMENTS_TEXT((_peerdb_data->>'%s')::JSON))::%s AS %s", + strings.Trim(columnName, "\""), pgType, columnName)) } else { - flattenedCastsSQLArray = append(flattenedCastsSQLArray, fmt.Sprintf("(_peerdb_data->>'%s')::%s AS \"%s\"", - columnName, pgType, columnName)) + 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", columnName, pgType) + primaryKeyColumnCast = fmt.Sprintf("(_peerdb_data->>'%s')::%s", strings.Trim(columnName, "\""), pgType) } } flattenedCastsSQL := strings.TrimSuffix(strings.Join(flattenedCastsSQLArray, ","), ",") @@ -527,19 +529,24 @@ func (c *PostgresConnector) generateUpdateStatement(allCols []string, unchangedT for _, cols := range unchangedToastColsLists { unchangedColsArray := strings.Split(cols, ",") + for i, col := range unchangedColsArray { + unchangedColsArray[i] = fmt.Sprintf("\"%s\"", col) + } otherCols := utils.ArrayMinus(allCols, unchangedColsArray) tmpArray := make([]string, 0) for _, colName := range otherCols { 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 `, cols, ssep) + THEN UPDATE SET %s `, quotedCols, ssep) updateStmts = append(updateStmts, updateStmt) } return strings.Join(updateStmts, "\n") } + func (c *PostgresConnector) getApproxTableCounts(tables []string) (int64, error) { countTablesBatch := &pgx.Batch{} totalCount := int64(0) diff --git a/flow/workflows/snapshot_flow.go b/flow/workflows/snapshot_flow.go index 625981b674..70d3376fb4 100644 --- a/flow/workflows/snapshot_flow.go +++ b/flow/workflows/snapshot_flow.go @@ -9,6 +9,7 @@ import ( "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/shared" "github.com/google/uuid" + logrus "github.com/sirupsen/logrus" "go.temporal.io/sdk/log" "go.temporal.io/sdk/temporal" "go.temporal.io/sdk/workflow" @@ -88,8 +89,18 @@ func (s *SnapshotFlowExecution) cloneTable( var childWorkflowID string if err := childWorkflowIDSideEffect.Get(&childWorkflowID); err != nil { + logrus.WithFields(logrus.Fields{ + "flowName": flowName, + "snapshotName": snapshotName, + }).Errorf("failed to get child id for source table %s and destination table %s", + sourceTable, destinationTableName) return nil, fmt.Errorf("failed to get child workflow ID: %w", err) } + logrus.WithFields(logrus.Fields{ + "flowName": flowName, + "snapshotName": snapshotName, + }).Infof("Obtained child id %s for source table %s and destination table %s", + childWorkflowID, sourceTable, destinationTableName) childCtx = workflow.WithChildOptions(childCtx, workflow.ChildWorkflowOptions{ WorkflowID: childWorkflowID, @@ -153,18 +164,26 @@ func (s *SnapshotFlowExecution) cloneTables( slotInfo *protos.SetupReplicationOutput, maxParallelClones int, ) { + logrus.Infof("cloning tables for slot name %s and snapshotName %s", + slotInfo.SlotName, slotInfo.SnapshotName) boundSelector := concurrency.NewBoundSelector(maxParallelClones, ctx) for srcTbl, dstTbl := range s.config.TableNameMapping { source := srcTbl + destination := dstTbl snapshotName := slotInfo.SnapshotName - - future, err := s.cloneTable(ctx, snapshotName, source, dstTbl) + logrus.WithFields(logrus.Fields{ + "snapshotName": snapshotName, + }).Infof("Cloning table with source table %s and destination table name %s", + source, destination) + future, err := s.cloneTable(ctx, snapshotName, source, destination) if err != nil { s.logger.Error("failed to start clone child workflow: ", err) continue } boundSelector.AddFuture(future, func(f workflow.Future) error { + logrus.Infof("Adding future for clone table for source name %s and destination %s", + source, destination) if err := f.Get(ctx, nil); err != nil { s.logger.Error("failed to clone table", "table", source, "error", err) return err From 6682ea1029002f9b29800e04b6658a7df22f57d5 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Fri, 25 Aug 2023 20:39:58 +0530 Subject: [PATCH 15/37] Adds variable assignments in clonetable (#355) --- flow/workflows/snapshot_flow.go | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/flow/workflows/snapshot_flow.go b/flow/workflows/snapshot_flow.go index 70d3376fb4..299097eff2 100644 --- a/flow/workflows/snapshot_flow.go +++ b/flow/workflows/snapshot_flow.go @@ -76,13 +76,14 @@ func (s *SnapshotFlowExecution) closeSlotKeepAlive( func (s *SnapshotFlowExecution) cloneTable( childCtx workflow.Context, snapshotName string, - sourceTable string, + sourceTableName string, destinationTableName string, ) (workflow.Future, error) { flowName := s.config.FlowJobName - + srcName := sourceTableName + dstName := destinationTableName childWorkflowIDSideEffect := workflow.SideEffect(childCtx, func(ctx workflow.Context) interface{} { - childWorkflowID := fmt.Sprintf("clone_%s_%s_%s", flowName, destinationTableName, uuid.New().String()) + childWorkflowID := fmt.Sprintf("clone_%s_%s_%s", flowName, dstName, uuid.New().String()) reg := regexp.MustCompile("[^a-zA-Z0-9]+") return reg.ReplaceAllString(childWorkflowID, "_") }) @@ -93,14 +94,14 @@ func (s *SnapshotFlowExecution) cloneTable( "flowName": flowName, "snapshotName": snapshotName, }).Errorf("failed to get child id for source table %s and destination table %s", - sourceTable, destinationTableName) + srcName, dstName) return nil, fmt.Errorf("failed to get child workflow ID: %w", err) } logrus.WithFields(logrus.Fields{ "flowName": flowName, "snapshotName": snapshotName, }).Infof("Obtained child id %s for source table %s and destination table %s", - childWorkflowID, sourceTable, destinationTableName) + childWorkflowID, srcName, dstName) childCtx = workflow.WithChildOptions(childCtx, workflow.ChildWorkflowOptions{ WorkflowID: childWorkflowID, @@ -118,7 +119,7 @@ func (s *SnapshotFlowExecution) cloneTable( sourcePostgres := s.config.Source sourcePostgres.GetPostgresConfig().TransactionSnapshot = snapshotName - query := fmt.Sprintf("SELECT * FROM %s WHERE ctid BETWEEN {{.start}} AND {{.end}}", sourceTable) + query := fmt.Sprintf("SELECT * FROM %s WHERE ctid BETWEEN {{.start}} AND {{.end}}", srcName) numWorkers := uint32(8) if s.config.SnapshotMaxParallelWorkers > 0 { @@ -136,9 +137,9 @@ func (s *SnapshotFlowExecution) cloneTable( DestinationPeer: s.config.Destination, Query: query, WatermarkColumn: "ctid", - WatermarkTable: sourceTable, + WatermarkTable: srcName, InitialCopyOnly: true, - DestinationTableIdentifier: destinationTableName, + DestinationTableIdentifier: dstName, NumRowsPerPartition: numRowsPerPartition, SyncMode: s.config.SnapshotSyncMode, MaxParallelWorkers: numWorkers, From 2190dec48176280c1d996ba38753a4bb9afee302 Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Sun, 27 Aug 2023 04:27:50 -0700 Subject: [PATCH 16/37] Rewrite Bound Selector (#357) Also fix range iterator order --- flow/concurrency/bound_selector.go | 64 +++-- flow/go.mod | 66 ++--- flow/go.sum | 395 ++++++++++++++++++++++------- flow/workflows/qrep_flow.go | 49 ++-- flow/workflows/setup_flow.go | 23 +- flow/workflows/snapshot_flow.go | 47 ++-- 6 files changed, 437 insertions(+), 207 deletions(-) diff --git a/flow/concurrency/bound_selector.go b/flow/concurrency/bound_selector.go index 200a8cb270..56f5b42891 100644 --- a/flow/concurrency/bound_selector.go +++ b/flow/concurrency/bound_selector.go @@ -7,46 +7,54 @@ import ( ) type BoundSelector struct { - ctx workflow.Context - limit int - selector workflow.Selector - futures map[workflow.Future]struct{} - ferrors []error + ctx workflow.Context + limit workflow.Channel + statusCh workflow.Channel + numFutures int } -func NewBoundSelector(limit int, ctx workflow.Context) *BoundSelector { +func NewBoundSelector(limit int, total int, ctx workflow.Context) *BoundSelector { return &BoundSelector{ - ctx: ctx, - limit: limit, - selector: workflow.NewSelector(ctx), - futures: make(map[workflow.Future]struct{}), - ferrors: make([]error, 0), + ctx: ctx, + limit: workflow.NewBufferedChannel(ctx, limit), + statusCh: workflow.NewBufferedChannel(ctx, total), + numFutures: 0, } } -func (s *BoundSelector) AddFuture(future workflow.Future, f func(workflow.Future) error) { - if len(s.futures) >= s.limit { - s.selector.Select(s.ctx) - } - - s.futures[future] = struct{}{} - s.selector.AddFuture(future, func(ready workflow.Future) { - delete(s.futures, ready) - - err := f(ready) - if err != nil { - s.ferrors = append(s.ferrors, err) - } +func (s *BoundSelector) SpawnChild(chCtx workflow.Context, w interface{}, args ...interface{}) { + s.numFutures++ + workflow.Go(s.ctx, func(ctx workflow.Context) { + s.limit.Send(ctx, struct{}{}) + future := workflow.ExecuteChildWorkflow(chCtx, w, args...) + err := future.Get(ctx, nil) + s.statusCh.Send(ctx, err) + s.limit.Receive(ctx, nil) }) } func (s *BoundSelector) Wait() error { - for len(s.futures) > 0 { - s.selector.Select(s.ctx) + defer s.statusCh.Close() + defer s.limit.Close() + + ferrors := make([]error, 0) + doneCount := 0 + + for doneCount < s.numFutures { + selector := workflow.NewSelector(s.ctx) + selector.AddReceive(s.statusCh, func(c workflow.ReceiveChannel, more bool) { + var err error + c.Receive(s.ctx, &err) + if err != nil { + ferrors = append(ferrors, err) + } + doneCount++ + }) + selector.Select(s.ctx) } - if len(s.ferrors) > 0 { - return errors.Join(s.ferrors...) + if len(ferrors) > 0 { + return errors.Join(ferrors...) } return nil diff --git a/flow/go.mod b/flow/go.mod index 3bbd71a923..df3e3ed06f 100644 --- a/flow/go.mod +++ b/flow/go.mod @@ -4,39 +4,39 @@ go 1.19 require ( cloud.google.com/go v0.110.7 - cloud.google.com/go/bigquery v1.53.0 - cloud.google.com/go/storage v1.31.0 + cloud.google.com/go/bigquery v1.54.0 + cloud.google.com/go/storage v1.32.0 github.com/Azure/azure-amqp-common-go/v4 v4.2.0 github.com/Azure/azure-event-hubs-go/v3 v3.6.1 - github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.3.0 + github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.3.1 github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/eventhub/armeventhub v1.1.1 - github.com/aws/aws-sdk-go v1.44.317 - github.com/google/uuid v1.3.0 + github.com/aws/aws-sdk-go v1.44.332 + github.com/google/uuid v1.3.1 github.com/hashicorp/go-multierror v1.1.1 - github.com/jackc/pglogrepl v0.0.0-20230728225306-38e8a4e50913 + github.com/jackc/pglogrepl v0.0.0-20230810221841-d0818e1fbef7 github.com/jackc/pgx/v5 v5.4.3 github.com/jmoiron/sqlx v1.3.5 github.com/joho/godotenv v1.5.1 github.com/lib/pq v1.10.9 github.com/linkedin/goavro/v2 v2.12.0 github.com/microsoft/go-mssqldb v1.5.0 - github.com/orcaman/concurrent-map/v2 v2.0.1 github.com/prometheus/client_golang v1.16.0 github.com/sirupsen/logrus v1.9.3 - github.com/snowflakedb/gosnowflake v1.6.23 + github.com/snowflakedb/gosnowflake v1.6.24 github.com/stretchr/testify v1.8.4 github.com/uber-go/tally/v4 v4.1.7 github.com/urfave/cli/v2 v2.25.7 - go.temporal.io/api v1.23.0 + go.temporal.io/api v1.24.0 go.temporal.io/sdk v1.24.0 go.uber.org/automaxprocs v1.5.3 - google.golang.org/api v0.134.0 + google.golang.org/api v0.138.0 google.golang.org/grpc v1.57.0 google.golang.org/protobuf v1.31.0 ) require ( github.com/golang-jwt/jwt/v5 v5.0.0 // indirect + github.com/grpc-ecosystem/grpc-gateway v1.16.0 // indirect github.com/pkg/errors v0.9.1 // indirect github.com/twmb/murmur3 v1.1.8 // indirect ) @@ -44,11 +44,11 @@ require ( require ( cloud.google.com/go/compute v1.23.0 // indirect cloud.google.com/go/compute/metadata v0.2.3 // indirect - cloud.google.com/go/iam v1.1.1 // indirect + cloud.google.com/go/iam v1.1.2 // indirect github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4 // indirect github.com/99designs/keyring v1.2.2 // indirect github.com/Azure/azure-sdk-for-go v68.0.0+incompatible // indirect - github.com/Azure/azure-sdk-for-go/sdk/azcore v1.7.0 // indirect + github.com/Azure/azure-sdk-for-go/sdk/azcore v1.7.1 // indirect github.com/Azure/azure-sdk-for-go/sdk/internal v1.3.0 // indirect github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.1.0 // indirect github.com/Azure/go-amqp v1.0.1 // indirect @@ -61,24 +61,24 @@ require ( github.com/Azure/go-autorest/autorest/validation v0.3.1 // indirect github.com/Azure/go-autorest/logger v0.2.1 // indirect github.com/Azure/go-autorest/tracing v0.6.0 // indirect - github.com/AzureAD/microsoft-authentication-library-for-go v1.1.0 // indirect + github.com/AzureAD/microsoft-authentication-library-for-go v1.1.1 // indirect github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c // indirect github.com/andybalholm/brotli v1.0.5 // indirect github.com/apache/arrow/go/v12 v12.0.1 // indirect github.com/apache/thrift v0.18.1 // indirect - github.com/aws/aws-sdk-go-v2 v1.20.0 // indirect - github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.11 // indirect - github.com/aws/aws-sdk-go-v2/credentials v1.13.31 // indirect - github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.76 // indirect - github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.37 // indirect - github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.31 // indirect - github.com/aws/aws-sdk-go-v2/internal/v4a v1.1.0 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.12 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.32 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.31 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.0 // indirect - github.com/aws/aws-sdk-go-v2/service/s3 v1.38.1 // indirect - github.com/aws/smithy-go v1.14.0 // indirect + github.com/aws/aws-sdk-go-v2 v1.21.0 // indirect + github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.13 // indirect + github.com/aws/aws-sdk-go-v2/credentials v1.13.35 // indirect + github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.81 // indirect + github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.41 // indirect + github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.35 // indirect + github.com/aws/aws-sdk-go-v2/internal/v4a v1.1.4 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.14 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.36 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.35 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.4 // indirect + github.com/aws/aws-sdk-go-v2/service/s3 v1.38.5 // indirect + github.com/aws/smithy-go v1.14.2 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect github.com/cpuguy83/go-md2man/v2 v2.0.2 // indirect @@ -103,7 +103,7 @@ require ( github.com/golang/snappy v0.0.4 // indirect github.com/google/flatbuffers v23.5.26+incompatible // indirect github.com/google/go-cmp v0.5.9 // indirect - github.com/google/s2a-go v0.1.4 // indirect + github.com/google/s2a-go v0.1.5 // indirect github.com/googleapis/enterprise-certificate-proxy v0.2.5 // indirect github.com/googleapis/gax-go/v2 v2.12.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect @@ -133,14 +133,14 @@ require ( github.com/prometheus/procfs v0.11.1 // indirect github.com/robfig/cron v1.2.0 // indirect github.com/russross/blackfriday/v2 v2.1.0 // indirect - github.com/stretchr/objx v0.5.0 // indirect + github.com/stretchr/objx v0.5.1 // indirect github.com/xrash/smetrics v0.0.0-20201216005158-039620a65673 // indirect github.com/zeebo/xxh3 v1.0.2 // indirect go.opencensus.io v0.24.0 // indirect go.temporal.io/sdk/contrib/tally v0.2.0 go.uber.org/atomic v1.11.0 // indirect golang.org/x/crypto v0.12.0 // indirect - golang.org/x/exp v0.0.0-20230801115018-d63ba01acd4b + golang.org/x/exp v0.0.0-20230817173708-d852ddb80c63 golang.org/x/mod v0.12.0 // indirect golang.org/x/net v0.14.0 // indirect golang.org/x/oauth2 v0.11.0 // indirect @@ -149,11 +149,11 @@ require ( golang.org/x/term v0.11.0 // indirect golang.org/x/text v0.12.0 // indirect golang.org/x/time v0.3.0 // indirect - golang.org/x/tools v0.11.1 // indirect + golang.org/x/tools v0.12.1-0.20230815132531-74c255bcf846 // indirect golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect google.golang.org/appengine v1.6.7 // indirect - google.golang.org/genproto v0.0.0-20230803162519-f966b187b2e5 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20230803162519-f966b187b2e5 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20230803162519-f966b187b2e5 // indirect + google.golang.org/genproto v0.0.0-20230822172742-b8732ec3820d // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20230822172742-b8732ec3820d // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20230822172742-b8732ec3820d // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/flow/go.sum b/flow/go.sum index 46fb293734..ee6182af9b 100644 --- a/flow/go.sum +++ b/flow/go.sum @@ -36,37 +36,48 @@ cloud.google.com/go v0.104.0/go.mod h1:OO6xxXdJyvuJPcEPBLN9BJPD+jep5G1+2U5B5gkRY cloud.google.com/go v0.105.0/go.mod h1:PrLgOJNe5nfE9UMxKxgXj4mD3voiP+YQ6gdt6KMFOKM= cloud.google.com/go v0.107.0/go.mod h1:wpc2eNrD7hXUTy8EKS10jkxpZBjASrORK7goS+3YX2I= cloud.google.com/go v0.110.0/go.mod h1:SJnCLqQ0FCFGSZMUNUf84MV3Aia54kn7pi8st7tMzaY= -cloud.google.com/go v0.110.6 h1:8uYAkj3YHTP/1iwReuHPxLSbdcyc+dSBbzFMrVwDR6Q= +cloud.google.com/go v0.110.2/go.mod h1:k04UEeEtb6ZBRTv3dZz4CeJC3jKGxyhl0sAiVVquxiw= +cloud.google.com/go v0.110.4/go.mod h1:+EYjdK8e5RME/VY/qLCAtuyALQ9q67dvuum8i+H5xsI= cloud.google.com/go v0.110.6/go.mod h1:+EYjdK8e5RME/VY/qLCAtuyALQ9q67dvuum8i+H5xsI= cloud.google.com/go v0.110.7 h1:rJyC7nWRg2jWGZ4wSJ5nY65GTdYJkg0cd/uXb+ACI6o= cloud.google.com/go v0.110.7/go.mod h1:+EYjdK8e5RME/VY/qLCAtuyALQ9q67dvuum8i+H5xsI= cloud.google.com/go/accessapproval v1.4.0/go.mod h1:zybIuC3KpDOvotz59lFe5qxRZx6C75OtwbisN56xYB4= cloud.google.com/go/accessapproval v1.5.0/go.mod h1:HFy3tuiGvMdcd/u+Cu5b9NkO1pEICJ46IR82PoUdplw= cloud.google.com/go/accessapproval v1.6.0/go.mod h1:R0EiYnwV5fsRFiKZkPHr6mwyk2wxUJ30nL4j2pcFY2E= +cloud.google.com/go/accessapproval v1.7.1/go.mod h1:JYczztsHRMK7NTXb6Xw+dwbs/WnOJxbo/2mTI+Kgg68= cloud.google.com/go/accesscontextmanager v1.3.0/go.mod h1:TgCBehyr5gNMz7ZaH9xubp+CE8dkrszb4oK9CWyvD4o= cloud.google.com/go/accesscontextmanager v1.4.0/go.mod h1:/Kjh7BBu/Gh83sv+K60vN9QE5NJcd80sU33vIe2IFPE= cloud.google.com/go/accesscontextmanager v1.6.0/go.mod h1:8XCvZWfYw3K/ji0iVnp+6pu7huxoQTLmxAbVjbloTtM= cloud.google.com/go/accesscontextmanager v1.7.0/go.mod h1:CEGLewx8dwa33aDAZQujl7Dx+uYhS0eay198wB/VumQ= +cloud.google.com/go/accesscontextmanager v1.8.0/go.mod h1:uI+AI/r1oyWK99NN8cQ3UK76AMelMzgZCvJfsi2c+ps= +cloud.google.com/go/accesscontextmanager v1.8.1/go.mod h1:JFJHfvuaTC+++1iL1coPiG1eu5D24db2wXCDWDjIrxo= cloud.google.com/go/aiplatform v1.22.0/go.mod h1:ig5Nct50bZlzV6NvKaTwmplLLddFx0YReh9WfTO5jKw= cloud.google.com/go/aiplatform v1.24.0/go.mod h1:67UUvRBKG6GTayHKV8DBv2RtR1t93YRu5B1P3x99mYY= cloud.google.com/go/aiplatform v1.27.0/go.mod h1:Bvxqtl40l0WImSb04d0hXFU7gDOiq9jQmorivIiWcKg= cloud.google.com/go/aiplatform v1.35.0/go.mod h1:7MFT/vCaOyZT/4IIFfxH4ErVg/4ku6lKv3w0+tFTgXQ= cloud.google.com/go/aiplatform v1.36.1/go.mod h1:WTm12vJRPARNvJ+v6P52RDHCNe4AhvjcIZ/9/RRHy/k= cloud.google.com/go/aiplatform v1.37.0/go.mod h1:IU2Cv29Lv9oCn/9LkFiiuKfwrRTq+QQMbW+hPCxJGZw= +cloud.google.com/go/aiplatform v1.45.0/go.mod h1:Iu2Q7sC7QGhXUeOhAj/oCK9a+ULz1O4AotZiqjQ8MYA= +cloud.google.com/go/aiplatform v1.48.0/go.mod h1:Iu2Q7sC7QGhXUeOhAj/oCK9a+ULz1O4AotZiqjQ8MYA= cloud.google.com/go/analytics v0.11.0/go.mod h1:DjEWCu41bVbYcKyvlws9Er60YE4a//bK6mnhWvQeFNI= cloud.google.com/go/analytics v0.12.0/go.mod h1:gkfj9h6XRf9+TS4bmuhPEShsh3hH8PAZzm/41OOhQd4= cloud.google.com/go/analytics v0.17.0/go.mod h1:WXFa3WSym4IZ+JiKmavYdJwGG/CvpqiqczmL59bTD9M= cloud.google.com/go/analytics v0.18.0/go.mod h1:ZkeHGQlcIPkw0R/GW+boWHhCOR43xz9RN/jn7WcqfIE= cloud.google.com/go/analytics v0.19.0/go.mod h1:k8liqf5/HCnOUkbawNtrWWc+UAzyDlW89doe8TtoDsE= +cloud.google.com/go/analytics v0.21.2/go.mod h1:U8dcUtmDmjrmUTnnnRnI4m6zKn/yaA5N9RlEkYFHpQo= +cloud.google.com/go/analytics v0.21.3/go.mod h1:U8dcUtmDmjrmUTnnnRnI4m6zKn/yaA5N9RlEkYFHpQo= cloud.google.com/go/apigateway v1.3.0/go.mod h1:89Z8Bhpmxu6AmUxuVRg/ECRGReEdiP3vQtk4Z1J9rJk= cloud.google.com/go/apigateway v1.4.0/go.mod h1:pHVY9MKGaH9PQ3pJ4YLzoj6U5FUDeDFBllIz7WmzJoc= cloud.google.com/go/apigateway v1.5.0/go.mod h1:GpnZR3Q4rR7LVu5951qfXPJCHquZt02jf7xQx7kpqN8= +cloud.google.com/go/apigateway v1.6.1/go.mod h1:ufAS3wpbRjqfZrzpvLC2oh0MFlpRJm2E/ts25yyqmXA= cloud.google.com/go/apigeeconnect v1.3.0/go.mod h1:G/AwXFAKo0gIXkPTVfZDd2qA1TxBXJ3MgMRBQkIi9jc= cloud.google.com/go/apigeeconnect v1.4.0/go.mod h1:kV4NwOKqjvt2JYR0AoIWo2QGfoRtn/pkS3QlHp0Ni04= cloud.google.com/go/apigeeconnect v1.5.0/go.mod h1:KFaCqvBRU6idyhSNyn3vlHXc8VMDJdRmwDF6JyFRqZ8= +cloud.google.com/go/apigeeconnect v1.6.1/go.mod h1:C4awq7x0JpLtrlQCr8AzVIzAaYgngRqWf9S5Uhg+wWs= cloud.google.com/go/apigeeregistry v0.4.0/go.mod h1:EUG4PGcsZvxOXAdyEghIdXwAEi/4MEaoqLMLDMIwKXY= cloud.google.com/go/apigeeregistry v0.5.0/go.mod h1:YR5+s0BVNZfVOUkMa5pAR2xGd0A473vA5M7j247o1wM= cloud.google.com/go/apigeeregistry v0.6.0/go.mod h1:BFNzW7yQVLZ3yj0TKcwzb8n25CFBri51GVGOEUcgQsc= +cloud.google.com/go/apigeeregistry v0.7.1/go.mod h1:1XgyjZye4Mqtw7T9TsY4NW10U7BojBvG4RMD+vRDrIw= cloud.google.com/go/apikeys v0.4.0/go.mod h1:XATS/yqZbaBK0HOssf+ALHp8jAlNHUgyfprvNcBIszU= cloud.google.com/go/apikeys v0.5.0/go.mod h1:5aQfwY4D+ewMMWScd3hm2en3hCj+BROlyrt3ytS7KLI= cloud.google.com/go/apikeys v0.6.0/go.mod h1:kbpXu5upyiAlGkKrJgQl8A0rKNNJ7dQ377pdroRSSi8= @@ -75,10 +86,12 @@ cloud.google.com/go/appengine v1.5.0/go.mod h1:TfasSozdkFI0zeoxW3PTBLiNqRmzraodC cloud.google.com/go/appengine v1.6.0/go.mod h1:hg6i0J/BD2cKmDJbaFSYHFyZkgBEfQrDg/X0V5fJn84= cloud.google.com/go/appengine v1.7.0/go.mod h1:eZqpbHFCqRGa2aCdope7eC0SWLV1j0neb/QnMJVWx6A= cloud.google.com/go/appengine v1.7.1/go.mod h1:IHLToyb/3fKutRysUlFO0BPt5j7RiQ45nrzEJmKTo6E= +cloud.google.com/go/appengine v1.8.1/go.mod h1:6NJXGLVhZCN9aQ/AEDvmfzKEfoYBlfB80/BHiKVputY= cloud.google.com/go/area120 v0.5.0/go.mod h1:DE/n4mp+iqVyvxHN41Vf1CR602GiHQjFPusMFW6bGR4= cloud.google.com/go/area120 v0.6.0/go.mod h1:39yFJqWVgm0UZqWTOdqkLhjoC7uFfgXRC8g/ZegeAh0= cloud.google.com/go/area120 v0.7.0/go.mod h1:a3+8EUD1SX5RUcCs3MY5YasiO1z6yLiNLRiFrykbynY= cloud.google.com/go/area120 v0.7.1/go.mod h1:j84i4E1RboTWjKtZVWXPqvK5VHQFJRF2c1Nm69pWm9k= +cloud.google.com/go/area120 v0.8.1/go.mod h1:BVfZpGpB7KFVNxPiQBuHkX6Ed0rS51xIgmGyjrAfzsg= cloud.google.com/go/artifactregistry v1.6.0/go.mod h1:IYt0oBPSAGYj/kprzsBjZ/4LnG/zOcHyFHjWPCi6SAQ= cloud.google.com/go/artifactregistry v1.7.0/go.mod h1:mqTOFOnGZx8EtSqK/ZWcsm/4U8B77rbcLP6ruDU2Ixk= cloud.google.com/go/artifactregistry v1.8.0/go.mod h1:w3GQXkJX8hiKN0v+at4b0qotwijQbYUqF2GWkZzAhC0= @@ -87,6 +100,7 @@ cloud.google.com/go/artifactregistry v1.11.1/go.mod h1:lLYghw+Itq9SONbCa1YWBoWs1 cloud.google.com/go/artifactregistry v1.11.2/go.mod h1:nLZns771ZGAwVLzTX/7Al6R9ehma4WUEhZGWV6CeQNQ= cloud.google.com/go/artifactregistry v1.12.0/go.mod h1:o6P3MIvtzTOnmvGagO9v/rOjjA0HmhJ+/6KAXrmYDCI= cloud.google.com/go/artifactregistry v1.13.0/go.mod h1:uy/LNfoOIivepGhooAUpL1i30Hgee3Cu0l4VTWHUC08= +cloud.google.com/go/artifactregistry v1.14.1/go.mod h1:nxVdG19jTaSTu7yA7+VbWL346r3rIdkZ142BSQqhn5E= cloud.google.com/go/asset v1.5.0/go.mod h1:5mfs8UvcM5wHhqtSv8J1CtxxaQq3AdBxxQi2jGW/K4o= cloud.google.com/go/asset v1.7.0/go.mod h1:YbENsRK4+xTiL+Ofoj5Ckf+O17kJtgp3Y3nn4uzZz5s= cloud.google.com/go/asset v1.8.0/go.mod h1:mUNGKhiqIdbr8X7KNayoYvyc4HbbFO9URsjbytpUaW0= @@ -95,27 +109,34 @@ cloud.google.com/go/asset v1.10.0/go.mod h1:pLz7uokL80qKhzKr4xXGvBQXnzHn5evJAEAt cloud.google.com/go/asset v1.11.1/go.mod h1:fSwLhbRvC9p9CXQHJ3BgFeQNM4c9x10lqlrdEUYXlJo= cloud.google.com/go/asset v1.12.0/go.mod h1:h9/sFOa4eDIyKmH6QMpm4eUK3pDojWnUhTgJlk762Hg= cloud.google.com/go/asset v1.13.0/go.mod h1:WQAMyYek/b7NBpYq/K4KJWcRqzoalEsxz/t/dTk4THw= +cloud.google.com/go/asset v1.14.1/go.mod h1:4bEJ3dnHCqWCDbWJ/6Vn7GVI9LerSi7Rfdi03hd+WTQ= cloud.google.com/go/assuredworkloads v1.5.0/go.mod h1:n8HOZ6pff6re5KYfBXcFvSViQjDwxFkAkmUFffJRbbY= cloud.google.com/go/assuredworkloads v1.6.0/go.mod h1:yo2YOk37Yc89Rsd5QMVECvjaMKymF9OP+QXWlKXUkXw= cloud.google.com/go/assuredworkloads v1.7.0/go.mod h1:z/736/oNmtGAyU47reJgGN+KVoYoxeLBoj4XkKYscNI= cloud.google.com/go/assuredworkloads v1.8.0/go.mod h1:AsX2cqyNCOvEQC8RMPnoc0yEarXQk6WEKkxYfL6kGIo= cloud.google.com/go/assuredworkloads v1.9.0/go.mod h1:kFuI1P78bplYtT77Tb1hi0FMxM0vVpRC7VVoJC3ZoT0= cloud.google.com/go/assuredworkloads v1.10.0/go.mod h1:kwdUQuXcedVdsIaKgKTp9t0UJkE5+PAVNhdQm4ZVq2E= +cloud.google.com/go/assuredworkloads v1.11.1/go.mod h1:+F04I52Pgn5nmPG36CWFtxmav6+7Q+c5QyJoL18Lry0= cloud.google.com/go/automl v1.5.0/go.mod h1:34EjfoFGMZ5sgJ9EoLsRtdPSNZLcfflJR39VbVNS2M0= cloud.google.com/go/automl v1.6.0/go.mod h1:ugf8a6Fx+zP0D59WLhqgTDsQI9w07o64uf/Is3Nh5p8= cloud.google.com/go/automl v1.7.0/go.mod h1:RL9MYCCsJEOmt0Wf3z9uzG0a7adTT1fe+aObgSpkCt8= cloud.google.com/go/automl v1.8.0/go.mod h1:xWx7G/aPEe/NP+qzYXktoBSDfjO+vnKMGgsApGJJquM= cloud.google.com/go/automl v1.12.0/go.mod h1:tWDcHDp86aMIuHmyvjuKeeHEGq76lD7ZqfGLN6B0NuU= +cloud.google.com/go/automl v1.13.1/go.mod h1:1aowgAHWYZU27MybSCFiukPO7xnyawv7pt3zK4bheQE= cloud.google.com/go/baremetalsolution v0.3.0/go.mod h1:XOrocE+pvK1xFfleEnShBlNAXf+j5blPPxrhjKgnIFc= cloud.google.com/go/baremetalsolution v0.4.0/go.mod h1:BymplhAadOO/eBa7KewQ0Ppg4A4Wplbn+PsFKRLo0uI= cloud.google.com/go/baremetalsolution v0.5.0/go.mod h1:dXGxEkmR9BMwxhzBhV0AioD0ULBmuLZI8CdwalUxuss= +cloud.google.com/go/baremetalsolution v1.1.1/go.mod h1:D1AV6xwOksJMV4OSlWHtWuFNZZYujJknMAP4Qa27QIA= cloud.google.com/go/batch v0.3.0/go.mod h1:TR18ZoAekj1GuirsUsR1ZTKN3FC/4UDnScjT8NXImFE= cloud.google.com/go/batch v0.4.0/go.mod h1:WZkHnP43R/QCGQsZ+0JyG4i79ranE2u8xvjq/9+STPE= cloud.google.com/go/batch v0.7.0/go.mod h1:vLZN95s6teRUqRQ4s3RLDsH8PvboqBK+rn1oevL159g= +cloud.google.com/go/batch v1.3.1/go.mod h1:VguXeQKXIYaeeIYbuozUmBR13AfL4SJP7IltNPS+A4A= cloud.google.com/go/beyondcorp v0.2.0/go.mod h1:TB7Bd+EEtcw9PCPQhCJtJGjk/7TC6ckmnSFS+xwTfm4= cloud.google.com/go/beyondcorp v0.3.0/go.mod h1:E5U5lcrcXMsCuoDNyGrpyTm/hn7ne941Jz2vmksAxW8= cloud.google.com/go/beyondcorp v0.4.0/go.mod h1:3ApA0mbhHx6YImmuubf5pyW8srKnCEPON32/5hj+RmM= cloud.google.com/go/beyondcorp v0.5.0/go.mod h1:uFqj9X+dSfrheVp7ssLTaRHd2EHqSL4QZmH4e8WXGGU= +cloud.google.com/go/beyondcorp v0.6.1/go.mod h1:YhxDWw946SCbmcWo3fAhw3V4XZMSpQ/VYfcKGAEU8/4= +cloud.google.com/go/beyondcorp v1.0.0/go.mod h1:YhxDWw946SCbmcWo3fAhw3V4XZMSpQ/VYfcKGAEU8/4= cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o= cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNFKH1/VBDHE= cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvftPBK2Dvzc= @@ -129,42 +150,52 @@ cloud.google.com/go/bigquery v1.47.0/go.mod h1:sA9XOgy0A8vQK9+MWhEQTY6Tix87M/Zur cloud.google.com/go/bigquery v1.48.0/go.mod h1:QAwSz+ipNgfL5jxiaK7weyOhzdoAy1zFm0Nf1fysJac= cloud.google.com/go/bigquery v1.49.0/go.mod h1:Sv8hMmTFFYBlt/ftw2uN6dFdQPzBlREY9yBh7Oy7/4Q= cloud.google.com/go/bigquery v1.50.0/go.mod h1:YrleYEh2pSEbgTBZYMJ5SuSr0ML3ypjRB1zgf7pvQLU= -cloud.google.com/go/bigquery v1.52.0 h1:JKLNdxI0N+TIUWD6t9KN646X27N5dQWq9dZbbTWZ8hc= cloud.google.com/go/bigquery v1.52.0/go.mod h1:3b/iXjRQGU4nKa87cXeg6/gogLjO8C6PmuM8i5Bi/u4= cloud.google.com/go/bigquery v1.53.0 h1:K3wLbjbnSlxhuG5q4pntHv5AEbQM1QqHKGYgwFIqOTg= cloud.google.com/go/bigquery v1.53.0/go.mod h1:3b/iXjRQGU4nKa87cXeg6/gogLjO8C6PmuM8i5Bi/u4= +cloud.google.com/go/bigquery v1.54.0 h1:ify6s7sy+kQuAimRnVTrPUzaeY0+X5GEsKt2C5CiA8w= +cloud.google.com/go/bigquery v1.54.0/go.mod h1:9Y5I3PN9kQWuid6183JFhOGOW3GcirA5LpsKCUn+2ec= cloud.google.com/go/billing v1.4.0/go.mod h1:g9IdKBEFlItS8bTtlrZdVLWSSdSyFUZKXNS02zKMOZY= cloud.google.com/go/billing v1.5.0/go.mod h1:mztb1tBc3QekhjSgmpf/CV4LzWXLzCArwpLmP2Gm88s= cloud.google.com/go/billing v1.6.0/go.mod h1:WoXzguj+BeHXPbKfNWkqVtDdzORazmCjraY+vrxcyvI= cloud.google.com/go/billing v1.7.0/go.mod h1:q457N3Hbj9lYwwRbnlD7vUpyjq6u5U1RAOArInEiD5Y= cloud.google.com/go/billing v1.12.0/go.mod h1:yKrZio/eu+okO/2McZEbch17O5CB5NpZhhXG6Z766ss= cloud.google.com/go/billing v1.13.0/go.mod h1:7kB2W9Xf98hP9Sr12KfECgfGclsH3CQR0R08tnRlRbc= +cloud.google.com/go/billing v1.16.0/go.mod h1:y8vx09JSSJG02k5QxbycNRrN7FGZB6F3CAcgum7jvGA= cloud.google.com/go/binaryauthorization v1.1.0/go.mod h1:xwnoWu3Y84jbuHa0zd526MJYmtnVXn0syOjaJgy4+dM= cloud.google.com/go/binaryauthorization v1.2.0/go.mod h1:86WKkJHtRcv5ViNABtYMhhNWRrD1Vpi//uKEy7aYEfI= cloud.google.com/go/binaryauthorization v1.3.0/go.mod h1:lRZbKgjDIIQvzYQS1p99A7/U1JqvqeZg0wiI5tp6tg0= cloud.google.com/go/binaryauthorization v1.4.0/go.mod h1:tsSPQrBd77VLplV70GUhBf/Zm3FsKmgSqgm4UmiDItk= cloud.google.com/go/binaryauthorization v1.5.0/go.mod h1:OSe4OU1nN/VswXKRBmciKpo9LulY41gch5c68htf3/Q= +cloud.google.com/go/binaryauthorization v1.6.1/go.mod h1:TKt4pa8xhowwffiBmbrbcxijJRZED4zrqnwZ1lKH51U= cloud.google.com/go/certificatemanager v1.3.0/go.mod h1:n6twGDvcUBFu9uBgt4eYvvf3sQ6My8jADcOVwHmzadg= cloud.google.com/go/certificatemanager v1.4.0/go.mod h1:vowpercVFyqs8ABSmrdV+GiFf2H/ch3KyudYQEMM590= cloud.google.com/go/certificatemanager v1.6.0/go.mod h1:3Hh64rCKjRAX8dXgRAyOcY5vQ/fE1sh8o+Mdd6KPgY8= +cloud.google.com/go/certificatemanager v1.7.1/go.mod h1:iW8J3nG6SaRYImIa+wXQ0g8IgoofDFRp5UMzaNk1UqI= cloud.google.com/go/channel v1.8.0/go.mod h1:W5SwCXDJsq/rg3tn3oG0LOxpAo6IMxNa09ngphpSlnk= cloud.google.com/go/channel v1.9.0/go.mod h1:jcu05W0my9Vx4mt3/rEHpfxc9eKi9XwsdDL8yBMbKUk= cloud.google.com/go/channel v1.11.0/go.mod h1:IdtI0uWGqhEeatSB62VOoJ8FSUhJ9/+iGkJVqp74CGE= cloud.google.com/go/channel v1.12.0/go.mod h1:VkxCGKASi4Cq7TbXxlaBezonAYpp1GCnKMY6tnMQnLU= +cloud.google.com/go/channel v1.16.0/go.mod h1:eN/q1PFSl5gyu0dYdmxNXscY/4Fi7ABmeHCJNf/oHmc= cloud.google.com/go/cloudbuild v1.3.0/go.mod h1:WequR4ULxlqvMsjDEEEFnOG5ZSRSgWOywXYDb1vPE6U= cloud.google.com/go/cloudbuild v1.4.0/go.mod h1:5Qwa40LHiOXmz3386FrjrYM93rM/hdRr7b53sySrTqA= cloud.google.com/go/cloudbuild v1.6.0/go.mod h1:UIbc/w9QCbH12xX+ezUsgblrWv+Cv4Tw83GiSMHOn9M= cloud.google.com/go/cloudbuild v1.7.0/go.mod h1:zb5tWh2XI6lR9zQmsm1VRA+7OCuve5d8S+zJUul8KTg= cloud.google.com/go/cloudbuild v1.9.0/go.mod h1:qK1d7s4QlO0VwfYn5YuClDGg2hfmLZEb4wQGAbIgL1s= +cloud.google.com/go/cloudbuild v1.10.1/go.mod h1:lyJg7v97SUIPq4RC2sGsz/9tNczhyv2AjML/ci4ulzU= +cloud.google.com/go/cloudbuild v1.13.0/go.mod h1:lyJg7v97SUIPq4RC2sGsz/9tNczhyv2AjML/ci4ulzU= cloud.google.com/go/clouddms v1.3.0/go.mod h1:oK6XsCDdW4Ib3jCCBugx+gVjevp2TMXFtgxvPSee3OM= cloud.google.com/go/clouddms v1.4.0/go.mod h1:Eh7sUGCC+aKry14O1NRljhjyrr0NFC0G2cjwX0cByRk= cloud.google.com/go/clouddms v1.5.0/go.mod h1:QSxQnhikCLUw13iAbffF2CZxAER3xDGNHjsTAkQJcQA= +cloud.google.com/go/clouddms v1.6.1/go.mod h1:Ygo1vL52Ov4TBZQquhz5fiw2CQ58gvu+PlS6PVXCpZI= cloud.google.com/go/cloudtasks v1.5.0/go.mod h1:fD92REy1x5woxkKEkLdvavGnPJGEn8Uic9nWuLzqCpY= cloud.google.com/go/cloudtasks v1.6.0/go.mod h1:C6Io+sxuke9/KNRkbQpihnW93SWDU3uXt92nu85HkYI= cloud.google.com/go/cloudtasks v1.7.0/go.mod h1:ImsfdYWwlWNJbdgPIIGJWC+gemEGTBK/SunNQQNCAb4= cloud.google.com/go/cloudtasks v1.8.0/go.mod h1:gQXUIwCSOI4yPVK7DgTVFiiP0ZW/eQkydWzwVMdHxrI= cloud.google.com/go/cloudtasks v1.9.0/go.mod h1:w+EyLsVkLWHcOaqNEyvcKAsWp9p29dL6uL9Nst1cI7Y= cloud.google.com/go/cloudtasks v1.10.0/go.mod h1:NDSoTLkZ3+vExFEWu2UJV1arUyzVDAiZtdWcsUyNwBs= +cloud.google.com/go/cloudtasks v1.11.1/go.mod h1:a9udmnou9KO2iulGscKR0qBYjreuX8oHwpmFsKspEvM= +cloud.google.com/go/cloudtasks v1.12.1/go.mod h1:a9udmnou9KO2iulGscKR0qBYjreuX8oHwpmFsKspEvM= cloud.google.com/go/compute v0.1.0/go.mod h1:GAesmwr110a34z04OlxYkATPBEfVhkymfTBXtfbBFow= cloud.google.com/go/compute v1.3.0/go.mod h1:cCZiE1NHEtai4wiufUhW8I8S1JKkAnhnQJWM7YD99wM= cloud.google.com/go/compute v1.5.0/go.mod h1:9SMHyhJlzhlkJqrPAc839t2BZFTSk6Jdj6mkzQJeu0M= @@ -179,8 +210,9 @@ cloud.google.com/go/compute v1.14.0/go.mod h1:YfLtxrj9sU4Yxv+sXzZkyPjEyPBZfXHUvj cloud.google.com/go/compute v1.15.1/go.mod h1:bjjoF/NtFUrkD/urWfdHaKuOPDR5nWIs63rR+SXhcpA= cloud.google.com/go/compute v1.18.0/go.mod h1:1X7yHxec2Ga+Ss6jPyjxRxpu2uu7PLgsOVXvgU0yacs= cloud.google.com/go/compute v1.19.0/go.mod h1:rikpw2y+UMidAe9tISo04EHNOIf42RLYF/q8Bs93scU= -cloud.google.com/go/compute v1.21.0 h1:JNBsyXVoOoNJtTQcnEY5uYpZIbeCTYIeDe0Xh1bySMk= -cloud.google.com/go/compute v1.21.0/go.mod h1:4tCnrn48xsqlwSAiLf1HXMQk8CONslYbdiEZc9FEIbM= +cloud.google.com/go/compute v1.19.1/go.mod h1:6ylj3a05WF8leseCdIf77NK0g1ey+nj5IKd5/kvShxE= +cloud.google.com/go/compute v1.19.3/go.mod h1:qxvISKp/gYnXkSAD1ppcSOveRAmzxicEv/JlizULFrI= +cloud.google.com/go/compute v1.20.1/go.mod h1:4tCnrn48xsqlwSAiLf1HXMQk8CONslYbdiEZc9FEIbM= cloud.google.com/go/compute v1.23.0 h1:tP41Zoavr8ptEqaW6j+LQOnyBBhO7OkOMAGrgLopTwY= cloud.google.com/go/compute v1.23.0/go.mod h1:4tCnrn48xsqlwSAiLf1HXMQk8CONslYbdiEZc9FEIbM= cloud.google.com/go/compute/metadata v0.1.0/go.mod h1:Z1VN+bulIf6bt4P/C37K4DyZYZEXYonfTBHHFPO/4UU= @@ -191,15 +223,20 @@ cloud.google.com/go/compute/metadata v0.2.3/go.mod h1:VAV5nSsACxMJvgaAuX6Pk2Aawl cloud.google.com/go/contactcenterinsights v1.3.0/go.mod h1:Eu2oemoePuEFc/xKFPjbTuPSj0fYJcPls9TFlPNnHHY= cloud.google.com/go/contactcenterinsights v1.4.0/go.mod h1:L2YzkGbPsv+vMQMCADxJoT9YiTTnSEd6fEvCeHTYVck= cloud.google.com/go/contactcenterinsights v1.6.0/go.mod h1:IIDlT6CLcDoyv79kDv8iWxMSTZhLxSCofVV5W6YFM/w= +cloud.google.com/go/contactcenterinsights v1.9.1/go.mod h1:bsg/R7zGLYMVxFFzfh9ooLTruLRCG9fnzhH9KznHhbM= +cloud.google.com/go/contactcenterinsights v1.10.0/go.mod h1:bsg/R7zGLYMVxFFzfh9ooLTruLRCG9fnzhH9KznHhbM= cloud.google.com/go/container v1.6.0/go.mod h1:Xazp7GjJSeUYo688S+6J5V+n/t+G5sKBTFkKNudGRxg= cloud.google.com/go/container v1.7.0/go.mod h1:Dp5AHtmothHGX3DwwIHPgq45Y8KmNsgN3amoYfxVkLo= cloud.google.com/go/container v1.13.1/go.mod h1:6wgbMPeQRw9rSnKBCAJXnds3Pzj03C4JHamr8asWKy4= cloud.google.com/go/container v1.14.0/go.mod h1:3AoJMPhHfLDxLvrlVWaK57IXzaPnLaZq63WX59aQBfM= cloud.google.com/go/container v1.15.0/go.mod h1:ft+9S0WGjAyjDggg5S06DXj+fHJICWg8L7isCQe9pQA= +cloud.google.com/go/container v1.22.1/go.mod h1:lTNExE2R7f+DLbAN+rJiKTisauFCaoDq6NURZ83eVH4= +cloud.google.com/go/container v1.24.0/go.mod h1:lTNExE2R7f+DLbAN+rJiKTisauFCaoDq6NURZ83eVH4= cloud.google.com/go/containeranalysis v0.5.1/go.mod h1:1D92jd8gRR/c0fGMlymRgxWD3Qw9C1ff6/T7mLgVL8I= cloud.google.com/go/containeranalysis v0.6.0/go.mod h1:HEJoiEIu+lEXM+k7+qLCci0h33lX3ZqoYFdmPcoO7s4= cloud.google.com/go/containeranalysis v0.7.0/go.mod h1:9aUL+/vZ55P2CXfuZjS4UjQ9AgXoSw8Ts6lemfmxBxI= cloud.google.com/go/containeranalysis v0.9.0/go.mod h1:orbOANbwk5Ejoom+s+DUCTTJ7IBdBQJDcSylAx/on9s= +cloud.google.com/go/containeranalysis v0.10.1/go.mod h1:Ya2jiILITMY68ZLPaogjmOMNkwsDrWBSTyBubGXO7j0= cloud.google.com/go/datacatalog v1.3.0/go.mod h1:g9svFY6tuR+j+hrTw3J2dNcmI0dzmSiyOzm8kpLq0a0= cloud.google.com/go/datacatalog v1.5.0/go.mod h1:M7GPLNQeLfWqeIm3iuiruhPzkt65+Bx8dAKvScX8jvs= cloud.google.com/go/datacatalog v1.6.0/go.mod h1:+aEyF8JKg+uXcIdAmmaMUmZ3q1b/lKLtXCmXdnc0lbc= @@ -208,46 +245,63 @@ cloud.google.com/go/datacatalog v1.8.0/go.mod h1:KYuoVOv9BM8EYz/4eMFxrr4DUKhGIOX cloud.google.com/go/datacatalog v1.8.1/go.mod h1:RJ58z4rMp3gvETA465Vg+ag8BGgBdnRPEMMSTr5Uv+M= cloud.google.com/go/datacatalog v1.12.0/go.mod h1:CWae8rFkfp6LzLumKOnmVh4+Zle4A3NXLzVJ1d1mRm0= cloud.google.com/go/datacatalog v1.13.0/go.mod h1:E4Rj9a5ZtAxcQJlEBTLgMTphfP11/lNaAshpoBgemX8= -cloud.google.com/go/datacatalog v1.14.1 h1:cFPBt8V5V2T3mu/96tc4nhcMB+5cYcpwjBfn79bZDI8= +cloud.google.com/go/datacatalog v1.14.0/go.mod h1:h0PrGtlihoutNMp/uvwhawLQ9+c63Kz65UFqh49Yo+E= +cloud.google.com/go/datacatalog v1.14.1/go.mod h1:d2CevwTG4yedZilwe+v3E3ZBDRMobQfSG/a6cCCN5R4= cloud.google.com/go/datacatalog v1.16.0 h1:qVeQcw1Cz93/cGu2E7TYUPh8Lz5dn5Ws2siIuQ17Vng= +cloud.google.com/go/datacatalog v1.16.0/go.mod h1:d2CevwTG4yedZilwe+v3E3ZBDRMobQfSG/a6cCCN5R4= cloud.google.com/go/dataflow v0.6.0/go.mod h1:9QwV89cGoxjjSR9/r7eFDqqjtvbKxAK2BaYU6PVk9UM= cloud.google.com/go/dataflow v0.7.0/go.mod h1:PX526vb4ijFMesO1o202EaUmouZKBpjHsTlCtB4parQ= cloud.google.com/go/dataflow v0.8.0/go.mod h1:Rcf5YgTKPtQyYz8bLYhFoIV/vP39eL7fWNcSOyFfLJE= +cloud.google.com/go/dataflow v0.9.1/go.mod h1:Wp7s32QjYuQDWqJPFFlnBKhkAtiFpMTdg00qGbnIHVw= cloud.google.com/go/dataform v0.3.0/go.mod h1:cj8uNliRlHpa6L3yVhDOBrUXH+BPAO1+KFMQQNSThKo= cloud.google.com/go/dataform v0.4.0/go.mod h1:fwV6Y4Ty2yIFL89huYlEkwUPtS7YZinZbzzj5S9FzCE= cloud.google.com/go/dataform v0.5.0/go.mod h1:GFUYRe8IBa2hcomWplodVmUx/iTL0FrsauObOM3Ipr0= cloud.google.com/go/dataform v0.6.0/go.mod h1:QPflImQy33e29VuapFdf19oPbE4aYTJxr31OAPV+ulA= cloud.google.com/go/dataform v0.7.0/go.mod h1:7NulqnVozfHvWUBpMDfKMUESr+85aJsC/2O0o3jWPDE= +cloud.google.com/go/dataform v0.8.1/go.mod h1:3BhPSiw8xmppbgzeBbmDvmSWlwouuJkXsXsb8UBih9M= cloud.google.com/go/datafusion v1.4.0/go.mod h1:1Zb6VN+W6ALo85cXnM1IKiPw+yQMKMhB9TsTSRDo/38= cloud.google.com/go/datafusion v1.5.0/go.mod h1:Kz+l1FGHB0J+4XF2fud96WMmRiq/wj8N9u007vyXZ2w= cloud.google.com/go/datafusion v1.6.0/go.mod h1:WBsMF8F1RhSXvVM8rCV3AeyWVxcC2xY6vith3iw3S+8= +cloud.google.com/go/datafusion v1.7.1/go.mod h1:KpoTBbFmoToDExJUso/fcCiguGDk7MEzOWXUsJo0wsI= cloud.google.com/go/datalabeling v0.5.0/go.mod h1:TGcJ0G2NzcsXSE/97yWjIZO0bXj0KbVlINXMG9ud42I= cloud.google.com/go/datalabeling v0.6.0/go.mod h1:WqdISuk/+WIGeMkpw/1q7bK/tFEZxsrFJOJdY2bXvTQ= cloud.google.com/go/datalabeling v0.7.0/go.mod h1:WPQb1y08RJbmpM3ww0CSUAGweL0SxByuW2E+FU+wXcM= +cloud.google.com/go/datalabeling v0.8.1/go.mod h1:XS62LBSVPbYR54GfYQsPXZjTW8UxCK2fkDciSrpRFdY= cloud.google.com/go/dataplex v1.3.0/go.mod h1:hQuRtDg+fCiFgC8j0zV222HvzFQdRd+SVX8gdmFcZzA= cloud.google.com/go/dataplex v1.4.0/go.mod h1:X51GfLXEMVJ6UN47ESVqvlsRplbLhcsAt0kZCCKsU0A= cloud.google.com/go/dataplex v1.5.2/go.mod h1:cVMgQHsmfRoI5KFYq4JtIBEUbYwc3c7tXmIDhRmNNVQ= cloud.google.com/go/dataplex v1.6.0/go.mod h1:bMsomC/aEJOSpHXdFKFGQ1b0TDPIeL28nJObeO1ppRs= +cloud.google.com/go/dataplex v1.8.1/go.mod h1:7TyrDT6BCdI8/38Uvp0/ZxBslOslP2X2MPDucliyvSE= +cloud.google.com/go/dataplex v1.9.0/go.mod h1:7TyrDT6BCdI8/38Uvp0/ZxBslOslP2X2MPDucliyvSE= cloud.google.com/go/dataproc v1.7.0/go.mod h1:CKAlMjII9H90RXaMpSxQ8EU6dQx6iAYNPcYPOkSbi8s= cloud.google.com/go/dataproc v1.8.0/go.mod h1:5OW+zNAH0pMpw14JVrPONsxMQYMBqJuzORhIBfBn9uI= cloud.google.com/go/dataproc v1.12.0/go.mod h1:zrF3aX0uV3ikkMz6z4uBbIKyhRITnxvr4i3IjKsKrw4= +cloud.google.com/go/dataproc/v2 v2.0.1/go.mod h1:7Ez3KRHdFGcfY7GcevBbvozX+zyWGcwLJvvAMwCaoZ4= cloud.google.com/go/dataqna v0.5.0/go.mod h1:90Hyk596ft3zUQ8NkFfvICSIfHFh1Bc7C4cK3vbhkeo= cloud.google.com/go/dataqna v0.6.0/go.mod h1:1lqNpM7rqNLVgWBJyk5NF6Uen2PHym0jtVJonplVsDA= cloud.google.com/go/dataqna v0.7.0/go.mod h1:Lx9OcIIeqCrw1a6KdO3/5KMP1wAmTc0slZWwP12Qq3c= +cloud.google.com/go/dataqna v0.8.1/go.mod h1:zxZM0Bl6liMePWsHA8RMGAfmTG34vJMapbHAxQ5+WA8= cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= cloud.google.com/go/datastore v1.10.0/go.mod h1:PC5UzAmDEkAmkfaknstTYbNpgE49HAgW2J1gcgUfmdM= cloud.google.com/go/datastore v1.11.0/go.mod h1:TvGxBIHCS50u8jzG+AW/ppf87v1of8nwzFNgEZU1D3c= +cloud.google.com/go/datastore v1.12.0/go.mod h1:KjdB88W897MRITkvWWJrg2OUtrR5XVj1EoLgSp6/N70= +cloud.google.com/go/datastore v1.12.1/go.mod h1:KjdB88W897MRITkvWWJrg2OUtrR5XVj1EoLgSp6/N70= +cloud.google.com/go/datastore v1.13.0/go.mod h1:KjdB88W897MRITkvWWJrg2OUtrR5XVj1EoLgSp6/N70= cloud.google.com/go/datastream v1.2.0/go.mod h1:i/uTP8/fZwgATHS/XFu0TcNUhuA0twZxxQ3EyCUQMwo= cloud.google.com/go/datastream v1.3.0/go.mod h1:cqlOX8xlyYF/uxhiKn6Hbv6WjwPPuI9W2M9SAXwaLLQ= cloud.google.com/go/datastream v1.4.0/go.mod h1:h9dpzScPhDTs5noEMQVWP8Wx8AFBRyS0s8KWPx/9r0g= cloud.google.com/go/datastream v1.5.0/go.mod h1:6TZMMNPwjUqZHBKPQ1wwXpb0d5VDVPl2/XoS5yi88q4= cloud.google.com/go/datastream v1.6.0/go.mod h1:6LQSuswqLa7S4rPAOZFVjHIG3wJIjZcZrw8JDEDJuIs= cloud.google.com/go/datastream v1.7.0/go.mod h1:uxVRMm2elUSPuh65IbZpzJNMbuzkcvu5CjMqVIUHrww= +cloud.google.com/go/datastream v1.9.1/go.mod h1:hqnmr8kdUBmrnk65k5wNRoHSCYksvpdZIcZIEl8h43Q= +cloud.google.com/go/datastream v1.10.0/go.mod h1:hqnmr8kdUBmrnk65k5wNRoHSCYksvpdZIcZIEl8h43Q= cloud.google.com/go/deploy v1.4.0/go.mod h1:5Xghikd4VrmMLNaF6FiRFDlHb59VM59YoDQnOUdsH/c= cloud.google.com/go/deploy v1.5.0/go.mod h1:ffgdD0B89tToyW/U/D2eL0jN2+IEV/3EMuXHA0l4r+s= cloud.google.com/go/deploy v1.6.0/go.mod h1:f9PTHehG/DjCom3QH0cntOVRm93uGBDt2vKzAPwpXQI= cloud.google.com/go/deploy v1.8.0/go.mod h1:z3myEJnA/2wnB4sgjqdMfgxCA0EqC3RBTNcVPs93mtQ= +cloud.google.com/go/deploy v1.11.0/go.mod h1:tKuSUV5pXbn67KiubiUNUejqLs4f5cxxiCNCeyl0F2g= +cloud.google.com/go/deploy v1.13.0/go.mod h1:tKuSUV5pXbn67KiubiUNUejqLs4f5cxxiCNCeyl0F2g= cloud.google.com/go/dialogflow v1.15.0/go.mod h1:HbHDWs33WOGJgn6rfzBW1Kv807BE3O1+xGbn59zZWI4= cloud.google.com/go/dialogflow v1.16.1/go.mod h1:po6LlzGfK+smoSmTBnbkIZY2w8ffjz/RcGSS+sh1el0= cloud.google.com/go/dialogflow v1.17.0/go.mod h1:YNP09C/kXA1aZdBgC/VtXX74G/TKn7XVCcVumTflA+8= @@ -256,35 +310,48 @@ cloud.google.com/go/dialogflow v1.19.0/go.mod h1:JVmlG1TwykZDtxtTXujec4tQ+D8SBFM cloud.google.com/go/dialogflow v1.29.0/go.mod h1:b+2bzMe+k1s9V+F2jbJwpHPzrnIyHihAdRFMtn2WXuM= cloud.google.com/go/dialogflow v1.31.0/go.mod h1:cuoUccuL1Z+HADhyIA7dci3N5zUssgpBJmCzI6fNRB4= cloud.google.com/go/dialogflow v1.32.0/go.mod h1:jG9TRJl8CKrDhMEcvfcfFkkpp8ZhgPz3sBGmAUYJ2qE= +cloud.google.com/go/dialogflow v1.38.0/go.mod h1:L7jnH+JL2mtmdChzAIcXQHXMvQkE3U4hTaNltEuxXn4= +cloud.google.com/go/dialogflow v1.40.0/go.mod h1:L7jnH+JL2mtmdChzAIcXQHXMvQkE3U4hTaNltEuxXn4= cloud.google.com/go/dlp v1.6.0/go.mod h1:9eyB2xIhpU0sVwUixfBubDoRwP+GjeUoxxeueZmqvmM= cloud.google.com/go/dlp v1.7.0/go.mod h1:68ak9vCiMBjbasxeVD17hVPxDEck+ExiHavX8kiHG+Q= cloud.google.com/go/dlp v1.9.0/go.mod h1:qdgmqgTyReTz5/YNSSuueR8pl7hO0o9bQ39ZhtgkWp4= +cloud.google.com/go/dlp v1.10.1/go.mod h1:IM8BWz1iJd8njcNcG0+Kyd9OPnqnRNkDV8j42VT5KOI= cloud.google.com/go/documentai v1.7.0/go.mod h1:lJvftZB5NRiFSX4moiye1SMxHx0Bc3x1+p9e/RfXYiU= cloud.google.com/go/documentai v1.8.0/go.mod h1:xGHNEB7CtsnySCNrCFdCyyMz44RhFEEX2Q7UD0c5IhU= cloud.google.com/go/documentai v1.9.0/go.mod h1:FS5485S8R00U10GhgBC0aNGrJxBP8ZVpEeJ7PQDZd6k= cloud.google.com/go/documentai v1.10.0/go.mod h1:vod47hKQIPeCfN2QS/jULIvQTugbmdc0ZvxxfQY1bg4= cloud.google.com/go/documentai v1.16.0/go.mod h1:o0o0DLTEZ+YnJZ+J4wNfTxmDVyrkzFvttBXXtYRMHkM= cloud.google.com/go/documentai v1.18.0/go.mod h1:F6CK6iUH8J81FehpskRmhLq/3VlwQvb7TvwOceQ2tbs= +cloud.google.com/go/documentai v1.20.0/go.mod h1:yJkInoMcK0qNAEdRnqY/D5asy73tnPe88I1YTZT+a8E= +cloud.google.com/go/documentai v1.22.0/go.mod h1:yJkInoMcK0qNAEdRnqY/D5asy73tnPe88I1YTZT+a8E= cloud.google.com/go/domains v0.6.0/go.mod h1:T9Rz3GasrpYk6mEGHh4rymIhjlnIuB4ofT1wTxDeT4Y= cloud.google.com/go/domains v0.7.0/go.mod h1:PtZeqS1xjnXuRPKE/88Iru/LdfoRyEHYA9nFQf4UKpg= cloud.google.com/go/domains v0.8.0/go.mod h1:M9i3MMDzGFXsydri9/vW+EWz9sWb4I6WyHqdlAk0idE= +cloud.google.com/go/domains v0.9.1/go.mod h1:aOp1c0MbejQQ2Pjf1iJvnVyT+z6R6s8pX66KaCSDYfE= cloud.google.com/go/edgecontainer v0.1.0/go.mod h1:WgkZ9tp10bFxqO8BLPqv2LlfmQF1X8lZqwW4r1BTajk= cloud.google.com/go/edgecontainer v0.2.0/go.mod h1:RTmLijy+lGpQ7BXuTDa4C4ssxyXT34NIuHIgKuP4s5w= cloud.google.com/go/edgecontainer v0.3.0/go.mod h1:FLDpP4nykgwwIfcLt6zInhprzw0lEi2P1fjO6Ie0qbc= cloud.google.com/go/edgecontainer v1.0.0/go.mod h1:cttArqZpBB2q58W/upSG++ooo6EsblxDIolxa3jSjbY= +cloud.google.com/go/edgecontainer v1.1.1/go.mod h1:O5bYcS//7MELQZs3+7mabRqoWQhXCzenBu0R8bz2rwk= cloud.google.com/go/errorreporting v0.3.0/go.mod h1:xsP2yaAp+OAW4OIm60An2bbLpqIhKXdWR/tawvl7QzU= cloud.google.com/go/essentialcontacts v1.3.0/go.mod h1:r+OnHa5jfj90qIfZDO/VztSFqbQan7HV75p8sA+mdGI= cloud.google.com/go/essentialcontacts v1.4.0/go.mod h1:8tRldvHYsmnBCHdFpvU+GL75oWiBKl80BiqlFh9tp+8= cloud.google.com/go/essentialcontacts v1.5.0/go.mod h1:ay29Z4zODTuwliK7SnX8E86aUF2CTzdNtvv42niCX0M= +cloud.google.com/go/essentialcontacts v1.6.2/go.mod h1:T2tB6tX+TRak7i88Fb2N9Ok3PvY3UNbUsMag9/BARh4= cloud.google.com/go/eventarc v1.7.0/go.mod h1:6ctpF3zTnaQCxUjHUdcfgcA1A2T309+omHZth7gDfmc= cloud.google.com/go/eventarc v1.8.0/go.mod h1:imbzxkyAU4ubfsaKYdQg04WS1NvncblHEup4kvF+4gw= cloud.google.com/go/eventarc v1.10.0/go.mod h1:u3R35tmZ9HvswGRBnF48IlYgYeBcPUCjkr4BTdem2Kw= cloud.google.com/go/eventarc v1.11.0/go.mod h1:PyUjsUKPWoRBCHeOxZd/lbOOjahV41icXyUY5kSTvVY= +cloud.google.com/go/eventarc v1.12.1/go.mod h1:mAFCW6lukH5+IZjkvrEss+jmt2kOdYlN8aMx3sRJiAI= +cloud.google.com/go/eventarc v1.13.0/go.mod h1:mAFCW6lukH5+IZjkvrEss+jmt2kOdYlN8aMx3sRJiAI= cloud.google.com/go/filestore v1.3.0/go.mod h1:+qbvHGvXU1HaKX2nD0WEPo92TP/8AQuCVEBXNY9z0+w= cloud.google.com/go/filestore v1.4.0/go.mod h1:PaG5oDfo9r224f8OYXURtAsY+Fbyq/bLYoINEK8XQAI= cloud.google.com/go/filestore v1.5.0/go.mod h1:FqBXDWBp4YLHqRnVGveOkHDf8svj9r5+mUDLupOWEDs= cloud.google.com/go/filestore v1.6.0/go.mod h1:di5unNuss/qfZTw2U9nhFqo8/ZDSc466dre85Kydllg= +cloud.google.com/go/filestore v1.7.1/go.mod h1:y10jsorq40JJnjR/lQ8AfFbbcGlw3g+Dp8oN7i7FjV4= cloud.google.com/go/firestore v1.9.0/go.mod h1:HMkjKHNTtRyZNiMzu7YAsLr9K3X2udY2AMwDaMEQiiE= +cloud.google.com/go/firestore v1.11.0/go.mod h1:b38dKhgzlmNNGTNZZwe7ZRFEuRab1Hay3/DBsIGKKy4= +cloud.google.com/go/firestore v1.12.0/go.mod h1:b38dKhgzlmNNGTNZZwe7ZRFEuRab1Hay3/DBsIGKKy4= cloud.google.com/go/functions v1.6.0/go.mod h1:3H1UA3qiIPRWD7PeZKLvHZ9SaQhR26XIJcC0A5GbvAk= cloud.google.com/go/functions v1.7.0/go.mod h1:+d+QBcWM+RsrgZfV9xo6KfA1GlzJfxcfZcRPEhDDfzg= cloud.google.com/go/functions v1.8.0/go.mod h1:RTZ4/HsQjIqIYP9a9YPbU+QFoQsAlYgrwOXJWHn1POY= @@ -292,28 +359,37 @@ cloud.google.com/go/functions v1.9.0/go.mod h1:Y+Dz8yGguzO3PpIjhLTbnqV1CWmgQ5Uwt cloud.google.com/go/functions v1.10.0/go.mod h1:0D3hEOe3DbEvCXtYOZHQZmD+SzYsi1YbI7dGvHfldXw= cloud.google.com/go/functions v1.12.0/go.mod h1:AXWGrF3e2C/5ehvwYo/GH6O5s09tOPksiKhz+hH8WkA= cloud.google.com/go/functions v1.13.0/go.mod h1:EU4O007sQm6Ef/PwRsI8N2umygGqPBS/IZQKBQBcJ3c= +cloud.google.com/go/functions v1.15.1/go.mod h1:P5yNWUTkyU+LvW/S9O6V+V423VZooALQlqoXdoPz5AE= cloud.google.com/go/gaming v1.5.0/go.mod h1:ol7rGcxP/qHTRQE/RO4bxkXq+Fix0j6D4LFPzYTIrDM= cloud.google.com/go/gaming v1.6.0/go.mod h1:YMU1GEvA39Qt3zWGyAVA9bpYz/yAhTvaQ1t2sK4KPUA= cloud.google.com/go/gaming v1.7.0/go.mod h1:LrB8U7MHdGgFG851iHAfqUdLcKBdQ55hzXy9xBJz0+w= cloud.google.com/go/gaming v1.8.0/go.mod h1:xAqjS8b7jAVW0KFYeRUxngo9My3f33kFmua++Pi+ggM= cloud.google.com/go/gaming v1.9.0/go.mod h1:Fc7kEmCObylSWLO334NcO+O9QMDyz+TKC4v1D7X+Bc0= +cloud.google.com/go/gaming v1.10.1/go.mod h1:XQQvtfP8Rb9Rxnxm5wFVpAp9zCQkJi2bLIb7iHGwB3s= cloud.google.com/go/gkebackup v0.2.0/go.mod h1:XKvv/4LfG829/B8B7xRkk8zRrOEbKtEam6yNfuQNH60= cloud.google.com/go/gkebackup v0.3.0/go.mod h1:n/E671i1aOQvUxT541aTkCwExO/bTer2HDlj4TsBRAo= cloud.google.com/go/gkebackup v0.4.0/go.mod h1:byAyBGUwYGEEww7xsbnUTBHIYcOPy/PgUWUtOeRm9Vg= +cloud.google.com/go/gkebackup v1.3.0/go.mod h1:vUDOu++N0U5qs4IhG1pcOnD1Mac79xWy6GoBFlWCWBU= cloud.google.com/go/gkeconnect v0.5.0/go.mod h1:c5lsNAg5EwAy7fkqX/+goqFsU1Da/jQFqArp+wGNr/o= cloud.google.com/go/gkeconnect v0.6.0/go.mod h1:Mln67KyU/sHJEBY8kFZ0xTeyPtzbq9StAVvEULYK16A= cloud.google.com/go/gkeconnect v0.7.0/go.mod h1:SNfmVqPkaEi3bF/B3CNZOAYPYdg7sU+obZ+QTky2Myw= +cloud.google.com/go/gkeconnect v0.8.1/go.mod h1:KWiK1g9sDLZqhxB2xEuPV8V9NYzrqTUmQR9shJHpOZw= cloud.google.com/go/gkehub v0.9.0/go.mod h1:WYHN6WG8w9bXU0hqNxt8rm5uxnk8IH+lPY9J2TV7BK0= cloud.google.com/go/gkehub v0.10.0/go.mod h1:UIPwxI0DsrpsVoWpLB0stwKCP+WFVG9+y977wO+hBH0= cloud.google.com/go/gkehub v0.11.0/go.mod h1:JOWHlmN+GHyIbuWQPl47/C2RFhnFKH38jH9Ascu3n0E= cloud.google.com/go/gkehub v0.12.0/go.mod h1:djiIwwzTTBrF5NaXCGv3mf7klpEMcST17VBTVVDcuaw= +cloud.google.com/go/gkehub v0.14.1/go.mod h1:VEXKIJZ2avzrbd7u+zeMtW00Y8ddk/4V9511C9CQGTY= cloud.google.com/go/gkemulticloud v0.3.0/go.mod h1:7orzy7O0S+5kq95e4Hpn7RysVA7dPs8W/GgfUtsPbrA= cloud.google.com/go/gkemulticloud v0.4.0/go.mod h1:E9gxVBnseLWCk24ch+P9+B2CoDFJZTyIgLKSalC7tuI= cloud.google.com/go/gkemulticloud v0.5.0/go.mod h1:W0JDkiyi3Tqh0TJr//y19wyb1yf8llHVto2Htf2Ja3Y= +cloud.google.com/go/gkemulticloud v0.6.1/go.mod h1:kbZ3HKyTsiwqKX7Yw56+wUGwwNZViRnxWK2DVknXWfw= +cloud.google.com/go/gkemulticloud v1.0.0/go.mod h1:kbZ3HKyTsiwqKX7Yw56+wUGwwNZViRnxWK2DVknXWfw= cloud.google.com/go/grafeas v0.2.0/go.mod h1:KhxgtF2hb0P191HlY5besjYm6MqTSTj3LSI+M+ByZHc= +cloud.google.com/go/grafeas v0.3.0/go.mod h1:P7hgN24EyONOTMyeJH6DxG4zD7fwiYa5Q6GUgyFSOU8= cloud.google.com/go/gsuiteaddons v1.3.0/go.mod h1:EUNK/J1lZEZO8yPtykKxLXI6JSVN2rg9bN8SXOa0bgM= cloud.google.com/go/gsuiteaddons v1.4.0/go.mod h1:rZK5I8hht7u7HxFQcFei0+AtfS9uSushomRlg+3ua1o= cloud.google.com/go/gsuiteaddons v1.5.0/go.mod h1:TFCClYLd64Eaa12sFVmUyG62tk4mdIsI7pAnSXRkcFo= +cloud.google.com/go/gsuiteaddons v1.6.1/go.mod h1:CodrdOqRZcLp5WOwejHWYBjZvfY0kOphkAKpF/3qdZY= cloud.google.com/go/iam v0.1.0/go.mod h1:vcUNEa0pEm0qRVpmWepWaFMIAI8/hjB9mO8rNCJtF6c= cloud.google.com/go/iam v0.3.0/go.mod h1:XzJPvDayI+9zsASAFO68Hk07u3z+f+JrT2xXNdp4bnY= cloud.google.com/go/iam v0.5.0/go.mod h1:wPU9Vt0P4UmCux7mqtRu6jcpPAb74cP1fh50J3QpkUc= @@ -323,20 +399,27 @@ cloud.google.com/go/iam v0.8.0/go.mod h1:lga0/y3iH6CX7sYqypWJ33hf7kkfXJag67naqGE cloud.google.com/go/iam v0.11.0/go.mod h1:9PiLDanza5D+oWFZiH1uG+RnRCfEGKoyl6yo4cgWZGY= cloud.google.com/go/iam v0.12.0/go.mod h1:knyHGviacl11zrtZUoDuYpDgLjvr28sLQaG0YB2GYAY= cloud.google.com/go/iam v0.13.0/go.mod h1:ljOg+rcNfzZ5d6f1nAUJ8ZIxOaZUVoS14bKCtaLZ/D0= +cloud.google.com/go/iam v1.0.1/go.mod h1:yR3tmSL8BcZB4bxByRv2jkSIahVmCtfKZwLYGBalRE8= +cloud.google.com/go/iam v1.1.0/go.mod h1:nxdHjaKfCr7fNYx/HJMM8LgiMugmveWlkatear5gVyk= cloud.google.com/go/iam v1.1.1 h1:lW7fzj15aVIXYHREOqjRBV9PsH0Z6u8Y46a1YGvQP4Y= cloud.google.com/go/iam v1.1.1/go.mod h1:A5avdyVL2tCppe4unb0951eI9jreack+RJ0/d+KUZOU= +cloud.google.com/go/iam v1.1.2 h1:gacbrBdWcoVmGLozRuStX45YKvJtzIjJdAolzUs1sm4= +cloud.google.com/go/iam v1.1.2/go.mod h1:A5avdyVL2tCppe4unb0951eI9jreack+RJ0/d+KUZOU= cloud.google.com/go/iap v1.4.0/go.mod h1:RGFwRJdihTINIe4wZ2iCP0zF/qu18ZwyKxrhMhygBEc= cloud.google.com/go/iap v1.5.0/go.mod h1:UH/CGgKd4KyohZL5Pt0jSKE4m3FR51qg6FKQ/z/Ix9A= cloud.google.com/go/iap v1.6.0/go.mod h1:NSuvI9C/j7UdjGjIde7t7HBz+QTwBcapPE07+sSRcLk= cloud.google.com/go/iap v1.7.0/go.mod h1:beqQx56T9O1G1yNPph+spKpNibDlYIiIixiqsQXxLIo= cloud.google.com/go/iap v1.7.1/go.mod h1:WapEwPc7ZxGt2jFGB/C/bm+hP0Y6NXzOYGjpPnmMS74= +cloud.google.com/go/iap v1.8.1/go.mod h1:sJCbeqg3mvWLqjZNsI6dfAtbbV1DL2Rl7e1mTyXYREQ= cloud.google.com/go/ids v1.1.0/go.mod h1:WIuwCaYVOzHIj2OhN9HAwvW+DBdmUAdcWlFxRl+KubM= cloud.google.com/go/ids v1.2.0/go.mod h1:5WXvp4n25S0rA/mQWAg1YEEBBq6/s+7ml1RDCW1IrcY= cloud.google.com/go/ids v1.3.0/go.mod h1:JBdTYwANikFKaDP6LtW5JAi4gubs57SVNQjemdt6xV4= +cloud.google.com/go/ids v1.4.1/go.mod h1:np41ed8YMU8zOgv53MMMoCntLTn2lF+SUzlM+O3u/jw= cloud.google.com/go/iot v1.3.0/go.mod h1:r7RGh2B61+B8oz0AGE+J72AhA0G7tdXItODWsaA2oLs= cloud.google.com/go/iot v1.4.0/go.mod h1:dIDxPOn0UvNDUMD8Ger7FIaTuvMkj+aGk94RPP0iV+g= cloud.google.com/go/iot v1.5.0/go.mod h1:mpz5259PDl3XJthEmh9+ap0affn/MqNSP4My77Qql9o= cloud.google.com/go/iot v1.6.0/go.mod h1:IqdAsmE2cTYYNO1Fvjfzo9po179rAtJeVGUvkLN3rLE= +cloud.google.com/go/iot v1.7.1/go.mod h1:46Mgw7ev1k9KqK1ao0ayW9h0lI+3hxeanz+L1zmbbbk= cloud.google.com/go/kms v1.4.0/go.mod h1:fajBHndQ+6ubNw6Ss2sSd+SWvjL26RNo/dr7uxsnnOA= cloud.google.com/go/kms v1.5.0/go.mod h1:QJS2YY0eJGBg3mnDfuaCyLauWwBJiHRboYxJ++1xJNg= cloud.google.com/go/kms v1.6.0/go.mod h1:Jjy850yySiasBUDi6KFUwUv2n1+o7QZFyuUJg6OgjA0= @@ -344,92 +427,123 @@ cloud.google.com/go/kms v1.8.0/go.mod h1:4xFEhYFqvW+4VMELtZyxomGSYtSQKzM178ylFW4 cloud.google.com/go/kms v1.9.0/go.mod h1:qb1tPTgfF9RQP8e1wq4cLFErVuTJv7UsSC915J8dh3w= cloud.google.com/go/kms v1.10.0/go.mod h1:ng3KTUtQQU9bPX3+QGLsflZIHlkbn8amFAMY63m8d24= cloud.google.com/go/kms v1.10.1/go.mod h1:rIWk/TryCkR59GMC3YtHtXeLzd634lBbKenvyySAyYI= +cloud.google.com/go/kms v1.11.0/go.mod h1:hwdiYC0xjnWsKQQCQQmIQnS9asjYVSK6jtXm+zFqXLM= +cloud.google.com/go/kms v1.12.1/go.mod h1:c9J991h5DTl+kg7gi3MYomh12YEENGrf48ee/N/2CDM= +cloud.google.com/go/kms v1.15.0/go.mod h1:c9J991h5DTl+kg7gi3MYomh12YEENGrf48ee/N/2CDM= cloud.google.com/go/language v1.4.0/go.mod h1:F9dRpNFQmJbkaop6g0JhSBXCNlO90e1KWx5iDdxbWic= cloud.google.com/go/language v1.6.0/go.mod h1:6dJ8t3B+lUYfStgls25GusK04NLh3eDLQnWM3mdEbhI= cloud.google.com/go/language v1.7.0/go.mod h1:DJ6dYN/W+SQOjF8e1hLQXMF21AkH2w9wiPzPCJa2MIE= cloud.google.com/go/language v1.8.0/go.mod h1:qYPVHf7SPoNNiCL2Dr0FfEFNil1qi3pQEyygwpgVKB8= cloud.google.com/go/language v1.9.0/go.mod h1:Ns15WooPM5Ad/5no/0n81yUetis74g3zrbeJBE+ptUY= +cloud.google.com/go/language v1.10.1/go.mod h1:CPp94nsdVNiQEt1CNjF5WkTcisLiHPyIbMhvR8H2AW0= cloud.google.com/go/lifesciences v0.5.0/go.mod h1:3oIKy8ycWGPUyZDR/8RNnTOYevhaMLqh5vLUXs9zvT8= cloud.google.com/go/lifesciences v0.6.0/go.mod h1:ddj6tSX/7BOnhxCSd3ZcETvtNr8NZ6t/iPhY2Tyfu08= cloud.google.com/go/lifesciences v0.8.0/go.mod h1:lFxiEOMqII6XggGbOnKiyZ7IBwoIqA84ClvoezaA/bo= +cloud.google.com/go/lifesciences v0.9.1/go.mod h1:hACAOd1fFbCGLr/+weUKRAJas82Y4vrL3O5326N//Wc= cloud.google.com/go/logging v1.6.1/go.mod h1:5ZO0mHHbvm8gEmeEUHrmDlTDSu5imF6MUP9OfilNXBw= cloud.google.com/go/logging v1.7.0/go.mod h1:3xjP2CjkM3ZkO73aj4ASA5wRPGGCRrPIAeNqVNkzY8M= cloud.google.com/go/longrunning v0.1.1/go.mod h1:UUFxuDWkv22EuY93jjmDMFT5GPQKeFVJBIF6QlTqdsE= cloud.google.com/go/longrunning v0.3.0/go.mod h1:qth9Y41RRSUE69rDcOn6DdK3HfQfsUI0YSmW3iIlLJc= cloud.google.com/go/longrunning v0.4.1/go.mod h1:4iWDqhBZ70CvZ6BfETbvam3T8FMvLK+eFj0E6AaRQTo= +cloud.google.com/go/longrunning v0.4.2/go.mod h1:OHrnaYyLUV6oqwh0xiS7e5sLQhP1m0QU9R+WhGDMgIQ= +cloud.google.com/go/longrunning v0.5.0/go.mod h1:0JNuqRShmscVAhIACGtskSAWtqtOoPkwP0YF1oVEchc= cloud.google.com/go/longrunning v0.5.1 h1:Fr7TXftcqTudoyRJa113hyaqlGdiBQkp0Gq7tErFDWI= +cloud.google.com/go/longrunning v0.5.1/go.mod h1:spvimkwdz6SPWKEt/XBij79E9fiTkHSQl/fRUUQJYJc= cloud.google.com/go/managedidentities v1.3.0/go.mod h1:UzlW3cBOiPrzucO5qWkNkh0w33KFtBJU281hacNvsdE= cloud.google.com/go/managedidentities v1.4.0/go.mod h1:NWSBYbEMgqmbZsLIyKvxrYbtqOsxY1ZrGM+9RgDqInM= cloud.google.com/go/managedidentities v1.5.0/go.mod h1:+dWcZ0JlUmpuxpIDfyP5pP5y0bLdRwOS4Lp7gMni/LA= +cloud.google.com/go/managedidentities v1.6.1/go.mod h1:h/irGhTN2SkZ64F43tfGPMbHnypMbu4RB3yl8YcuEak= cloud.google.com/go/maps v0.1.0/go.mod h1:BQM97WGyfw9FWEmQMpZ5T6cpovXXSd1cGmFma94eubI= cloud.google.com/go/maps v0.6.0/go.mod h1:o6DAMMfb+aINHz/p/jbcY+mYeXBoZoxTfdSQ8VAJaCw= cloud.google.com/go/maps v0.7.0/go.mod h1:3GnvVl3cqeSvgMcpRlQidXsPYuDGQ8naBis7MVzpXsY= +cloud.google.com/go/maps v1.3.0/go.mod h1:6mWTUv+WhnOwAgjVsSW2QPPECmW+s3PcRyOa9vgG/5s= +cloud.google.com/go/maps v1.4.0/go.mod h1:6mWTUv+WhnOwAgjVsSW2QPPECmW+s3PcRyOa9vgG/5s= cloud.google.com/go/mediatranslation v0.5.0/go.mod h1:jGPUhGTybqsPQn91pNXw0xVHfuJ3leR1wj37oU3y1f4= cloud.google.com/go/mediatranslation v0.6.0/go.mod h1:hHdBCTYNigsBxshbznuIMFNe5QXEowAuNmmC7h8pu5w= cloud.google.com/go/mediatranslation v0.7.0/go.mod h1:LCnB/gZr90ONOIQLgSXagp8XUW1ODs2UmUMvcgMfI2I= +cloud.google.com/go/mediatranslation v0.8.1/go.mod h1:L/7hBdEYbYHQJhX2sldtTO5SZZ1C1vkapubj0T2aGig= cloud.google.com/go/memcache v1.4.0/go.mod h1:rTOfiGZtJX1AaFUrOgsMHX5kAzaTQ8azHiuDoTPzNsE= cloud.google.com/go/memcache v1.5.0/go.mod h1:dk3fCK7dVo0cUU2c36jKb4VqKPS22BTkf81Xq617aWM= cloud.google.com/go/memcache v1.6.0/go.mod h1:XS5xB0eQZdHtTuTF9Hf8eJkKtR3pVRCcvJwtm68T3rA= cloud.google.com/go/memcache v1.7.0/go.mod h1:ywMKfjWhNtkQTxrWxCkCFkoPjLHPW6A7WOTVI8xy3LY= cloud.google.com/go/memcache v1.9.0/go.mod h1:8oEyzXCu+zo9RzlEaEjHl4KkgjlNDaXbCQeQWlzNFJM= +cloud.google.com/go/memcache v1.10.1/go.mod h1:47YRQIarv4I3QS5+hoETgKO40InqzLP6kpNLvyXuyaA= cloud.google.com/go/metastore v1.5.0/go.mod h1:2ZNrDcQwghfdtCwJ33nM0+GrBGlVuh8rakL3vdPY3XY= cloud.google.com/go/metastore v1.6.0/go.mod h1:6cyQTls8CWXzk45G55x57DVQ9gWg7RiH65+YgPsNh9s= cloud.google.com/go/metastore v1.7.0/go.mod h1:s45D0B4IlsINu87/AsWiEVYbLaIMeUSoxlKKDqBGFS8= cloud.google.com/go/metastore v1.8.0/go.mod h1:zHiMc4ZUpBiM7twCIFQmJ9JMEkDSyZS9U12uf7wHqSI= cloud.google.com/go/metastore v1.10.0/go.mod h1:fPEnH3g4JJAk+gMRnrAnoqyv2lpUCqJPWOodSaf45Eo= +cloud.google.com/go/metastore v1.11.1/go.mod h1:uZuSo80U3Wd4zi6C22ZZliOUJ3XeM/MlYi/z5OAOWRA= +cloud.google.com/go/metastore v1.12.0/go.mod h1:uZuSo80U3Wd4zi6C22ZZliOUJ3XeM/MlYi/z5OAOWRA= cloud.google.com/go/monitoring v1.7.0/go.mod h1:HpYse6kkGo//7p6sT0wsIC6IBDET0RhIsnmlA53dvEk= cloud.google.com/go/monitoring v1.8.0/go.mod h1:E7PtoMJ1kQXWxPjB6mv2fhC5/15jInuulFdYYtlcvT4= cloud.google.com/go/monitoring v1.12.0/go.mod h1:yx8Jj2fZNEkL/GYZyTLS4ZtZEZN8WtDEiEqG4kLK50w= cloud.google.com/go/monitoring v1.13.0/go.mod h1:k2yMBAB1H9JT/QETjNkgdCGD9bPF712XiLTVr+cBrpw= +cloud.google.com/go/monitoring v1.15.1/go.mod h1:lADlSAlFdbqQuwwpaImhsJXu1QSdd3ojypXrFSMr2rM= cloud.google.com/go/networkconnectivity v1.4.0/go.mod h1:nOl7YL8odKyAOtzNX73/M5/mGZgqqMeryi6UPZTk/rA= cloud.google.com/go/networkconnectivity v1.5.0/go.mod h1:3GzqJx7uhtlM3kln0+x5wyFvuVH1pIBJjhCpjzSt75o= cloud.google.com/go/networkconnectivity v1.6.0/go.mod h1:OJOoEXW+0LAxHh89nXd64uGG+FbQoeH8DtxCHVOMlaM= cloud.google.com/go/networkconnectivity v1.7.0/go.mod h1:RMuSbkdbPwNMQjB5HBWD5MpTBnNm39iAVpC3TmsExt8= cloud.google.com/go/networkconnectivity v1.10.0/go.mod h1:UP4O4sWXJG13AqrTdQCD9TnLGEbtNRqjuaaA7bNjF5E= cloud.google.com/go/networkconnectivity v1.11.0/go.mod h1:iWmDD4QF16VCDLXUqvyspJjIEtBR/4zq5hwnY2X3scM= +cloud.google.com/go/networkconnectivity v1.12.1/go.mod h1:PelxSWYM7Sh9/guf8CFhi6vIqf19Ir/sbfZRUwXh92E= cloud.google.com/go/networkmanagement v1.4.0/go.mod h1:Q9mdLLRn60AsOrPc8rs8iNV6OHXaGcDdsIQe1ohekq8= cloud.google.com/go/networkmanagement v1.5.0/go.mod h1:ZnOeZ/evzUdUsnvRt792H0uYEnHQEMaz+REhhzJRcf4= cloud.google.com/go/networkmanagement v1.6.0/go.mod h1:5pKPqyXjB/sgtvB5xqOemumoQNB7y95Q7S+4rjSOPYY= +cloud.google.com/go/networkmanagement v1.8.0/go.mod h1:Ho/BUGmtyEqrttTgWEe7m+8vDdK74ibQc+Be0q7Fof0= cloud.google.com/go/networksecurity v0.5.0/go.mod h1:xS6fOCoqpVC5zx15Z/MqkfDwH4+m/61A3ODiDV1xmiQ= cloud.google.com/go/networksecurity v0.6.0/go.mod h1:Q5fjhTr9WMI5mbpRYEbiexTzROf7ZbDzvzCrNl14nyU= cloud.google.com/go/networksecurity v0.7.0/go.mod h1:mAnzoxx/8TBSyXEeESMy9OOYwo1v+gZ5eMRnsT5bC8k= cloud.google.com/go/networksecurity v0.8.0/go.mod h1:B78DkqsxFG5zRSVuwYFRZ9Xz8IcQ5iECsNrPn74hKHU= +cloud.google.com/go/networksecurity v0.9.1/go.mod h1:MCMdxOKQ30wsBI1eI659f9kEp4wuuAueoC9AJKSPWZQ= cloud.google.com/go/notebooks v1.2.0/go.mod h1:9+wtppMfVPUeJ8fIWPOq1UnATHISkGXGqTkxeieQ6UY= cloud.google.com/go/notebooks v1.3.0/go.mod h1:bFR5lj07DtCPC7YAAJ//vHskFBxA5JzYlH68kXVdk34= cloud.google.com/go/notebooks v1.4.0/go.mod h1:4QPMngcwmgb6uw7Po99B2xv5ufVoIQ7nOGDyL4P8AgA= cloud.google.com/go/notebooks v1.5.0/go.mod h1:q8mwhnP9aR8Hpfnrc5iN5IBhrXUy8S2vuYs+kBJ/gu0= cloud.google.com/go/notebooks v1.7.0/go.mod h1:PVlaDGfJgj1fl1S3dUwhFMXFgfYGhYQt2164xOMONmE= cloud.google.com/go/notebooks v1.8.0/go.mod h1:Lq6dYKOYOWUCTvw5t2q1gp1lAp0zxAxRycayS0iJcqQ= +cloud.google.com/go/notebooks v1.9.1/go.mod h1:zqG9/gk05JrzgBt4ghLzEepPHNwE5jgPcHZRKhlC1A8= cloud.google.com/go/optimization v1.1.0/go.mod h1:5po+wfvX5AQlPznyVEZjGJTMr4+CAkJf2XSTQOOl9l4= cloud.google.com/go/optimization v1.2.0/go.mod h1:Lr7SOHdRDENsh+WXVmQhQTrzdu9ybg0NecjHidBq6xs= cloud.google.com/go/optimization v1.3.1/go.mod h1:IvUSefKiwd1a5p0RgHDbWCIbDFgKuEdB+fPPuP0IDLI= +cloud.google.com/go/optimization v1.4.1/go.mod h1:j64vZQP7h9bO49m2rVaTVoNM0vEBEN5eKPUPbZyXOrk= cloud.google.com/go/orchestration v1.3.0/go.mod h1:Sj5tq/JpWiB//X/q3Ngwdl5K7B7Y0KZ7bfv0wL6fqVA= cloud.google.com/go/orchestration v1.4.0/go.mod h1:6W5NLFWs2TlniBphAViZEVhrXRSMgUGDfW7vrWKvsBk= cloud.google.com/go/orchestration v1.6.0/go.mod h1:M62Bevp7pkxStDfFfTuCOaXgaaqRAga1yKyoMtEoWPQ= +cloud.google.com/go/orchestration v1.8.1/go.mod h1:4sluRF3wgbYVRqz7zJ1/EUNc90TTprliq9477fGobD8= cloud.google.com/go/orgpolicy v1.4.0/go.mod h1:xrSLIV4RePWmP9P3tBl8S93lTmlAxjm06NSm2UTmKvE= cloud.google.com/go/orgpolicy v1.5.0/go.mod h1:hZEc5q3wzwXJaKrsx5+Ewg0u1LxJ51nNFlext7Tanwc= cloud.google.com/go/orgpolicy v1.10.0/go.mod h1:w1fo8b7rRqlXlIJbVhOMPrwVljyuW5mqssvBtU18ONc= +cloud.google.com/go/orgpolicy v1.11.0/go.mod h1:2RK748+FtVvnfuynxBzdnyu7sygtoZa1za/0ZfpOs1M= +cloud.google.com/go/orgpolicy v1.11.1/go.mod h1:8+E3jQcpZJQliP+zaFfayC2Pg5bmhuLK755wKhIIUCE= cloud.google.com/go/osconfig v1.7.0/go.mod h1:oVHeCeZELfJP7XLxcBGTMBvRO+1nQ5tFG9VQTmYS2Fs= cloud.google.com/go/osconfig v1.8.0/go.mod h1:EQqZLu5w5XA7eKizepumcvWx+m8mJUhEwiPqWiZeEdg= cloud.google.com/go/osconfig v1.9.0/go.mod h1:Yx+IeIZJ3bdWmzbQU4fxNl8xsZ4amB+dygAwFPlvnNo= cloud.google.com/go/osconfig v1.10.0/go.mod h1:uMhCzqC5I8zfD9zDEAfvgVhDS8oIjySWh+l4WK6GnWw= cloud.google.com/go/osconfig v1.11.0/go.mod h1:aDICxrur2ogRd9zY5ytBLV89KEgT2MKB2L/n6x1ooPw= +cloud.google.com/go/osconfig v1.12.0/go.mod h1:8f/PaYzoS3JMVfdfTubkowZYGmAhUCjjwnjqWI7NVBc= +cloud.google.com/go/osconfig v1.12.1/go.mod h1:4CjBxND0gswz2gfYRCUoUzCm9zCABp91EeTtWXyz0tE= cloud.google.com/go/oslogin v1.4.0/go.mod h1:YdgMXWRaElXz/lDk1Na6Fh5orF7gvmJ0FGLIs9LId4E= cloud.google.com/go/oslogin v1.5.0/go.mod h1:D260Qj11W2qx/HVF29zBg+0fd6YCSjSqLUkY/qEenQU= cloud.google.com/go/oslogin v1.6.0/go.mod h1:zOJ1O3+dTU8WPlGEkFSh7qeHPPSoxrcMbbK1Nm2iX70= cloud.google.com/go/oslogin v1.7.0/go.mod h1:e04SN0xO1UNJ1M5GP0vzVBFicIe4O53FOfcixIqTyXo= cloud.google.com/go/oslogin v1.9.0/go.mod h1:HNavntnH8nzrn8JCTT5fj18FuJLFJc4NaZJtBnQtKFs= +cloud.google.com/go/oslogin v1.10.1/go.mod h1:x692z7yAue5nE7CsSnoG0aaMbNoRJRXO4sn73R+ZqAs= cloud.google.com/go/phishingprotection v0.5.0/go.mod h1:Y3HZknsK9bc9dMi+oE8Bim0lczMU6hrX0UpADuMefr0= cloud.google.com/go/phishingprotection v0.6.0/go.mod h1:9Y3LBLgy0kDTcYET8ZH3bq/7qni15yVUoAxiFxnlSUA= cloud.google.com/go/phishingprotection v0.7.0/go.mod h1:8qJI4QKHoda/sb/7/YmMQ2omRLSLYSu9bU0EKCNI+Lk= +cloud.google.com/go/phishingprotection v0.8.1/go.mod h1:AxonW7GovcA8qdEk13NfHq9hNx5KPtfxXNeUxTDxB6I= cloud.google.com/go/policytroubleshooter v1.3.0/go.mod h1:qy0+VwANja+kKrjlQuOzmlvscn4RNsAc0e15GGqfMxg= cloud.google.com/go/policytroubleshooter v1.4.0/go.mod h1:DZT4BcRw3QoO8ota9xw/LKtPa8lKeCByYeKTIf/vxdE= cloud.google.com/go/policytroubleshooter v1.5.0/go.mod h1:Rz1WfV+1oIpPdN2VvvuboLVRsB1Hclg3CKQ53j9l8vw= cloud.google.com/go/policytroubleshooter v1.6.0/go.mod h1:zYqaPTsmfvpjm5ULxAyD/lINQxJ0DDsnWOP/GZ7xzBc= +cloud.google.com/go/policytroubleshooter v1.7.1/go.mod h1:0NaT5v3Ag1M7U5r0GfDCpUFkWd9YqpubBWsQlhanRv0= +cloud.google.com/go/policytroubleshooter v1.8.0/go.mod h1:tmn5Ir5EToWe384EuboTcVQT7nTag2+DuH3uHmKd1HU= cloud.google.com/go/privatecatalog v0.5.0/go.mod h1:XgosMUvvPyxDjAVNDYxJ7wBW8//hLDDYmnsNcMGq1K0= cloud.google.com/go/privatecatalog v0.6.0/go.mod h1:i/fbkZR0hLN29eEWiiwue8Pb+GforiEIBnV9yrRUOKI= cloud.google.com/go/privatecatalog v0.7.0/go.mod h1:2s5ssIFO69F5csTXcwBP7NPFTZvps26xGzvQ2PQaBYg= cloud.google.com/go/privatecatalog v0.8.0/go.mod h1:nQ6pfaegeDAq/Q5lrfCQzQLhubPiZhSaNhIgfJlnIXs= +cloud.google.com/go/privatecatalog v0.9.1/go.mod h1:0XlDXW2unJXdf9zFz968Hp35gl/bhF4twwpXZAW50JA= cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+R3AArQw= cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= @@ -438,9 +552,12 @@ cloud.google.com/go/pubsub v1.26.0/go.mod h1:QgBH3U/jdJy/ftjPhTkyXNj543Tin1pRYcd cloud.google.com/go/pubsub v1.27.1/go.mod h1:hQN39ymbV9geqBnfQq6Xf63yNhUAhv9CZhzp5O6qsW0= cloud.google.com/go/pubsub v1.28.0/go.mod h1:vuXFpwaVoIPQMGXqRyUQigu/AX1S3IWugR9xznmcXX8= cloud.google.com/go/pubsub v1.30.0/go.mod h1:qWi1OPS0B+b5L+Sg6Gmc9zD1Y+HaM0MdUr7LsupY1P4= +cloud.google.com/go/pubsub v1.32.0/go.mod h1:f+w71I33OMyxf9VpMVcZbnG5KSUkCOUHYpFd5U1GdRc= +cloud.google.com/go/pubsub v1.33.0/go.mod h1:f+w71I33OMyxf9VpMVcZbnG5KSUkCOUHYpFd5U1GdRc= cloud.google.com/go/pubsublite v1.5.0/go.mod h1:xapqNQ1CuLfGi23Yda/9l4bBCKz/wC3KIJ5gKcxveZg= cloud.google.com/go/pubsublite v1.6.0/go.mod h1:1eFCS0U11xlOuMFV/0iBqw3zP12kddMeCbj/F3FSj9k= cloud.google.com/go/pubsublite v1.7.0/go.mod h1:8hVMwRXfDfvGm3fahVbtDbiLePT3gpoiJYJY+vxWxVM= +cloud.google.com/go/pubsublite v1.8.1/go.mod h1:fOLdU4f5xldK4RGJrBMm+J7zMWNj/k4PxwEZXy39QS0= cloud.google.com/go/recaptchaenterprise v1.3.1/go.mod h1:OdD+q+y4XGeAlxRaMn1Y7/GveP6zmq76byL6tjPE7d4= cloud.google.com/go/recaptchaenterprise/v2 v2.1.0/go.mod h1:w9yVqajwroDNTfGuhmOjPDN//rZGySaf6PtFVcSCa7o= cloud.google.com/go/recaptchaenterprise/v2 v2.2.0/go.mod h1:/Zu5jisWGeERrd5HnlS3EUGb/D335f9k51B/FVil0jk= @@ -449,46 +566,56 @@ cloud.google.com/go/recaptchaenterprise/v2 v2.4.0/go.mod h1:Am3LHfOuBstrLrNCBrlI cloud.google.com/go/recaptchaenterprise/v2 v2.5.0/go.mod h1:O8LzcHXN3rz0j+LBC91jrwI3R+1ZSZEWrfL7XHgNo9U= cloud.google.com/go/recaptchaenterprise/v2 v2.6.0/go.mod h1:RPauz9jeLtB3JVzg6nCbe12qNoaa8pXc4d/YukAmcnA= cloud.google.com/go/recaptchaenterprise/v2 v2.7.0/go.mod h1:19wVj/fs5RtYtynAPJdDTb69oW0vNHYDBTbB4NvMD9c= +cloud.google.com/go/recaptchaenterprise/v2 v2.7.2/go.mod h1:kR0KjsJS7Jt1YSyWFkseQ756D45kaYNTlDPPaRAvDBU= cloud.google.com/go/recommendationengine v0.5.0/go.mod h1:E5756pJcVFeVgaQv3WNpImkFP8a+RptV6dDLGPILjvg= cloud.google.com/go/recommendationengine v0.6.0/go.mod h1:08mq2umu9oIqc7tDy8sx+MNJdLG0fUi3vaSVbztHgJ4= cloud.google.com/go/recommendationengine v0.7.0/go.mod h1:1reUcE3GIu6MeBz/h5xZJqNLuuVjNg1lmWMPyjatzac= +cloud.google.com/go/recommendationengine v0.8.1/go.mod h1:MrZihWwtFYWDzE6Hz5nKcNz3gLizXVIDI/o3G1DLcrE= cloud.google.com/go/recommender v1.5.0/go.mod h1:jdoeiBIVrJe9gQjwd759ecLJbxCDED4A6p+mqoqDvTg= cloud.google.com/go/recommender v1.6.0/go.mod h1:+yETpm25mcoiECKh9DEScGzIRyDKpZ0cEhWGo+8bo+c= cloud.google.com/go/recommender v1.7.0/go.mod h1:XLHs/W+T8olwlGOgfQenXBTbIseGclClff6lhFVe9Bs= cloud.google.com/go/recommender v1.8.0/go.mod h1:PkjXrTT05BFKwxaUxQmtIlrtj0kph108r02ZZQ5FE70= cloud.google.com/go/recommender v1.9.0/go.mod h1:PnSsnZY7q+VL1uax2JWkt/UegHssxjUVVCrX52CuEmQ= +cloud.google.com/go/recommender v1.10.1/go.mod h1:XFvrE4Suqn5Cq0Lf+mCP6oBHD/yRMA8XxP5sb7Q7gpA= cloud.google.com/go/redis v1.7.0/go.mod h1:V3x5Jq1jzUcg+UNsRvdmsfuFnit1cfe3Z/PGyq/lm4Y= cloud.google.com/go/redis v1.8.0/go.mod h1:Fm2szCDavWzBk2cDKxrkmWBqoCiL1+Ctwq7EyqBCA/A= cloud.google.com/go/redis v1.9.0/go.mod h1:HMYQuajvb2D0LvMgZmLDZW8V5aOC/WxstZHiy4g8OiA= cloud.google.com/go/redis v1.10.0/go.mod h1:ThJf3mMBQtW18JzGgh41/Wld6vnDDc/F/F35UolRZPM= cloud.google.com/go/redis v1.11.0/go.mod h1:/X6eicana+BWcUda5PpwZC48o37SiFVTFSs0fWAJ7uQ= +cloud.google.com/go/redis v1.13.1/go.mod h1:VP7DGLpE91M6bcsDdMuyCm2hIpB6Vp2hI090Mfd1tcg= cloud.google.com/go/resourcemanager v1.3.0/go.mod h1:bAtrTjZQFJkiWTPDb1WBjzvc6/kifjj4QBYuKCCoqKA= cloud.google.com/go/resourcemanager v1.4.0/go.mod h1:MwxuzkumyTX7/a3n37gmsT3py7LIXwrShilPh3P1tR0= cloud.google.com/go/resourcemanager v1.5.0/go.mod h1:eQoXNAiAvCf5PXxWxXjhKQoTMaUSNrEfg+6qdf/wots= cloud.google.com/go/resourcemanager v1.6.0/go.mod h1:YcpXGRs8fDzcUl1Xw8uOVmI8JEadvhRIkoXXUNVYcVo= cloud.google.com/go/resourcemanager v1.7.0/go.mod h1:HlD3m6+bwhzj9XCouqmeiGuni95NTrExfhoSrkC/3EI= +cloud.google.com/go/resourcemanager v1.9.1/go.mod h1:dVCuosgrh1tINZ/RwBufr8lULmWGOkPS8gL5gqyjdT8= cloud.google.com/go/resourcesettings v1.3.0/go.mod h1:lzew8VfESA5DQ8gdlHwMrqZs1S9V87v3oCnKCWoOuQU= cloud.google.com/go/resourcesettings v1.4.0/go.mod h1:ldiH9IJpcrlC3VSuCGvjR5of/ezRrOxFtpJoJo5SmXg= cloud.google.com/go/resourcesettings v1.5.0/go.mod h1:+xJF7QSG6undsQDfsCJyqWXyBwUoJLhetkRMDRnIoXA= +cloud.google.com/go/resourcesettings v1.6.1/go.mod h1:M7mk9PIZrC5Fgsu1kZJci6mpgN8o0IUzVx3eJU3y4Jw= cloud.google.com/go/retail v1.8.0/go.mod h1:QblKS8waDmNUhghY2TI9O3JLlFk8jybHeV4BF19FrE4= cloud.google.com/go/retail v1.9.0/go.mod h1:g6jb6mKuCS1QKnH/dpu7isX253absFl6iE92nHwlBUY= cloud.google.com/go/retail v1.10.0/go.mod h1:2gDk9HsL4HMS4oZwz6daui2/jmKvqShXKQuB2RZ+cCc= cloud.google.com/go/retail v1.11.0/go.mod h1:MBLk1NaWPmh6iVFSz9MeKG/Psyd7TAgm6y/9L2B4x9Y= cloud.google.com/go/retail v1.12.0/go.mod h1:UMkelN/0Z8XvKymXFbD4EhFJlYKRx1FGhQkVPU5kF14= +cloud.google.com/go/retail v1.14.1/go.mod h1:y3Wv3Vr2k54dLNIrCzenyKG8g8dhvhncT2NcNjb/6gE= cloud.google.com/go/run v0.2.0/go.mod h1:CNtKsTA1sDcnqqIFR3Pb5Tq0usWxJJvsWOCPldRU3Do= cloud.google.com/go/run v0.3.0/go.mod h1:TuyY1+taHxTjrD0ZFk2iAR+xyOXEA0ztb7U3UNA0zBo= cloud.google.com/go/run v0.8.0/go.mod h1:VniEnuBwqjigv0A7ONfQUaEItaiCRVujlMqerPPiktM= cloud.google.com/go/run v0.9.0/go.mod h1:Wwu+/vvg8Y+JUApMwEDfVfhetv30hCG4ZwDR/IXl2Qg= +cloud.google.com/go/run v1.2.0/go.mod h1:36V1IlDzQ0XxbQjUx6IYbw8H3TJnWvhii963WW3B/bo= cloud.google.com/go/scheduler v1.4.0/go.mod h1:drcJBmxF3aqZJRhmkHQ9b3uSSpQoltBPGPxGAWROx6s= cloud.google.com/go/scheduler v1.5.0/go.mod h1:ri073ym49NW3AfT6DZi21vLZrG07GXr5p3H1KxN5QlI= cloud.google.com/go/scheduler v1.6.0/go.mod h1:SgeKVM7MIwPn3BqtcBntpLyrIJftQISRrYB5ZtT+KOk= cloud.google.com/go/scheduler v1.7.0/go.mod h1:jyCiBqWW956uBjjPMMuX09n3x37mtyPJegEWKxRsn44= cloud.google.com/go/scheduler v1.8.0/go.mod h1:TCET+Y5Gp1YgHT8py4nlg2Sew8nUHMqcpousDgXJVQc= cloud.google.com/go/scheduler v1.9.0/go.mod h1:yexg5t+KSmqu+njTIh3b7oYPheFtBWGcbVUYF1GGMIc= +cloud.google.com/go/scheduler v1.10.1/go.mod h1:R63Ldltd47Bs4gnhQkmNDse5w8gBRrhObZ54PxgR2Oo= cloud.google.com/go/secretmanager v1.6.0/go.mod h1:awVa/OXF6IiyaU1wQ34inzQNc4ISIDIrId8qE5QGgKA= cloud.google.com/go/secretmanager v1.8.0/go.mod h1:hnVgi/bN5MYHd3Gt0SPuTPPp5ENina1/LxM+2W9U9J4= cloud.google.com/go/secretmanager v1.9.0/go.mod h1:b71qH2l1yHmWQHt9LC80akm86mX8AL6X1MA01dW8ht4= cloud.google.com/go/secretmanager v1.10.0/go.mod h1:MfnrdvKMPNra9aZtQFvBcvRU54hbPD8/HayQdlUgJpU= +cloud.google.com/go/secretmanager v1.11.1/go.mod h1:znq9JlXgTNdBeQk9TBW/FnR/W4uChEKGeqQWAJ8SXFw= cloud.google.com/go/security v1.5.0/go.mod h1:lgxGdyOKKjHL4YG3/YwIL2zLqMFCKs0UbQwgyZmfJl4= cloud.google.com/go/security v1.7.0/go.mod h1:mZklORHl6Bg7CNnnjLH//0UlAlaXqiG7Lb9PsPXLfD0= cloud.google.com/go/security v1.8.0/go.mod h1:hAQOwgmaHhztFhiQ41CjDODdWP0+AE1B3sX4OFlq+GU= @@ -496,12 +623,14 @@ cloud.google.com/go/security v1.9.0/go.mod h1:6Ta1bO8LXI89nZnmnsZGp9lVoVWXqsVbIq cloud.google.com/go/security v1.10.0/go.mod h1:QtOMZByJVlibUT2h9afNDWRZ1G96gVywH8T5GUSb9IA= cloud.google.com/go/security v1.12.0/go.mod h1:rV6EhrpbNHrrxqlvW0BWAIawFWq3X90SduMJdFwtLB8= cloud.google.com/go/security v1.13.0/go.mod h1:Q1Nvxl1PAgmeW0y3HTt54JYIvUdtcpYKVfIB8AOMZ+0= +cloud.google.com/go/security v1.15.1/go.mod h1:MvTnnbsWnehoizHi09zoiZob0iCHVcL4AUBj76h9fXA= cloud.google.com/go/securitycenter v1.13.0/go.mod h1:cv5qNAqjY84FCN6Y9z28WlkKXyWsgLO832YiWwkCWcU= cloud.google.com/go/securitycenter v1.14.0/go.mod h1:gZLAhtyKv85n52XYWt6RmeBdydyxfPeTrpToDPw4Auc= cloud.google.com/go/securitycenter v1.15.0/go.mod h1:PeKJ0t8MoFmmXLXWm41JidyzI3PJjd8sXWaVqg43WWk= cloud.google.com/go/securitycenter v1.16.0/go.mod h1:Q9GMaLQFUD+5ZTabrbujNWLtSLZIZF7SAR0wWECrjdk= cloud.google.com/go/securitycenter v1.18.1/go.mod h1:0/25gAzCM/9OL9vVx4ChPeM/+DlfGQJDwBy/UC8AKK0= cloud.google.com/go/securitycenter v1.19.0/go.mod h1:LVLmSg8ZkkyaNy4u7HCIshAngSQ8EcIRREP3xBnyfag= +cloud.google.com/go/securitycenter v1.23.0/go.mod h1:8pwQ4n+Y9WCWM278R8W3nF65QtY172h4S8aXyI9/hsQ= cloud.google.com/go/servicecontrol v1.4.0/go.mod h1:o0hUSJ1TXJAmi/7fLJAedOovnujSEvjKCAFNXPQ1RaU= cloud.google.com/go/servicecontrol v1.5.0/go.mod h1:qM0CnXHhyqKVuiZnGKrIurvVImCs8gmqWsDoqe9sU1s= cloud.google.com/go/servicecontrol v1.10.0/go.mod h1:pQvyvSRh7YzUF2efw7H87V92mxU8FnFDawMClGCNuAA= @@ -513,6 +642,8 @@ cloud.google.com/go/servicedirectory v1.6.0/go.mod h1:pUlbnWsLH9c13yGkxCmfumWEPj cloud.google.com/go/servicedirectory v1.7.0/go.mod h1:5p/U5oyvgYGYejufvxhgwjL8UVXjkuw7q5XcG10wx1U= cloud.google.com/go/servicedirectory v1.8.0/go.mod h1:srXodfhY1GFIPvltunswqXpVxFPpZjf8nkKQT7XcXaY= cloud.google.com/go/servicedirectory v1.9.0/go.mod h1:29je5JjiygNYlmsGz8k6o+OZ8vd4f//bQLtvzkPPT/s= +cloud.google.com/go/servicedirectory v1.10.1/go.mod h1:Xv0YVH8s4pVOwfM/1eMTl0XJ6bzIOSLDt8f8eLaGOxQ= +cloud.google.com/go/servicedirectory v1.11.0/go.mod h1:Xv0YVH8s4pVOwfM/1eMTl0XJ6bzIOSLDt8f8eLaGOxQ= cloud.google.com/go/servicemanagement v1.4.0/go.mod h1:d8t8MDbezI7Z2R1O/wu8oTggo3BI2GKYbdG4y/SJTco= cloud.google.com/go/servicemanagement v1.5.0/go.mod h1:XGaCRe57kfqu4+lRxaFEAuqmjzF0r+gWHjWqKqBvKFo= cloud.google.com/go/servicemanagement v1.6.0/go.mod h1:aWns7EeeCOtGEX4OvZUWCCJONRZeFKiptqKf1D0l/Jc= @@ -524,15 +655,19 @@ cloud.google.com/go/serviceusage v1.6.0/go.mod h1:R5wwQcbOWsyuOfbP9tGdAnCAc6B9DR cloud.google.com/go/shell v1.3.0/go.mod h1:VZ9HmRjZBsjLGXusm7K5Q5lzzByZmJHf1d0IWHEN5X4= cloud.google.com/go/shell v1.4.0/go.mod h1:HDxPzZf3GkDdhExzD/gs8Grqk+dmYcEjGShZgYa9URw= cloud.google.com/go/shell v1.6.0/go.mod h1:oHO8QACS90luWgxP3N9iZVuEiSF84zNyLytb+qE2f9A= +cloud.google.com/go/shell v1.7.1/go.mod h1:u1RaM+huXFaTojTbW4g9P5emOrrmLE69KrxqQahKn4g= cloud.google.com/go/spanner v1.41.0/go.mod h1:MLYDBJR/dY4Wt7ZaMIQ7rXOTLjYrmxLE/5ve9vFfWos= cloud.google.com/go/spanner v1.44.0/go.mod h1:G8XIgYdOK+Fbcpbs7p2fiprDw4CaZX63whnSMLVBxjk= cloud.google.com/go/spanner v1.45.0/go.mod h1:FIws5LowYz8YAE1J8fOS7DJup8ff7xJeetWEo5REA2M= +cloud.google.com/go/spanner v1.47.0/go.mod h1:IXsJwVW2j4UKs0eYDqodab6HgGuA1bViSqW4uH9lfUI= cloud.google.com/go/speech v1.6.0/go.mod h1:79tcr4FHCimOp56lwC01xnt/WPJZc4v3gzyT7FoBkCM= cloud.google.com/go/speech v1.7.0/go.mod h1:KptqL+BAQIhMsj1kOP2la5DSEEerPDuOP/2mmkhHhZQ= cloud.google.com/go/speech v1.8.0/go.mod h1:9bYIl1/tjsAnMgKGHKmBZzXKEkGgtU+MpdDPTE9f7y0= cloud.google.com/go/speech v1.9.0/go.mod h1:xQ0jTcmnRFFM2RfX/U+rk6FQNUF6DQlydUSyoooSpco= cloud.google.com/go/speech v1.14.1/go.mod h1:gEosVRPJ9waG7zqqnsHpYTOoAS4KouMRLDFMekpJ0J0= cloud.google.com/go/speech v1.15.0/go.mod h1:y6oH7GhqCaZANH7+Oe0BhgIogsNInLlz542tg3VqeYI= +cloud.google.com/go/speech v1.17.1/go.mod h1:8rVNzU43tQvxDaGvqOhpDqgkJTFowBpDvCJ14kGlJYo= +cloud.google.com/go/speech v1.19.0/go.mod h1:8rVNzU43tQvxDaGvqOhpDqgkJTFowBpDvCJ14kGlJYo= cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw= cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0ZeosJ0Rtdos= cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohlUTyfDhBk= @@ -544,43 +679,56 @@ cloud.google.com/go/storage v1.23.0/go.mod h1:vOEEDNFnciUMhBeT6hsJIn3ieU5cFRmzeL cloud.google.com/go/storage v1.27.0/go.mod h1:x9DOL8TK/ygDUMieqwfhdpQryTeEkhGKMi80i/iqR2s= cloud.google.com/go/storage v1.28.1/go.mod h1:Qnisd4CqDdo6BGs2AD5LLnEsmSQ80wQ5ogcBBKhU86Y= cloud.google.com/go/storage v1.29.0/go.mod h1:4puEjyTKnku6gfKoTfNOU/W+a9JyuVNxjpS5GBrB8h4= +cloud.google.com/go/storage v1.30.1/go.mod h1:NfxhC0UJE1aXSx7CIIbCf7y9HKT7BiccwkR7+P7gN8E= cloud.google.com/go/storage v1.31.0 h1:+S3LjjEN2zZ+L5hOwj4+1OkGCsLVe0NzpXKQ1pSdTCI= cloud.google.com/go/storage v1.31.0/go.mod h1:81ams1PrhW16L4kF7qg+4mTq7SRs5HsbDTM0bWvrwJ0= +cloud.google.com/go/storage v1.32.0 h1:5w6DxEGOnktmJHarxAOUywxVW9lbNWIzlzzUltG/3+o= +cloud.google.com/go/storage v1.32.0/go.mod h1:Hhh/dogNRGca7IWv1RC2YqEn0c0G77ctA/OxflYkiD8= cloud.google.com/go/storagetransfer v1.5.0/go.mod h1:dxNzUopWy7RQevYFHewchb29POFv3/AaBgnhqzqiK0w= cloud.google.com/go/storagetransfer v1.6.0/go.mod h1:y77xm4CQV/ZhFZH75PLEXY0ROiS7Gh6pSKrM8dJyg6I= cloud.google.com/go/storagetransfer v1.7.0/go.mod h1:8Giuj1QNb1kfLAiWM1bN6dHzfdlDAVC9rv9abHot2W4= cloud.google.com/go/storagetransfer v1.8.0/go.mod h1:JpegsHHU1eXg7lMHkvf+KE5XDJ7EQu0GwNJbbVGanEw= +cloud.google.com/go/storagetransfer v1.10.0/go.mod h1:DM4sTlSmGiNczmV6iZyceIh2dbs+7z2Ayg6YAiQlYfA= cloud.google.com/go/talent v1.1.0/go.mod h1:Vl4pt9jiHKvOgF9KoZo6Kob9oV4lwd/ZD5Cto54zDRw= cloud.google.com/go/talent v1.2.0/go.mod h1:MoNF9bhFQbiJ6eFD3uSsg0uBALw4n4gaCaEjBw9zo8g= cloud.google.com/go/talent v1.3.0/go.mod h1:CmcxwJ/PKfRgd1pBjQgU6W3YBwiewmUzQYH5HHmSCmM= cloud.google.com/go/talent v1.4.0/go.mod h1:ezFtAgVuRf8jRsvyE6EwmbTK5LKciD4KVnHuDEFmOOA= cloud.google.com/go/talent v1.5.0/go.mod h1:G+ODMj9bsasAEJkQSzO2uHQWXHHXUomArjWQQYkqK6c= +cloud.google.com/go/talent v1.6.2/go.mod h1:CbGvmKCG61mkdjcqTcLOkb2ZN1SrQI8MDyma2l7VD24= cloud.google.com/go/texttospeech v1.4.0/go.mod h1:FX8HQHA6sEpJ7rCMSfXuzBcysDAuWusNNNvN9FELDd8= cloud.google.com/go/texttospeech v1.5.0/go.mod h1:oKPLhR4n4ZdQqWKURdwxMy0uiTS1xU161C8W57Wkea4= cloud.google.com/go/texttospeech v1.6.0/go.mod h1:YmwmFT8pj1aBblQOI3TfKmwibnsfvhIBzPXcW4EBovc= +cloud.google.com/go/texttospeech v1.7.1/go.mod h1:m7QfG5IXxeneGqTapXNxv2ItxP/FS0hCZBwXYqucgSk= cloud.google.com/go/tpu v1.3.0/go.mod h1:aJIManG0o20tfDQlRIej44FcwGGl/cD0oiRyMKG19IQ= cloud.google.com/go/tpu v1.4.0/go.mod h1:mjZaX8p0VBgllCzF6wcU2ovUXN9TONFLd7iz227X2Xg= cloud.google.com/go/tpu v1.5.0/go.mod h1:8zVo1rYDFuW2l4yZVY0R0fb/v44xLh3llq7RuV61fPM= +cloud.google.com/go/tpu v1.6.1/go.mod h1:sOdcHVIgDEEOKuqUoi6Fq53MKHJAtOwtz0GuKsWSH3E= cloud.google.com/go/trace v1.3.0/go.mod h1:FFUE83d9Ca57C+K8rDl/Ih8LwOzWIV1krKgxg6N0G28= cloud.google.com/go/trace v1.4.0/go.mod h1:UG0v8UBqzusp+z63o7FK74SdFE+AXpCLdFb1rshXG+Y= cloud.google.com/go/trace v1.8.0/go.mod h1:zH7vcsbAhklH8hWFig58HvxcxyQbaIqMarMg9hn5ECA= cloud.google.com/go/trace v1.9.0/go.mod h1:lOQqpE5IaWY0Ixg7/r2SjixMuc6lfTFeO4QGM4dQWOk= +cloud.google.com/go/trace v1.10.1/go.mod h1:gbtL94KE5AJLH3y+WVpfWILmqgc6dXcqgNXdOPAQTYk= cloud.google.com/go/translate v1.3.0/go.mod h1:gzMUwRjvOqj5i69y/LYLd8RrNQk+hOmIXTi9+nb3Djs= cloud.google.com/go/translate v1.4.0/go.mod h1:06Dn/ppvLD6WvA5Rhdp029IX2Mi3Mn7fpMRLPvXT5Wg= cloud.google.com/go/translate v1.5.0/go.mod h1:29YDSYveqqpA1CQFD7NQuP49xymq17RXNaUDdc0mNu0= cloud.google.com/go/translate v1.6.0/go.mod h1:lMGRudH1pu7I3n3PETiOB2507gf3HnfLV8qlkHZEyos= cloud.google.com/go/translate v1.7.0/go.mod h1:lMGRudH1pu7I3n3PETiOB2507gf3HnfLV8qlkHZEyos= +cloud.google.com/go/translate v1.8.1/go.mod h1:d1ZH5aaOA0CNhWeXeC8ujd4tdCFw8XoNWRljklu5RHs= +cloud.google.com/go/translate v1.8.2/go.mod h1:d1ZH5aaOA0CNhWeXeC8ujd4tdCFw8XoNWRljklu5RHs= cloud.google.com/go/video v1.8.0/go.mod h1:sTzKFc0bUSByE8Yoh8X0mn8bMymItVGPfTuUBUyRgxk= cloud.google.com/go/video v1.9.0/go.mod h1:0RhNKFRF5v92f8dQt0yhaHrEuH95m068JYOvLZYnJSw= cloud.google.com/go/video v1.12.0/go.mod h1:MLQew95eTuaNDEGriQdcYn0dTwf9oWiA4uYebxM5kdg= cloud.google.com/go/video v1.13.0/go.mod h1:ulzkYlYgCp15N2AokzKjy7MQ9ejuynOJdf1tR5lGthk= cloud.google.com/go/video v1.14.0/go.mod h1:SkgaXwT+lIIAKqWAJfktHT/RbgjSuY6DobxEp0C5yTQ= cloud.google.com/go/video v1.15.0/go.mod h1:SkgaXwT+lIIAKqWAJfktHT/RbgjSuY6DobxEp0C5yTQ= +cloud.google.com/go/video v1.17.1/go.mod h1:9qmqPqw/Ib2tLqaeHgtakU+l5TcJxCJbhFXM7UJjVzU= +cloud.google.com/go/video v1.19.0/go.mod h1:9qmqPqw/Ib2tLqaeHgtakU+l5TcJxCJbhFXM7UJjVzU= cloud.google.com/go/videointelligence v1.6.0/go.mod h1:w0DIDlVRKtwPCn/C4iwZIJdvC69yInhW0cfi+p546uU= cloud.google.com/go/videointelligence v1.7.0/go.mod h1:k8pI/1wAhjznARtVT9U1llUaFNPh7muw8QyOUpavru4= cloud.google.com/go/videointelligence v1.8.0/go.mod h1:dIcCn4gVDdS7yte/w+koiXn5dWVplOZkE+xwG9FgK+M= cloud.google.com/go/videointelligence v1.9.0/go.mod h1:29lVRMPDYHikk3v8EdPSaL8Ku+eMzDljjuvRs105XoU= cloud.google.com/go/videointelligence v1.10.0/go.mod h1:LHZngX1liVtUhZvi2uNS0VQuOzNi2TkY1OakiuoUOjU= +cloud.google.com/go/videointelligence v1.11.1/go.mod h1:76xn/8InyQHarjTWsBR058SmlPCwQjgcvoW0aZykOvo= cloud.google.com/go/vision v1.2.0/go.mod h1:SmNwgObm5DpFBme2xpyOyasvBc1aPdjvMk2bBk0tKD0= cloud.google.com/go/vision/v2 v2.2.0/go.mod h1:uCdV4PpN1S0jyCyq8sIM42v2Y6zOLkZs+4R9LrGYwFo= cloud.google.com/go/vision/v2 v2.3.0/go.mod h1:UO61abBx9QRMFkNBbf1D8B1LXdS2cGiiCRx0vSpZoUo= @@ -588,29 +736,37 @@ cloud.google.com/go/vision/v2 v2.4.0/go.mod h1:VtI579ll9RpVTrdKdkMzckdnwMyX2JILb cloud.google.com/go/vision/v2 v2.5.0/go.mod h1:MmaezXOOE+IWa+cS7OhRRLK2cNv1ZL98zhqFFZaaH2E= cloud.google.com/go/vision/v2 v2.6.0/go.mod h1:158Hes0MvOS9Z/bDMSFpjwsUrZ5fPrdwuyyvKSGAGMY= cloud.google.com/go/vision/v2 v2.7.0/go.mod h1:H89VysHy21avemp6xcf9b9JvZHVehWbET0uT/bcuY/0= +cloud.google.com/go/vision/v2 v2.7.2/go.mod h1:jKa8oSYBWhYiXarHPvP4USxYANYUEdEsQrloLjrSwJU= cloud.google.com/go/vmmigration v1.2.0/go.mod h1:IRf0o7myyWFSmVR1ItrBSFLFD/rJkfDCUTO4vLlJvsE= cloud.google.com/go/vmmigration v1.3.0/go.mod h1:oGJ6ZgGPQOFdjHuocGcLqX4lc98YQ7Ygq8YQwHh9A7g= cloud.google.com/go/vmmigration v1.5.0/go.mod h1:E4YQ8q7/4W9gobHjQg4JJSgXXSgY21nA5r8swQV+Xxc= cloud.google.com/go/vmmigration v1.6.0/go.mod h1:bopQ/g4z+8qXzichC7GW1w2MjbErL54rk3/C843CjfY= +cloud.google.com/go/vmmigration v1.7.1/go.mod h1:WD+5z7a/IpZ5bKK//YmT9E047AD+rjycCAvyMxGJbro= cloud.google.com/go/vmwareengine v0.1.0/go.mod h1:RsdNEf/8UDvKllXhMz5J40XxDrNJNN4sagiox+OI208= cloud.google.com/go/vmwareengine v0.2.2/go.mod h1:sKdctNJxb3KLZkE/6Oui94iw/xs9PRNC2wnNLXsHvH8= cloud.google.com/go/vmwareengine v0.3.0/go.mod h1:wvoyMvNWdIzxMYSpH/R7y2h5h3WFkx6d+1TIsP39WGY= +cloud.google.com/go/vmwareengine v0.4.1/go.mod h1:Px64x+BvjPZwWuc4HdmVhoygcXqEkGHXoa7uyfTgSI0= +cloud.google.com/go/vmwareengine v1.0.0/go.mod h1:Px64x+BvjPZwWuc4HdmVhoygcXqEkGHXoa7uyfTgSI0= cloud.google.com/go/vpcaccess v1.4.0/go.mod h1:aQHVbTWDYUR1EbTApSVvMq1EnT57ppDmQzZ3imqIk4w= cloud.google.com/go/vpcaccess v1.5.0/go.mod h1:drmg4HLk9NkZpGfCmZ3Tz0Bwnm2+DKqViEpeEpOq0m8= cloud.google.com/go/vpcaccess v1.6.0/go.mod h1:wX2ILaNhe7TlVa4vC5xce1bCnqE3AeH27RV31lnmZes= +cloud.google.com/go/vpcaccess v1.7.1/go.mod h1:FogoD46/ZU+JUBX9D606X21EnxiszYi2tArQwLY4SXs= cloud.google.com/go/webrisk v1.4.0/go.mod h1:Hn8X6Zr+ziE2aNd8SliSDWpEnSS1u4R9+xXZmFiHmGE= cloud.google.com/go/webrisk v1.5.0/go.mod h1:iPG6fr52Tv7sGk0H6qUFzmL3HHZev1htXuWDEEsqMTg= cloud.google.com/go/webrisk v1.6.0/go.mod h1:65sW9V9rOosnc9ZY7A7jsy1zoHS5W9IAXv6dGqhMQMc= cloud.google.com/go/webrisk v1.7.0/go.mod h1:mVMHgEYH0r337nmt1JyLthzMr6YxwN1aAIEc2fTcq7A= cloud.google.com/go/webrisk v1.8.0/go.mod h1:oJPDuamzHXgUc+b8SiHRcVInZQuybnvEW72PqTc7sSg= +cloud.google.com/go/webrisk v1.9.1/go.mod h1:4GCmXKcOa2BZcZPn6DCEvE7HypmEJcJkr4mtM+sqYPc= cloud.google.com/go/websecurityscanner v1.3.0/go.mod h1:uImdKm2wyeXQevQJXeh8Uun/Ym1VqworNDlBXQevGMo= cloud.google.com/go/websecurityscanner v1.4.0/go.mod h1:ebit/Fp0a+FWu5j4JOmJEV8S8CzdTkAS77oDsiSqYWQ= cloud.google.com/go/websecurityscanner v1.5.0/go.mod h1:Y6xdCPy81yi0SQnDY1xdNTNpfY1oAgXUlcfN3B3eSng= +cloud.google.com/go/websecurityscanner v1.6.1/go.mod h1:Njgaw3rttgRHXzwCB8kgCYqv5/rGpFCsBOvPbYgszpg= cloud.google.com/go/workflows v1.6.0/go.mod h1:6t9F5h/unJz41YqfBmqSASJSXccBLtD1Vwf+KmJENM0= cloud.google.com/go/workflows v1.7.0/go.mod h1:JhSrZuVZWuiDfKEFxU0/F1PQjmpnpcoISEXH2bcHC3M= cloud.google.com/go/workflows v1.8.0/go.mod h1:ysGhmEajwZxGn1OhGOGKsTXc5PyxOc0vfKf5Af+to4M= cloud.google.com/go/workflows v1.9.0/go.mod h1:ZGkj1aFIOd9c8Gerkjjq7OW7I5+l6cSvT3ujaO/WwSA= cloud.google.com/go/workflows v1.10.0/go.mod h1:fZ8LmRmZQWacon9UCX1r/g/DfAXx5VcPALq2CxzdePw= +cloud.google.com/go/workflows v1.11.1/go.mod h1:Z+t10G1wF7h8LgdY/EmRcQY8ptBD/nvofaL6FqlET6g= dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= gioui.org v0.0.0-20210308172011-57750fc8a0a6/go.mod h1:RSH6KIUZ0p2xy5zHDxgAM4zumjgTw83q2ge/PI+yyw8= git.sr.ht/~sbinet/gg v0.3.1/go.mod h1:KGYtlADtqsqANL9ueOFkWymvzUvLMQllU5Ixo+8v3pc= @@ -620,8 +776,6 @@ github.com/99designs/keyring v1.2.2 h1:pZd3neh/EmUzWONb35LxQfvuY7kiSXAq3HQd97+XB github.com/99designs/keyring v1.2.2/go.mod h1:wes/FrByc8j7lFOAGLGSNEg8f/PaI3cgTBqhFkHUrPk= github.com/Azure/azure-amqp-common-go/v4 v4.2.0 h1:q/jLx1KJ8xeI8XGfkOWMN9XrXzAfVTkyvCxPvHCjd2I= github.com/Azure/azure-amqp-common-go/v4 v4.2.0/go.mod h1:GD3m/WPPma+621UaU6KNjKEo5Hl09z86viKwQjTpV0Q= -github.com/Azure/azure-event-hubs-go/v3 v3.6.0 h1:UXRi5KewXYoTiekVjrj0gyGfbyGvtbYdot6/4IMf4I4= -github.com/Azure/azure-event-hubs-go/v3 v3.6.0/go.mod h1:UgyRnRU7H5e33igaLHJTqbkoNR1uj0j3MA/n7dABU24= github.com/Azure/azure-event-hubs-go/v3 v3.6.1 h1:vSiMmn3tOwgiLyfnmhT5K6Of/3QWRLaaNZPI0hFvZyU= github.com/Azure/azure-event-hubs-go/v3 v3.6.1/go.mod h1:i2NByb9Pr2na7y8wi/XefEVKkuA2CDUjCNoWQJtTsGo= github.com/Azure/azure-sdk-for-go v68.0.0+incompatible h1:fcYLmCpyNYRnvJbPerq7U0hS+6+I79yEDJBqVNcqUzU= @@ -629,8 +783,12 @@ github.com/Azure/azure-sdk-for-go v68.0.0+incompatible/go.mod h1:9XXNKU+eRnpl9mo github.com/Azure/azure-sdk-for-go/sdk/azcore v1.6.0/go.mod h1:bjGvMhVMb+EEm3VRNQawDMUyMMjo+S5ewNjflkep/0Q= github.com/Azure/azure-sdk-for-go/sdk/azcore v1.7.0 h1:8q4SaHjFsClSvuVne0ID/5Ka8u3fcIHyqkLjcFpNRHQ= github.com/Azure/azure-sdk-for-go/sdk/azcore v1.7.0/go.mod h1:bjGvMhVMb+EEm3VRNQawDMUyMMjo+S5ewNjflkep/0Q= +github.com/Azure/azure-sdk-for-go/sdk/azcore v1.7.1 h1:/iHxaJhsFr0+xVFfbMr5vxz848jyiWuIEDhYq3y5odY= +github.com/Azure/azure-sdk-for-go/sdk/azcore v1.7.1/go.mod h1:bjGvMhVMb+EEm3VRNQawDMUyMMjo+S5ewNjflkep/0Q= github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.3.0 h1:vcYCAze6p19qBW7MhZybIsqD8sMV8js0NyQM8JDnVtg= github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.3.0/go.mod h1:OQeznEEkTZ9OrhHJoDD8ZDq51FHgXjqtP9z6bEwBq9U= +github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.3.1 h1:LNHhpdK7hzUcx/k1LIcuh5k7k1LGIWLQfCjaneSj7Fc= +github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.3.1/go.mod h1:uE9zaUfEQT/nbQjVi2IblCG9iaLtZsuYZ8ne+PuQ02M= github.com/Azure/azure-sdk-for-go/sdk/internal v1.3.0 h1:sXr+ck84g/ZlZUOZiNELInmMgOsuGwdjjVkEIde0OtY= github.com/Azure/azure-sdk-for-go/sdk/internal v1.3.0/go.mod h1:okt5dMMTOFjX/aovMlrjvvXoPMBVSPzk9185BT0+eZM= github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/eventhub/armeventhub v1.1.1 h1:gZ1ZZvrVUhDNsGNpbo2N87Y0CJB8p3IS5UH9Z4Ui97g= @@ -668,10 +826,11 @@ github.com/Azure/go-autorest/logger v0.2.1 h1:IG7i4p/mDa2Ce4TRyAO8IHnVhAVF3RFU+Z github.com/Azure/go-autorest/logger v0.2.1/go.mod h1:T9E3cAhj2VqvPOtCYAvby9aBXkZmbF5NWuPV8+WeEW8= github.com/Azure/go-autorest/tracing v0.6.0 h1:TYi4+3m5t6K48TGI9AUdb+IzbnSxvnvUMfuitfgcfuo= github.com/Azure/go-autorest/tracing v0.6.0/go.mod h1:+vhtPC754Xsa23ID7GlGsrdKBpUA79WCAKPPZVC2DeU= -github.com/AzureAD/microsoft-authentication-library-for-go v1.0.0 h1:OBhqkivkhkMqLPymWEppkm7vgPQY2XsHoEkaMQ0AdZY= github.com/AzureAD/microsoft-authentication-library-for-go v1.0.0/go.mod h1:kgDmCTgBzIEPFElEF+FK0SdjAor06dRq2Go927dnQ6o= github.com/AzureAD/microsoft-authentication-library-for-go v1.1.0 h1:HCc0+LpPfpCKs6LGGLAhwBARt9632unrVcI6i8s/8os= github.com/AzureAD/microsoft-authentication-library-for-go v1.1.0/go.mod h1:wP83P5OoQ5p6ip3ScPr0BAq0BvuPAvacpEuSzyouqAI= +github.com/AzureAD/microsoft-authentication-library-for-go v1.1.1 h1:WpB/QDNLpMw72xHJc34BNNykqSOeEJDAWkhf0u12/Jk= +github.com/AzureAD/microsoft-authentication-library-for-go v1.1.1/go.mod h1:wP83P5OoQ5p6ip3ScPr0BAq0BvuPAvacpEuSzyouqAI= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c h1:RGWPOewvKIROun94nF7v2cua9qP+thov/7M50KEoeSU= @@ -692,91 +851,92 @@ github.com/andybalholm/brotli v1.0.5/go.mod h1:fO7iG3H7G2nSZ7m0zPUDn85XEX2GTukHG github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= github.com/apache/arrow/go/v10 v10.0.1/go.mod h1:YvhnlEePVnBS4+0z3fhPfUy7W1Ikj0Ih0vcRo/gZ1M0= github.com/apache/arrow/go/v11 v11.0.0/go.mod h1:Eg5OsL5H+e299f7u5ssuXsuHQVEGC4xei5aX110hRiI= +github.com/apache/arrow/go/v12 v12.0.0/go.mod h1:d+tV/eHZZ7Dz7RPrFKtPK02tpr+c9/PEd/zm8mDS9Vg= github.com/apache/arrow/go/v12 v12.0.1 h1:JsR2+hzYYjgSUkBSaahpqCetqZMr76djX80fF/DiJbg= github.com/apache/arrow/go/v12 v12.0.1/go.mod h1:weuTY7JvTG/HDPtMQxEUp7pU73vkLWMLpY67QwZ/WWw= github.com/apache/thrift v0.16.0/go.mod h1:PHK3hniurgQaNMZYaCLEqXKsYK8upmhPbmdP2FXSqgU= github.com/apache/thrift v0.18.1 h1:lNhK/1nqjbwbiOPDBPFJVKxgDEGSepKuTh6OLiXW8kg= github.com/apache/thrift v0.18.1/go.mod h1:rdQn/dCcDKEWjjylUeueum4vQEjG2v8v2PqriUnbr+I= -github.com/aws/aws-sdk-go v1.44.300 h1:Zn+3lqgYahIf9yfrwZ+g+hq/c3KzUBaQ8wqY/ZXiAbY= -github.com/aws/aws-sdk-go v1.44.300/go.mod h1:aVsgQcEevwlmQ7qHE9I3h+dtQgpqhFB+i8Phjh7fkwI= github.com/aws/aws-sdk-go v1.44.317 h1:+8XWrLmGMwPPXSRSLPzhgcGnzJ2mYkgkrcB9C/GnSOU= github.com/aws/aws-sdk-go v1.44.317/go.mod h1:aVsgQcEevwlmQ7qHE9I3h+dtQgpqhFB+i8Phjh7fkwI= -github.com/aws/aws-sdk-go-v2 v1.19.0 h1:klAT+y3pGFBU/qVf1uzwttpBbiuozJYWzNLHioyDJ+k= -github.com/aws/aws-sdk-go-v2 v1.19.0/go.mod h1:uzbQtefpm44goOPmdKyAlXSNcwlRgF3ePWVW6EtJvvw= +github.com/aws/aws-sdk-go v1.44.332 h1:Ze+98F41+LxoJUdsisAFThV+0yYYLYw17/Vt0++nFYM= +github.com/aws/aws-sdk-go v1.44.332/go.mod h1:aVsgQcEevwlmQ7qHE9I3h+dtQgpqhFB+i8Phjh7fkwI= github.com/aws/aws-sdk-go-v2 v1.20.0 h1:INUDpYLt4oiPOJl0XwZDK2OVAVf0Rzo+MGVTv9f+gy8= github.com/aws/aws-sdk-go-v2 v1.20.0/go.mod h1:uWOr0m0jDsiWw8nnXiqZ+YG6LdvAlGYDLLf2NmHZoy4= -github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.10 h1:dK82zF6kkPeCo8J1e+tGx4JdvDIQzj7ygIoLg8WMuGs= -github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.10/go.mod h1:VeTZetY5KRJLuD/7fkQXMU6Mw7H5m/KP2J5Iy9osMno= +github.com/aws/aws-sdk-go-v2 v1.21.0 h1:gMT0IW+03wtYJhRqTVYn0wLzwdnK9sRMcxmtfGzRdJc= +github.com/aws/aws-sdk-go-v2 v1.21.0/go.mod h1:/RfNgGmRxI+iFOB1OeJUyxiU+9s88k3pfHvDagGEp0M= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.11 h1:/MS8AzqYNAhhRNalOmxUvYs8VEbNGifTnzhPFdcRQkQ= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.11/go.mod h1:va22++AdXht4ccO3kH2SHkHHYvZ2G9Utz+CXKmm2CaU= -github.com/aws/aws-sdk-go-v2/config v1.18.28 h1:TINEaKyh1Td64tqFvn09iYpKiWjmHYrG1fa91q2gnqw= -github.com/aws/aws-sdk-go-v2/config v1.18.28/go.mod h1:nIL+4/8JdAuNHEjn/gPEXqtnS02Q3NXB/9Z7o5xE4+A= +github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.13 h1:OPLEkmhXf6xFPiz0bLeDArZIDx1NNS4oJyG4nv3Gct0= +github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.13/go.mod h1:gpAbvyDGQFozTEmlTFO8XcQKHzubdq0LzRyJpG6MiXM= github.com/aws/aws-sdk-go-v2/config v1.18.32 h1:tqEOvkbTxwEV7hToRcJ1xZRjcATqwDVsWbAscgRKyNI= github.com/aws/aws-sdk-go-v2/config v1.18.32/go.mod h1:U3ZF0fQRRA4gnbn9GGvOWLoT2EzzZfAWeKwnVrm1rDc= -github.com/aws/aws-sdk-go-v2/credentials v1.13.27 h1:dz0yr/yR1jweAnsCx+BmjerUILVPQ6FS5AwF/OyG1kA= -github.com/aws/aws-sdk-go-v2/credentials v1.13.27/go.mod h1:syOqAek45ZXZp29HlnRS/BNgMIW6uiRmeuQsz4Qh2UE= +github.com/aws/aws-sdk-go-v2/config v1.18.37 h1:RNAfbPqw1CstCooHaTPhScz7z1PyocQj0UL+l95CgzI= +github.com/aws/aws-sdk-go-v2/config v1.18.37/go.mod h1:8AnEFxW9/XGKCbjYDCJy7iltVNyEI9Iu9qC21UzhhgQ= github.com/aws/aws-sdk-go-v2/credentials v1.13.31 h1:vJyON3lG7R8VOErpJJBclBADiWTwzcwdkQpTKx8D2sk= github.com/aws/aws-sdk-go-v2/credentials v1.13.31/go.mod h1:T4sESjBtY2lNxLgkIASmeP57b5j7hTQqCbqG0tWnxC4= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.5 h1:kP3Me6Fy3vdi+9uHd7YLr6ewPxRL+PU6y15urfTaamU= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.5/go.mod h1:Gj7tm95r+QsDoN2Fhuz/3npQvcZbkEf5mL70n3Xfluc= +github.com/aws/aws-sdk-go-v2/credentials v1.13.35 h1:QpsNitYJu0GgvMBLUIYu9H4yryA5kMksjeIVQfgXrt8= +github.com/aws/aws-sdk-go-v2/credentials v1.13.35/go.mod h1:o7rCaLtvK0hUggAGclf76mNGGkaG5a9KWlp+d9IpcV8= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.7 h1:X3H6+SU21x+76LRglk21dFRgMTJMa5QcpW+SqUf5BBg= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.7/go.mod h1:3we0V09SwcJBzNlnyovrR2wWJhWmVdqAsmVs4uronv8= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.72 h1:m0MmP89v1B0t3b8W8rtATU76KNsodak69QtiokHyEvo= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.72/go.mod h1:ylOTxIuoTL+XjH46Omv2iPjHdeGUk3SQ4hxYho4EHMA= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.11 h1:uDZJF1hu0EVT/4bogChk8DyjSF6fof6uL/0Y26Ma7Fg= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.11/go.mod h1:TEPP4tENqBGO99KwVpV9MlOX4NSrSLP8u3KRy2CDwA8= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.76 h1:DJ1kHj0GI9BbX+XhF0kHxlzOVjcncmDUXmCvXdbfdAE= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.76/go.mod h1:/AZCdswMSgwpB2yMSFfY5H4pVeBLnCuPehdmO/r3xSM= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.35 h1:hMUCiE3Zi5AHrRNGf5j985u0WyqI6r2NULhUfo0N/No= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.35/go.mod h1:ipR5PvpSPqIqL5Mi82BxLnfMkHVbmco8kUwO2xrCi0M= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.81 h1:PQ9zoe2GEoTVSVPuNtjNrKeVPvyVPWesETMPb7KB3Fk= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.81/go.mod h1:EztVLIU9xGitjdZ1TyHWL9IcNx4952FlqKJe6GLG2z4= github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.37 h1:zr/gxAZkMcvP71ZhQOcvdm8ReLjFgIXnIn0fw5AM7mo= github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.37/go.mod h1:Pdn4j43v49Kk6+82spO3Tu5gSeQXRsxo56ePPQAvFiA= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.29 h1:yOpYx+FTBdpk/g+sBU6Cb1H0U/TLEcYYp66mYqsPpcc= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.29/go.mod h1:M/eUABlDbw2uVrdAn+UsI6M727qp2fxkp8K0ejcBDUY= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.41 h1:22dGT7PneFMx4+b3pz7lMTRyN8ZKH7M2cW4GP9yUS2g= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.41/go.mod h1:CrObHAuPneJBlfEJ5T3szXOUkLEThaGfvnhTf33buas= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.31 h1:0HCMIkAkVY9KMgueD8tf4bRTUanzEYvhw7KkPXIMpO0= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.31/go.mod h1:fTJDMe8LOFYtqiFFFeHA+SVMAwqLhoq0kcInYoLa9Js= -github.com/aws/aws-sdk-go-v2/internal/ini v1.3.36 h1:8r5m1BoAWkn0TDC34lUculryf7nUF25EgIMdjvGCkgo= -github.com/aws/aws-sdk-go-v2/internal/ini v1.3.36/go.mod h1:Rmw2M1hMVTwiUhjwMoIBFWFJMhvJbct06sSidxInkhY= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.35 h1:SijA0mgjV8E+8G45ltVHs0fvKpTj8xmZJ3VwhGKtUSI= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.35/go.mod h1:SJC1nEVVva1g3pHAIdCp7QsRIkMmLAgoDquQ9Rr8kYw= github.com/aws/aws-sdk-go-v2/internal/ini v1.3.38 h1:+i1DOFrW3YZ3apE45tCal9+aDKK6kNEbW6Ib7e1nFxE= github.com/aws/aws-sdk-go-v2/internal/ini v1.3.38/go.mod h1:1/jLp0OgOaWIetycOmycW+vYTYgTZFPttJQRgsI1PoU= -github.com/aws/aws-sdk-go-v2/internal/v4a v1.0.27 h1:cZG7psLfqpkB6H+fIrgUDWmlzM474St1LP0jcz272yI= -github.com/aws/aws-sdk-go-v2/internal/v4a v1.0.27/go.mod h1:ZdjYvJpDlefgh8/hWelJhqgqJeodxu4SmbVsSdBlL7E= +github.com/aws/aws-sdk-go-v2/internal/ini v1.3.42 h1:GPUcE/Yq7Ur8YSUk6lVkoIMWnJNO0HT18GUzCWCgCI0= +github.com/aws/aws-sdk-go-v2/internal/ini v1.3.42/go.mod h1:rzfdUlfA+jdgLDmPKjd3Chq9V7LVLYo1Nz++Wb91aRo= github.com/aws/aws-sdk-go-v2/internal/v4a v1.1.0 h1:U5yySdwt2HPo/pnQec04DImLzWORbeWML1fJiLkKruI= github.com/aws/aws-sdk-go-v2/internal/v4a v1.1.0/go.mod h1:EhC/83j8/hL/UB1WmExo3gkElaja/KlmZM/gl1rTfjM= -github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.11 h1:y2+VQzC6Zh2ojtV2LoC0MNwHWc6qXv/j2vrQtlftkdA= -github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.11/go.mod h1:iV4q2hsqtNECrfmlXyord9u4zyuFEJX9eLgLpSPzWA8= +github.com/aws/aws-sdk-go-v2/internal/v4a v1.1.4 h1:6lJvvkQ9HmbHZ4h/IEwclwv2mrTW8Uq1SOB/kXy0mfw= +github.com/aws/aws-sdk-go-v2/internal/v4a v1.1.4/go.mod h1:1PrKYwxTM+zjpw9Y41KFtoJCQrJ34Z47Y4VgVbfndjo= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.12 h1:uAiiHnWihGP2rVp64fHwzLDrswGjEjsPszwRYMiYQPU= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.12/go.mod h1:fUTHpOXqRQpXvEpDPSa3zxCc2fnpW6YnBoba+eQr+Bg= -github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.30 h1:Bje8Xkh2OWpjBdNfXLrnn8eZg569dUQmhgtydxAYyP0= -github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.30/go.mod h1:qQtIBl5OVMfmeQkz8HaVyh5DzFmmFXyvK27UgIgOr4c= +github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.14 h1:m0QTSI6pZYJTk5WSKx3fm5cNW/DCicVzULBgU/6IyD0= +github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.14/go.mod h1:dDilntgHy9WnHXsh7dDtUPgHKEfTJIBUTHM8OWm0f/0= github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.32 h1:kvN1jPHr9UffqqG3bSgZ8tx4+1zKVHz/Ktw/BwW6hX8= github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.32/go.mod h1:QmMEM7es84EUkbYWcpnkx8i5EW2uERPfrTFeOch128Y= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.29 h1:IiDolu/eLmuB18DRZibj77n1hHQT7z12jnGO7Ze3pLc= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.29/go.mod h1:fDbkK4o7fpPXWn8YAPmTieAMuB9mk/VgvW64uaUqxd4= +github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.36 h1:eev2yZX7esGRjqRbnVk1UxMLw4CyVZDpZXRCcy75oQk= +github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.36/go.mod h1:lGnOkH9NJATw0XEPcAknFBj3zzNTEGRHtSw+CwC1YTg= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.31 h1:auGDJ0aLZahF5SPvkJ6WcUuX7iQ7kyl2MamV7Tm8QBk= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.31/go.mod h1:3+lloe3sZuBQw1aBc5MyndvodzQlyqCZ7x1QPDHaWP4= -github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.14.4 h1:hx4WksB0NRQ9utR+2c3gEGzl6uKj3eM6PMQ6tN3lgXs= -github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.14.4/go.mod h1:JniVpqvw90sVjNqanGLufrVapWySL28fhBlYgl96Q/w= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.35 h1:CdzPW9kKitgIiLV1+MHobfR5Xg25iYnyzWZhyQuSlDI= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.35/go.mod h1:QGF2Rs33W5MaN9gYdEQOBBFPLwTZkEhRwI33f7KIG0o= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.0 h1:Wgjft9X4W5pMeuqgPCHIQtbZ87wsgom7S5F8obreg+c= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.0/go.mod h1:FWNzS4+zcWAP05IF7TDYTY1ysZAzIvogxWaDT9p8fsA= -github.com/aws/aws-sdk-go-v2/service/s3 v1.37.0 h1:PalLOEGZ/4XfQxpGZFTLaoJSmPoybnqJYotaIZEf/Rg= -github.com/aws/aws-sdk-go-v2/service/s3 v1.37.0/go.mod h1:PwyKKVL0cNkC37QwLcrhyeCrAk+5bY8O2ou7USyAS2A= +github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.4 h1:v0jkRigbSD6uOdwcaUQmgEwG1BkPfAPDqaeNt/29ghg= +github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.4/go.mod h1:LhTyt8J04LL+9cIt7pYJ5lbS/U98ZmXovLOR/4LUsk8= github.com/aws/aws-sdk-go-v2/service/s3 v1.38.1 h1:mTgFVlfQT8gikc5+/HwD8UL9jnUro5MGv8n/VEYF12I= github.com/aws/aws-sdk-go-v2/service/s3 v1.38.1/go.mod h1:6SOWLiobcZZshbmECRTADIRYliPL0etqFSigauQEeT0= -github.com/aws/aws-sdk-go-v2/service/sso v1.12.13 h1:sWDv7cMITPcZ21QdreULwxOOAmE05JjEsT6fCDtDA9k= -github.com/aws/aws-sdk-go-v2/service/sso v1.12.13/go.mod h1:DfX0sWuT46KpcqbMhJ9QWtxAIP1VozkDWf8VAkByjYY= +github.com/aws/aws-sdk-go-v2/service/s3 v1.38.5 h1:A42xdtStObqy7NGvzZKpnyNXvoOmm+FENobZ0/ssHWk= +github.com/aws/aws-sdk-go-v2/service/s3 v1.38.5/go.mod h1:rDGMZA7f4pbmTtPOk5v5UM2lmX6UAbRnMDJeDvnH7AM= github.com/aws/aws-sdk-go-v2/service/sso v1.13.1 h1:DSNpSbfEgFXRV+IfEcKE5kTbqxm+MeF5WgyeRlsLnHY= github.com/aws/aws-sdk-go-v2/service/sso v1.13.1/go.mod h1:TC9BubuFMVScIU+TLKamO6VZiYTkYoEHqlSQwAe2omw= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.14.13 h1:BFubHS/xN5bjl818QaroN6mQdjneYQ+AOx44KNXlyH4= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.14.13/go.mod h1:BzqsVVFduubEmzrVtUFQQIQdFqvUItF8XUq2EnS8Wog= +github.com/aws/aws-sdk-go-v2/service/sso v1.13.5 h1:oCvTFSDi67AX0pOX3PuPdGFewvLRU2zzFSrTsgURNo0= +github.com/aws/aws-sdk-go-v2/service/sso v1.13.5/go.mod h1:fIAwKQKBFu90pBxx07BFOMJLpRUGu8VOzLJakeY+0K4= github.com/aws/aws-sdk-go-v2/service/ssooidc v1.15.1 h1:hd0SKLMdOL/Sl6Z0np1PX9LeH2gqNtBe0MhTedA8MGI= github.com/aws/aws-sdk-go-v2/service/ssooidc v1.15.1/go.mod h1:XO/VcyoQ8nKyKfFW/3DMsRQXsfh/052tHTWmg3xBXRg= -github.com/aws/aws-sdk-go-v2/service/sts v1.19.3 h1:e5mnydVdCVWxP+5rPAGi2PYxC7u2OZgH1ypC114H04U= -github.com/aws/aws-sdk-go-v2/service/sts v1.19.3/go.mod h1:yVGZA1CPkmUhBdA039jXNJJG7/6t+G+EBWmFq23xqnY= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.15.5 h1:dnInJb4S0oy8aQuri1mV6ipLlnZPfnsDNB9BGO9PDNY= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.15.5/go.mod h1:yygr8ACQRY2PrEcy3xsUI357stq2AxnFM6DIsR9lij4= github.com/aws/aws-sdk-go-v2/service/sts v1.21.1 h1:pAOJj+80tC8sPVgSDHzMYD6KLWsaLQ1kZw31PTeORbs= github.com/aws/aws-sdk-go-v2/service/sts v1.21.1/go.mod h1:G8SbvL0rFk4WOJroU8tKBczhsbhj2p/YY7qeJezJ3CI= -github.com/aws/smithy-go v1.13.5 h1:hgz0X/DX0dGqTYpGALqXJoRKRj5oQ7150i5FdTePzO8= -github.com/aws/smithy-go v1.13.5/go.mod h1:Tg+OJXh4MB2R/uN61Ko2f6hTZwB/ZYGOtib8J3gBHzA= +github.com/aws/aws-sdk-go-v2/service/sts v1.21.5 h1:CQBFElb0LS8RojMJlxRSo/HXipvTZW2S44Lt9Mk2aYQ= +github.com/aws/aws-sdk-go-v2/service/sts v1.21.5/go.mod h1:VC7JDqsqiwXukYEDjoHh9U0fOJtNWh04FPQz4ct4GGU= github.com/aws/smithy-go v1.14.0 h1:+X90sB94fizKjDmwb4vyl2cTTPXTE5E2G/1mjByb0io= github.com/aws/smithy-go v1.14.0/go.mod h1:Tg+OJXh4MB2R/uN61Ko2f6hTZwB/ZYGOtib8J3gBHzA= +github.com/aws/smithy-go v1.14.2 h1:MJU9hqBGbvWZdApzpvoF2WAIJDbtjK2NDJSiJP7HblQ= +github.com/aws/smithy-go v1.14.2/go.mod h1:Tg+OJXh4MB2R/uN61Ko2f6hTZwB/ZYGOtib8J3gBHzA= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= @@ -811,6 +971,7 @@ github.com/cncf/xds/go v0.0.0-20211011173535-cb28da3451f1/go.mod h1:eXthEFrGJvWH github.com/cncf/xds/go v0.0.0-20220314180256-7f1daf1720fc/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cncf/xds/go v0.0.0-20230105202645-06c439db220b/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cncf/xds/go v0.0.0-20230310173818-32f1caf87195/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= +github.com/cncf/xds/go v0.0.0-20230607035331-e9ce68804cb4/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cpuguy83/go-md2man/v2 v2.0.2 h1:p1EgwI/C7NhT0JmVkwCD2ZBK8j4aeHQX2pMHHBfMQ6w= github.com/cpuguy83/go-md2man/v2 v2.0.2/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= @@ -828,6 +989,7 @@ github.com/dnaeon/go-vcr v1.2.0 h1:zHCHvJYTMh1N7xnV7zf1m1GPBF9Ad0Jk/whtQ1663qI= github.com/dnaeon/go-vcr v1.2.0/go.mod h1:R4UdLID7HZT3taECzJs4YgbbH6PIGXB6W/sc5OLb6RQ= github.com/docopt/docopt-go v0.0.0-20180111231733-ee0de3bc6815/go.mod h1:WwZ+bS3ebgob9U8Nd0kOddGdZWjyMGR8Wziv+TBNwSE= github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= +github.com/dustin/go-humanize v1.0.1/go.mod h1:Mu1zIs6XwVuF/gI1OepvI0qD18qycQx+mFykh5fBlto= github.com/dvsekhvalnov/jose2go v1.5.0 h1:3j8ya4Z4kMCwT5nXIKFSV84YS+HdqSSO0VsTQxaLAeM= github.com/dvsekhvalnov/jose2go v1.5.0/go.mod h1:QsHjhyTlD/lAVqn/NSbVZmSCGeDehTB/mPZadG+mhXU= github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= @@ -841,10 +1003,12 @@ github.com/envoyproxy/go-control-plane v0.9.10-0.20210907150352-cf90f659a021/go. github.com/envoyproxy/go-control-plane v0.10.2-0.20220325020618-49ff273808a1/go.mod h1:KJwIaB5Mv44NWtYuAOFCVOjcI94vtpEz2JU/D2v6IjE= github.com/envoyproxy/go-control-plane v0.10.3/go.mod h1:fJJn/j26vwOu972OllsvAgJJM//w9BV6Fxbg2LuVd34= github.com/envoyproxy/go-control-plane v0.11.0/go.mod h1:VnHyVMpzcLvCFt9yUz1UnCwHLhwx1WguiVDV7pTG/tI= +github.com/envoyproxy/go-control-plane v0.11.1-0.20230524094728-9239064ad72f/go.mod h1:sfYdkwUW4BA3PbKjySwjJy+O4Pu0h62rlqCMHNk+K+Q= github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/envoyproxy/protoc-gen-validate v0.6.7/go.mod h1:dyJXwwfPK2VSqiB9Klm1J6romD608Ba7Hij42vrOBCo= github.com/envoyproxy/protoc-gen-validate v0.9.1/go.mod h1:OKNgG7TCp5pF4d6XftA0++PMirau2/yoOwVac3AbF2w= github.com/envoyproxy/protoc-gen-validate v0.10.0/go.mod h1:DRjgyB0I43LtJapqN6NiRwroiAU2PaFuvk/vjgh61ss= +github.com/envoyproxy/protoc-gen-validate v0.10.1/go.mod h1:DRjgyB0I43LtJapqN6NiRwroiAU2PaFuvk/vjgh61ss= github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a h1:yDWHCSQ40h88yih2JAcL6Ls/kVkSE8GFACTGVnMPruw= github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a/go.mod h1:7Ga40egUymuWXxAe151lTNnCv97MddSOVsjpPPkityA= github.com/fogleman/gg v1.2.1-0.20190220221249-0403632d5b90/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= @@ -990,12 +1154,18 @@ github.com/google/pprof v0.0.0-20210601050228-01bbb1931b22/go.mod h1:kpwsk12EmLe github.com/google/pprof v0.0.0-20210609004039-a478d1d731e9/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20210720184732-4bb14d4b1be1/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= +github.com/google/s2a-go v0.1.0/go.mod h1:OJpEgntRZo8ugHpF9hkoLJbS5dSI20XZeXJ9JVywLlM= +github.com/google/s2a-go v0.1.3/go.mod h1:Ej+mSEMGRnqRzjc7VtF+jdBwYG5fuJfiZ8ELkjEwM0A= github.com/google/s2a-go v0.1.4 h1:1kZ/sQM3srePvKs3tXAvQzo66XfcReoqFpIpIccE7Oc= github.com/google/s2a-go v0.1.4/go.mod h1:Ej+mSEMGRnqRzjc7VtF+jdBwYG5fuJfiZ8ELkjEwM0A= +github.com/google/s2a-go v0.1.5 h1:8IYp3w9nysqv3JH+NJgXJzGbDHzLOTj43BmSkp+O7qg= +github.com/google/s2a-go v0.1.5/go.mod h1:Ej+mSEMGRnqRzjc7VtF+jdBwYG5fuJfiZ8ELkjEwM0A= github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I= github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/google/uuid v1.3.1 h1:KjJaJ9iWZ3jOFZIf1Lqf4laDRCasjl0BCmnEGxkdLb4= +github.com/google/uuid v1.3.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/enterprise-certificate-proxy v0.0.0-20220520183353-fd19c99a87aa/go.mod h1:17drOmN3MwGY7t0e+Ei9b45FFGA3fBs3x36SsCg1hq8= github.com/googleapis/enterprise-certificate-proxy v0.1.0/go.mod h1:17drOmN3MwGY7t0e+Ei9b45FFGA3fBs3x36SsCg1hq8= github.com/googleapis/enterprise-certificate-proxy v0.2.0/go.mod h1:8C0jb7/mgJe/9KK8Lm7X9ctZC2t60YyIpYEI16jx0Qg= @@ -1014,6 +1184,9 @@ github.com/googleapis/gax-go/v2 v2.5.1/go.mod h1:h6B0KMMFNtI2ddbGJn3T3ZbwkeT6yqE github.com/googleapis/gax-go/v2 v2.6.0/go.mod h1:1mjbznJAPHFpesgE5ucqfYEscaz5kMdcIDwU/6+DDoY= github.com/googleapis/gax-go/v2 v2.7.0/go.mod h1:TEop28CZZQ2y+c0VxMUmu1lV+fQx57QpBWsYpwqHJx8= github.com/googleapis/gax-go/v2 v2.7.1/go.mod h1:4orTrqY6hXxxaUL4LHIPl6lGo8vAE38/qKbhSAKP6QI= +github.com/googleapis/gax-go/v2 v2.8.0/go.mod h1:4orTrqY6hXxxaUL4LHIPl6lGo8vAE38/qKbhSAKP6QI= +github.com/googleapis/gax-go/v2 v2.10.0/go.mod h1:4UOEnMCrxsSqQ940WnTiD6qJ63le2ev3xfyagutxiPw= +github.com/googleapis/gax-go/v2 v2.11.0/go.mod h1:DxmR61SGKkGLa2xigwuZIQpkCI2S5iydzRfb3peWZJI= github.com/googleapis/gax-go/v2 v2.12.0 h1:A+gCJKdRfqXkr+BIRGtZLibNXf0m1f9E4HG56etFpas= github.com/googleapis/gax-go/v2 v2.12.0/go.mod h1:y+aIqrI5eb1YGMVJfuV3185Ts/D7qKpsEkdD5+I6QGU= github.com/googleapis/go-type-adapters v1.0.0/go.mod h1:zHW75FOG2aur7gAO2B+MLby+cLsWGBF62rFAi7WjWO4= @@ -1023,6 +1196,7 @@ github.com/gorilla/sessions v1.2.1/go.mod h1:dk2InVEVJ0sfLlnXv9EAgkf6ecYs/i80K/z github.com/grpc-ecosystem/go-grpc-middleware v1.3.0/go.mod h1:z0ButlSOZa5vEBq9m2m2hlwIgKw+rp3sdCBRoJY+30Y= github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= github.com/grpc-ecosystem/go-grpc-middleware v1.4.0/go.mod h1:g5qyo/la0ALbONm6Vbp88Yd8NsDy6rZz+RcrMPxvld8= +github.com/grpc-ecosystem/grpc-gateway v1.16.0 h1:gmcG1KaJ57LophUzW0Hy8NmPhnMZb4M0+kPpLofRdBo= github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw= github.com/grpc-ecosystem/grpc-gateway/v2 v2.7.0/go.mod h1:hgWBS7lorOAVIJEQMi4ZsPv9hVvWI6+ch50m39Pf2Ks= github.com/grpc-ecosystem/grpc-gateway/v2 v2.11.3/go.mod h1:o//XUCC/F+yRGJoPO/VU0GSB0f8Nhgmxx0VIRUvaC0w= @@ -1042,23 +1216,19 @@ github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1: github.com/ianlancetaylor/demangle v0.0.0-20200824232613-28f6c0f3b639/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= github.com/jackc/pgio v1.0.0 h1:g12B9UwVnzGhueNavwioyEEpAmqMe1E/BN9ES+8ovkE= github.com/jackc/pgio v1.0.0/go.mod h1:oP+2QK2wFfUWgr+gxjoBH9KGBb31Eio69xUb0w5bYf8= -github.com/jackc/pglogrepl v0.0.0-20230630212501-5fd22a600b50 h1:88/G11oNDrFAk2kZzxLDUE1jiYkFfVYHxUyWen7Ro5c= -github.com/jackc/pglogrepl v0.0.0-20230630212501-5fd22a600b50/go.mod h1:Y1HIk+uK2wXiU8vuvQh0GaSzVh+MXFn2kfKBMpn6CZg= github.com/jackc/pglogrepl v0.0.0-20230728225306-38e8a4e50913 h1:n35fv0W1w8e7x68zjhoQx8W+N6tEhaImAErPypDQ7ik= github.com/jackc/pglogrepl v0.0.0-20230728225306-38e8a4e50913/go.mod h1:Y1HIk+uK2wXiU8vuvQh0GaSzVh+MXFn2kfKBMpn6CZg= +github.com/jackc/pglogrepl v0.0.0-20230810221841-d0818e1fbef7 h1:gpfct0XvEOnv5N1sbTo2KnWC2VwrWKJBOqcxtAUEi3g= +github.com/jackc/pglogrepl v0.0.0-20230810221841-d0818e1fbef7/go.mod h1:Y1HIk+uK2wXiU8vuvQh0GaSzVh+MXFn2kfKBMpn6CZg= github.com/jackc/pgpassfile v1.0.0 h1:/6Hmqy13Ss2zCq62VdNG8tM1wchn8zjSGOBJ6icpsIM= github.com/jackc/pgpassfile v1.0.0/go.mod h1:CEx0iS5ambNFdcRtxPj5JhEz+xB6uRky5eyVu/W2HEg= github.com/jackc/pgservicefile v0.0.0-20200714003250-2b9c44734f2b/go.mod h1:vsD4gTJCa9TptPL8sPkXrLZ+hDuNrZCnj29CQpr4X1E= github.com/jackc/pgservicefile v0.0.0-20221227161230-091c0ba34f0a h1:bbPeKD0xmW/Y25WS6cokEszi5g+S0QxI/d45PkRi7Nk= github.com/jackc/pgservicefile v0.0.0-20221227161230-091c0ba34f0a/go.mod h1:5TJZWKEWniPve33vlWYSoGYefn3gLQRzjfDlhSJ9ZKM= github.com/jackc/pgx/v5 v5.0.3/go.mod h1:JBbvW3Hdw77jKl9uJrEDATUZIFM2VFPzRq4RWIhkF4o= -github.com/jackc/pgx/v5 v5.4.2 h1:u1gmGDwbdRUZiwisBm/Ky2M14uQyUP65bG8+20nnyrg= -github.com/jackc/pgx/v5 v5.4.2/go.mod h1:q6iHT8uDNXWiFNOlRqJzBTaSH3+2xCXkokxHZC5qWFY= github.com/jackc/pgx/v5 v5.4.3 h1:cxFyXhxlvAifxnkKKdlxv8XqUf59tDlYjnV5YYfsJJY= github.com/jackc/pgx/v5 v5.4.3/go.mod h1:Ig06C2Vu0t5qXC60W8sqIthScaEnFvojjj9dSljmHRA= github.com/jackc/puddle/v2 v2.0.0/go.mod h1:itE7ZJY8xnoo0JqJEpSMprN0f+NQkMCuEV/N9j8h0oc= -github.com/jackc/puddle/v2 v2.2.0 h1:RdcDk92EJBuBS55nQMMYFXTxwstHug4jkhT5pq8VxPk= -github.com/jackc/puddle/v2 v2.2.0/go.mod h1:vriiEXHvEE654aYKXXjOvZM39qJ0q+azkZFrfEOc3H4= github.com/jackc/puddle/v2 v2.2.1 h1:RhxXJtFG022u4ibrCSMSiu5aOq1i77R3OHKNJj77OAk= github.com/jackc/puddle/v2 v2.2.1/go.mod h1:vriiEXHvEE654aYKXXjOvZM39qJ0q+azkZFrfEOc3H4= github.com/jcmturner/aescts/v2 v2.0.0/go.mod h1:AiaICIRyfYg35RUkr8yESTqvSy7csK90qZ5xfvvsoNs= @@ -1123,14 +1293,15 @@ github.com/lyft/protoc-gen-star v0.6.1/go.mod h1:TGAoBVkt8w7MPG72TrKIu85MIdXwDuz github.com/lyft/protoc-gen-star/v2 v2.0.1/go.mod h1:RcCdONR2ScXaYnQC5tUzxzlpA3WVYF7/opLeUgcQs/o= github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU= github.com/mattn/go-isatty v0.0.16/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/yFXSvRLM= +github.com/mattn/go-isatty v0.0.17/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/yFXSvRLM= github.com/mattn/go-sqlite3 v1.14.6/go.mod h1:NyWgC/yNuGj7Q9rpYnZvas74GogHl5/Z4A/KQRfk6bU= github.com/mattn/go-sqlite3 v1.14.14 h1:qZgc/Rwetq+MtyE18WhzjokPD93dNqLGNT3QJuLvBGw= github.com/mattn/go-sqlite3 v1.14.14/go.mod h1:NyWgC/yNuGj7Q9rpYnZvas74GogHl5/Z4A/KQRfk6bU= +github.com/mattn/go-sqlite3 v1.14.15 h1:vfoHhTN1af61xCRSWzFIWzx2YskyMTwHLrExkBOjvxI= +github.com/mattn/go-sqlite3 v1.14.15/go.mod h1:2eHXhiwb8IkHr+BDWZGa96P6+rkvnG63S2DGjv9HUNg= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= github.com/matttproud/golang_protobuf_extensions v1.0.4 h1:mmDVorXM7PCGKw94cs5zkfA9PSy5pEvNWRP0ET0TIVo= github.com/matttproud/golang_protobuf_extensions v1.0.4/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4= -github.com/microsoft/go-mssqldb v1.3.0 h1:JcPVl+acL8Z/cQcJc9zP0OkjQ+l20bco/cCDpMbmGJk= -github.com/microsoft/go-mssqldb v1.3.0/go.mod h1:lmWsjHD8XX/Txr0f8ZqgbEZSC+BZjmEQy/Ms+rLrvho= github.com/microsoft/go-mssqldb v1.5.0 h1:CgENxkwtOBNj3Jg6T1X209y2blCfTTcwuOlznd2k9fk= github.com/microsoft/go-mssqldb v1.5.0/go.mod h1:lmWsjHD8XX/Txr0f8ZqgbEZSC+BZjmEQy/Ms+rLrvho= github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8 h1:AMFGa4R4MiIpspGNG7Z948v4n35fFGB3RR3G/ry4FWs= @@ -1154,8 +1325,6 @@ github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRW github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= -github.com/orcaman/concurrent-map/v2 v2.0.1 h1:jOJ5Pg2w1oeB6PeDurIYf6k9PQ+aTITr/6lP/L/zp6c= -github.com/orcaman/concurrent-map/v2 v2.0.1/go.mod h1:9Eq3TG2oBe5FirmYWQfYO5iH1q0Jv47PLaNK++uCdOM= github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= github.com/phpdave11/gofpdf v1.4.2/go.mod h1:zpO6xFn9yxo3YLyMvW8HcKWVdbNqgIfOOp2dXMnm1mY= @@ -1175,6 +1344,7 @@ github.com/pkg/sftp v1.10.1/go.mod h1:lYOWFsE0bwd1+KfKJaKeuokY15vzFx25BLbzYYoAxZ github.com/pkg/sftp v1.13.1/go.mod h1:3HaPG6Dq1ILlpPZRO0HVMrsydcdLt6HRDccSgb87qRg= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/prashantv/gostub v1.1.0 h1:BTyx3RfQjRHnUWaGF9oQos79AlQ5k8WNktv7VGvVH4g= github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M= @@ -1185,32 +1355,27 @@ github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1: github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.3.0 h1:UBgGFHqYdG/TPFD1B1ogZywDqEkwp3fBMvqdiQ7Xew4= github.com/prometheus/client_model v0.3.0/go.mod h1:LDGWKZIo7rky3hgvBe+caln+Dr3dPggB5dvjtD7w9+w= github.com/prometheus/client_model v0.4.0 h1:5lQXD3cAg1OXBf4Wq03gTrXHeaV0TQvGfUooCfx1yqY= github.com/prometheus/client_model v0.4.0/go.mod h1:oMQmHW1/JoDwqLtg57MGgP/Fb1CJEYF2imWWhWtMkYU= github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= github.com/prometheus/common v0.26.0/go.mod h1:M7rCNAaPfAosfx8veZJCuw84e35h3Cfd9VFqTh1DIvc= -github.com/prometheus/common v0.42.0 h1:EKsfXEYo4JpWMHH5cg+KOUWeuJSov1Id8zGR8eeI1YM= -github.com/prometheus/common v0.42.0/go.mod h1:xBwqVerjNdUDjgODMpudtOMwlOwf2SaTr1yjz4b7Zbc= github.com/prometheus/common v0.44.0 h1:+5BrQJwiBB9xsMygAB3TNvpQKOwlkc25LbISbrdOOfY= github.com/prometheus/common v0.44.0/go.mod h1:ofAIvZbQ1e/nugmZGz4/qCb9Ap1VoSTIO7x0VV9VvuY= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= -github.com/prometheus/procfs v0.10.1 h1:kYK1Va/YMlutzCGazswoHKo//tZVlFpKYh+PymziUAg= -github.com/prometheus/procfs v0.10.1/go.mod h1:nwNm2aOCAYw8uTR/9bWRREkZFxAUcWzPHWJq+XBB/FM= github.com/prometheus/procfs v0.11.1 h1:xRC8Iq1yyca5ypa9n1EZnWZkt7dwcoRPQwX/5gwaUuI= github.com/prometheus/procfs v0.11.1/go.mod h1:eesXgaPo1q7lBpVMoMy0ZOFTth9hBn4W/y0/p/ScXhY= github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= +github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= github.com/robfig/cron v1.2.0 h1:ZjScXvvxeQ63Dbyxy76Fj3AT3Ut0aKsyd2/tl3DTMuQ= github.com/robfig/cron v1.2.0/go.mod h1:JGuDeoQd7Z6yL4zQhZ3OPEVHB7fL6Ka6skscFHfmt2k= github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTEfhy4qGm1nDQc= -github.com/rogpeppe/go-internal v1.9.0 h1:73kH8U+JUqXU8lRuOHeVHaa/SZPifC7BkcraZVejAe8= github.com/rogpeppe/go-internal v1.9.0/go.mod h1:WtVeX8xhTBvf0smdhujwtBcq4Qrzq/fJaraNFVN+nFs= github.com/rogpeppe/go-internal v1.10.0 h1:TMyTOH3F/DB16zRVcYyreMH6GnZZrwQVAoYjRBZyWFQ= github.com/russross/blackfriday/v2 v2.1.0 h1:JIOH55/0cWyOuilr9/qlrm0BSXldqnqwMsf35Ld67mk= @@ -1222,10 +1387,10 @@ github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6Mwd github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= github.com/sirupsen/logrus v1.9.3 h1:dueUQJ1C2q9oE3F7wvmSGAaVtTmUizReu6fjN8uqzbQ= github.com/sirupsen/logrus v1.9.3/go.mod h1:naHLuLoDiP4jHNo9R0sCBMtWGeIprob74mVsIT4qYEQ= -github.com/snowflakedb/gosnowflake v1.6.22 h1:2crLpqmFVyV03NPAxxAtzQBMFn6wUPqOJ1uRl4ruOJ4= -github.com/snowflakedb/gosnowflake v1.6.22/go.mod h1:P2fE/xiD2kQXpr48OdgnazkzPsKD6aVtnHD3WP8yD9c= github.com/snowflakedb/gosnowflake v1.6.23 h1:uO+zMTXJcSHzOm6ks5To8ergNjt5Dy6cr5QtStpRFT8= github.com/snowflakedb/gosnowflake v1.6.23/go.mod h1:KfO4F7bk+aXPUIvBqYxvPhxLlu2/w4TtSC8Rw/yr5Mg= +github.com/snowflakedb/gosnowflake v1.6.24 h1:NiBh1WSstNtr12qywmdFMS1XHaYdF5iWWGnjIQb1cEY= +github.com/snowflakedb/gosnowflake v1.6.24/go.mod h1:KfO4F7bk+aXPUIvBqYxvPhxLlu2/w4TtSC8Rw/yr5Mg= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= github.com/spf13/afero v1.3.3/go.mod h1:5KUK8ByomD5Ti5Artl0RtHeI5pTF7MIDuXL3yY520V4= github.com/spf13/afero v1.6.0/go.mod h1:Ai8FlHk4v/PARR026UzYexafAt9roJ7LcLMAmO6Z93I= @@ -1236,6 +1401,8 @@ github.com/stretchr/objx v0.3.0/go.mod h1:qt09Ya8vawLte6SNmTgCsAVtYtaKzEcn8ATUoH github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= github.com/stretchr/objx v0.5.0 h1:1zr/of2m5FGMsad5YfcqgdqdWrIhu+EBEJRhR1U7z/c= github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo= +github.com/stretchr/objx v0.5.1 h1:4VhoImhV/Bm0ToFkXFi8hXNXwpDRZ/ynw3amt82mzq0= +github.com/stretchr/objx v0.5.1/go.mod h1:/iHQpkQwBD6DLUmQ4pE+s1TXdob1mORJ4/UFdrifcy0= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= @@ -1250,7 +1417,6 @@ github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o github.com/stretchr/testify v1.8.3/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= -github.com/twmb/murmur3 v1.1.5 h1:i9OLS9fkuLzBXjt6dptlAEyk58fJsSTXbRg3SgVyqgk= github.com/twmb/murmur3 v1.1.5/go.mod h1:Qq/R7NUyOfr65zD+6Q5IHKsJLwP7exErjN6lyyq3OSQ= github.com/twmb/murmur3 v1.1.8 h1:8Yt9taO/WN3l08xErzjeschgZU2QSrwm1kclYq+0aRg= github.com/twmb/murmur3 v1.1.8/go.mod h1:Qq/R7NUyOfr65zD+6Q5IHKsJLwP7exErjN6lyyq3OSQ= @@ -1288,9 +1454,9 @@ go.temporal.io/api v1.5.0/go.mod h1:BqKxEJJYdxb5dqf0ODfzfMxh8UEQ5L3zKS51FiIYYkA= go.temporal.io/api v1.21.0/go.mod h1:xlsUEakkN2vU2/WV7e5NqMG4N93nfuNfvbXdaXUpU8w= go.temporal.io/api v1.23.0 h1:4y9mTQjEHsE0Du0WJ2ExJUcP/1/a+B/UefzIDm4ALTE= go.temporal.io/api v1.23.0/go.mod h1:AcJd1+rc1j0zte+ZBIkOHGHjntR/17LnZWFz+gMFHQ0= +go.temporal.io/api v1.24.0 h1:WWjMYSXNh4+T4Y4jq1e/d9yCNnWoHhq4bIwflHY6fic= +go.temporal.io/api v1.24.0/go.mod h1:4ackgCMjQHMpJYr1UQ6Tr/nknIqFkJ6dZ/SZsGv+St0= go.temporal.io/sdk v1.12.0/go.mod h1:lSp3lH1lI0TyOsus0arnO3FYvjVXBZGi/G7DjnAnm6o= -go.temporal.io/sdk v1.23.1 h1:HzOaw5+f6QgDW/HH1jzwgupII7nVz+fzxFPjmFJqKiQ= -go.temporal.io/sdk v1.23.1/go.mod h1:S7vWxU01lGcCny0sWx03bkkYw4VtVrpzeqBTn2A6y+E= go.temporal.io/sdk v1.24.0 h1:mAk5VFR+z4s8QVzRx3iIpRnHcEO3m10CYNjnRXrhVq4= go.temporal.io/sdk v1.24.0/go.mod h1:S7vWxU01lGcCny0sWx03bkkYw4VtVrpzeqBTn2A6y+E= go.temporal.io/sdk/contrib/tally v0.2.0 h1:XnTJIQcjOv+WuCJ1u8Ve2nq+s2H4i/fys34MnWDRrOo= @@ -1323,11 +1489,10 @@ golang.org/x/crypto v0.0.0-20211215153901-e495a2d5b3d3/go.mod h1:IxCIyHEi3zRg3s0 golang.org/x/crypto v0.0.0-20220314234659-1baeb1ce4c0b/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.0.0-20220722155217-630584e8d5aa/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.0.0-20220829220503-c86fa9a7ed90/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= +golang.org/x/crypto v0.1.0/go.mod h1:RecgLatLF4+eUMCP1PoPZQb+cVrJcOPbHkTkbkB9sbw= golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= golang.org/x/crypto v0.7.0/go.mod h1:pYwdfH91IfpZVANVyUOhSIPZaFoJGxTFbZhFTx+dXZU= golang.org/x/crypto v0.9.0/go.mod h1:yrmDGqONDYtNj3tH8X9dzUun2m2lzPa9ngI6/RUPGR0= -golang.org/x/crypto v0.11.0 h1:6Ewdq3tDic1mg5xRO4milcWCfMVQhI4NkqWWvqejpuA= -golang.org/x/crypto v0.11.0/go.mod h1:xgJhtzW8F9jGdVFWZESrid1U1bjeNy4zgy5cRr/CIio= golang.org/x/crypto v0.12.0 h1:tFM/ta59kqch6LlvYnPa0yx5a83cL2nHflFhYKvv9Yk= golang.org/x/crypto v0.12.0/go.mod h1:NF0Gs7EO5K4qLn+Ylc+fih8BSTeIjAP05siRnAh98yw= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -1345,10 +1510,10 @@ golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u0 golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM= golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= golang.org/x/exp v0.0.0-20220827204233-334a2380cb91/go.mod h1:cyybsKvd6eL0RnXn6p/Grxp8F5bW7iYuBgsNCOHpMYE= -golang.org/x/exp v0.0.0-20230713183714-613f0c0eb8a1 h1:MGwJjxBy0HJshjDNfLsYO8xppfqWlA5ZT9OhtUUhTNw= -golang.org/x/exp v0.0.0-20230713183714-613f0c0eb8a1/go.mod h1:FXUEEKJgO7OQYeo8N01OfiKP8RXMtf6e8aTskBGqWdc= golang.org/x/exp v0.0.0-20230801115018-d63ba01acd4b h1:r+vk0EmXNmekl0S0BascoeeoHk/L7wmaW2QF90K+kYI= golang.org/x/exp v0.0.0-20230801115018-d63ba01acd4b/go.mod h1:FXUEEKJgO7OQYeo8N01OfiKP8RXMtf6e8aTskBGqWdc= +golang.org/x/exp v0.0.0-20230817173708-d852ddb80c63 h1:m64FZMko/V45gv0bNmrNYoDEq8U5YUhetc9cBWKS1TQ= +golang.org/x/exp v0.0.0-20230817173708-d852ddb80c63/go.mod h1:0v4NqG35kSWCMzLaMeX+IQrlSnVE/bqGSyC2cz/9Le8= golang.org/x/image v0.0.0-20180708004352-c73c2afc3b81/go.mod h1:ux5Hcp/YLpHSI86hEcLt0YII63i6oz57MZXIpbrjZUs= golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= @@ -1391,6 +1556,7 @@ golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91 golang.org/x/mod v0.7.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.9.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= +golang.org/x/mod v0.10.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.12.0 h1:rmsUpXtvNzj340zd98LZ4KntptpfRHwpFOHG188oHXc= golang.org/x/mod v0.12.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -1454,9 +1620,8 @@ golang.org/x/net v0.5.0/go.mod h1:DivGGAXEgPSlEBzxGzZI+ZLohi+xUj054jfeKui00ws= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.8.0/go.mod h1:QVkue5JL9kW//ek3r6jTKnTFis1tRmNAW2P1shuFdJc= +golang.org/x/net v0.9.0/go.mod h1:d48xBJpPfHeWQsugry2m+kC02ZBRGRgulfHnEXEuWns= golang.org/x/net v0.10.0/go.mod h1:0qNGK6F8kojg2nk9dLZ2mShWaEBan6FAoqfSigmmuDg= -golang.org/x/net v0.12.0 h1:cfawfvKITfUsFCeJIHJrbSxpeu/E81khclypR0GVT50= -golang.org/x/net v0.12.0/go.mod h1:zEVYFnQC7m/vmpQFELhcD1EWkZlX69l4oqgmer6hfKA= golang.org/x/net v0.14.0 h1:BONx9s002vGdD9umnlX1Po8vOZmrgH34qlHcD1MfK14= golang.org/x/net v0.14.0/go.mod h1:PpSgVXXLK0OxS0F31C1/tv6XNguvCrnXIDrFMspZIUI= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= @@ -1487,8 +1652,8 @@ golang.org/x/oauth2 v0.0.0-20221014153046-6fdb5e3db783/go.mod h1:h4gKUeWbJ4rQPri golang.org/x/oauth2 v0.4.0/go.mod h1:RznEsdpjGAINPTOF0UH/t+xJ75L18YO3Ho6Pyn+uRec= golang.org/x/oauth2 v0.5.0/go.mod h1:9/XBHVqLaWO3/BRHs5jbpYCnOZVjj5V0ndyaAM7KB4I= golang.org/x/oauth2 v0.6.0/go.mod h1:ycmewcwgD4Rpr3eZJLSB4Kyyljb3qDh40vJ8STE5HKw= -golang.org/x/oauth2 v0.10.0 h1:zHCpF2Khkwy4mMB4bv0U37YtJdTGW8jI0glAApi0Kh8= -golang.org/x/oauth2 v0.10.0/go.mod h1:kTpgurOux7LqtuxjuyZa4Gj2gdezIt/jQtGnNFfypQI= +golang.org/x/oauth2 v0.7.0/go.mod h1:hPLQkd9LyjfXTiRohC/41GhcFqxisoUQ99sCUOHO9x4= +golang.org/x/oauth2 v0.8.0/go.mod h1:yr7u4HXZRm1R1kBWqr/xKNqewf0plRYoB7sla+BCIXE= golang.org/x/oauth2 v0.11.0 h1:vPL4xzxBM4niKCW6g9whtaWVXTJf1U5e4aZxxFx/gbU= golang.org/x/oauth2 v0.11.0/go.mod h1:LdF7O/8bLR/qWK9DrpXmbHLTouvRHK0SgJl0GmDBchk= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -1507,6 +1672,7 @@ golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20220819030929-7fc1605a5dde/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220929204114-8fcdb60fdcc0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.2.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.3.0 h1:ftCYgMx6zT/asHUrPw8BLLscYtGznsLAnjq5RH9P66E= golang.org/x/sync v0.3.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -1598,9 +1764,8 @@ golang.org/x/sys v0.3.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.4.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.7.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.10.0 h1:SqMFp9UcQJZa+pmYuAKjd9xq1f0j5rLcDIk0mj4qAsA= -golang.org/x/sys v0.10.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0 h1:eG7RXZHdqOJ1i+0lgLgCpSXAp6M3LYlAo6osgSi0xOM= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= @@ -1611,9 +1776,8 @@ golang.org/x/term v0.3.0/go.mod h1:q750SLmJuPmVoN1blW3UFBPREJfb1KmY3vwxfr+nFDA= golang.org/x/term v0.4.0/go.mod h1:9P2UbLfCdcvo3p/nzKvsmas4TnlujnuoV9hGgYzW1lQ= golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= golang.org/x/term v0.6.0/go.mod h1:m6U89DPEgQRMq3DNkDClhWw02AUbt2daBVO4cn4Hv9U= +golang.org/x/term v0.7.0/go.mod h1:P32HKFT3hSsZrRxla30E9HqToFYAQPCMs/zFMBUFqPY= golang.org/x/term v0.8.0/go.mod h1:xPskH00ivmX89bAKVGSKKtLOWNx2+17Eiy94tnKShWo= -golang.org/x/term v0.10.0 h1:3R7pNqamzBraeqj/Tj8qt1aQ2HpmlC+Cx/qL/7hn4/c= -golang.org/x/term v0.10.0/go.mod h1:lpqdcUyK/oCiQxvxVrppt5ggO2KCZ5QblwqPnfZ6d5o= golang.org/x/term v0.11.0 h1:F9tnn/DA/Im8nCwm+fX+1/eBwi4qFjRT++MhtVC4ZX0= golang.org/x/term v0.11.0/go.mod h1:zC9APTIj3jG3FdV/Ons+XE1riIZXG4aZ4GTHiPZJPIU= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -1632,8 +1796,6 @@ golang.org/x/text v0.6.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.8.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= golang.org/x/text v0.9.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= -golang.org/x/text v0.11.0 h1:LAntKIrcmeSKERyiOh0XMV39LXS8IE9UL2yP7+f5ij4= -golang.org/x/text v0.11.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= golang.org/x/text v0.12.0 h1:k+n5B8goJNdU7hSvEtMUz3d1Q6D/XW4COJSJR6fN0mc= golang.org/x/text v0.12.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -1708,10 +1870,11 @@ golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc golang.org/x/tools v0.3.0/go.mod h1:/rWhSS2+zyEVwoJf8YAX6L2f0ntZ7Kn/mGgAWcipA5k= golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU= golang.org/x/tools v0.7.0/go.mod h1:4pg6aUX35JBAogB10C9AtvVL+qowtN4pT3CGSQex14s= -golang.org/x/tools v0.11.0 h1:EMCa6U9S2LtZXLAMoWiR/R8dAQFRqbAitmbJ2UKhoi8= -golang.org/x/tools v0.11.0/go.mod h1:anzJrxPjNtfgiYQYirP2CPGzGLxrH2u2QBhn6Bf3qY8= +golang.org/x/tools v0.9.1/go.mod h1:owI94Op576fPu3cIGQeHs3joujW/2Oc6MtlxbF5dfNc= golang.org/x/tools v0.11.1 h1:ojD5zOW8+7dOGzdnNgersm8aPfcDjhMp12UfG93NIMc= golang.org/x/tools v0.11.1/go.mod h1:anzJrxPjNtfgiYQYirP2CPGzGLxrH2u2QBhn6Bf3qY8= +golang.org/x/tools v0.12.1-0.20230815132531-74c255bcf846 h1:Vve/L0v7CXXuxUmaMGIEK/dEeq7uiqb5qBgQrZzIE7E= +golang.org/x/tools v0.12.1-0.20230815132531-74c255bcf846/go.mod h1:Sc0INKfu04TlqNoRA1hgpFZbhYXHPr4V5DzpSBTPqQM= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -1787,10 +1950,15 @@ google.golang.org/api v0.108.0/go.mod h1:2Ts0XTHNVWxypznxWOYUeI4g3WdP9Pk2Qk58+a/ google.golang.org/api v0.110.0/go.mod h1:7FC4Vvx1Mooxh8C5HWjzZHcavuS2f6pmJpZx60ca7iI= google.golang.org/api v0.111.0/go.mod h1:qtFHvU9mhgTJegR31csQ+rwxyUTHOKFqCKWp1J0fdw0= google.golang.org/api v0.114.0/go.mod h1:ifYI2ZsFK6/uGddGfAD5BMxlnkBqCmqHSDUVi45N5Yg= -google.golang.org/api v0.131.0 h1:AcgWS2edQ4chVEt/SxgDKubVu/9/idCJy00tBGuGB4M= -google.golang.org/api v0.131.0/go.mod h1:7vtkbKv2REjJbxmHSkBTBQ5LUGvPdAqjjvt84XAfhpA= +google.golang.org/api v0.118.0/go.mod h1:76TtD3vkgmZ66zZzp72bUUklpmQmKlhh6sYtIjYK+5E= +google.golang.org/api v0.122.0/go.mod h1:gcitW0lvnyWjSp9nKxAbdHKIZ6vF4aajGueeslZOyms= +google.golang.org/api v0.124.0/go.mod h1:xu2HQurE5gi/3t1aFCvhPD781p0a3p11sdunTJ2BlP4= +google.golang.org/api v0.125.0/go.mod h1:mBwVAtz+87bEN6CbA1GtZPDOqY2R5ONPqJeIlvyo4Aw= +google.golang.org/api v0.126.0/go.mod h1:mBwVAtz+87bEN6CbA1GtZPDOqY2R5ONPqJeIlvyo4Aw= google.golang.org/api v0.134.0 h1:ktL4Goua+UBgoP1eL1/60LwZJqa1sIzkLmvoR3hR6Gw= google.golang.org/api v0.134.0/go.mod h1:sjRL3UnjTx5UqNQS9EWr9N8p7xbHpy1k0XGRLCf3Spk= +google.golang.org/api v0.138.0 h1:K/tVp05MxNVbHShRw9m7e9VJGdagNeTdMzqPH7AUqr0= +google.golang.org/api v0.138.0/go.mod h1:4xyob8CxC+0GChNBvEUAk8VBKNvYOTWM9T3v3UfRxuY= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= @@ -1932,23 +2100,45 @@ google.golang.org/genproto v0.0.0-20230320184635-7606e756e683/go.mod h1:NWraEVix google.golang.org/genproto v0.0.0-20230323212658-478b75c54725/go.mod h1:UUQDJDOlWu4KYeJZffbWgBkS1YFobzKbLVfK69pe0Ak= google.golang.org/genproto v0.0.0-20230330154414-c0448cd141ea/go.mod h1:UUQDJDOlWu4KYeJZffbWgBkS1YFobzKbLVfK69pe0Ak= google.golang.org/genproto v0.0.0-20230331144136-dcfb400f0633/go.mod h1:UUQDJDOlWu4KYeJZffbWgBkS1YFobzKbLVfK69pe0Ak= +google.golang.org/genproto v0.0.0-20230403163135-c38d8f061ccd/go.mod h1:UUQDJDOlWu4KYeJZffbWgBkS1YFobzKbLVfK69pe0Ak= +google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1/go.mod h1:nKE/iIaLqn2bQwXBg8f1g2Ylh6r5MN5CmZvuzZCgsCU= google.golang.org/genproto v0.0.0-20230525154841-bd750badd5c6/go.mod h1:nKE/iIaLqn2bQwXBg8f1g2Ylh6r5MN5CmZvuzZCgsCU= google.golang.org/genproto v0.0.0-20230525234025-438c736192d0/go.mod h1:9ExIQyXL5hZrHzQceCwuSYwZZ5QZBazOcprJ5rgs3lY= -google.golang.org/genproto v0.0.0-20230711160842-782d3b101e98 h1:Z0hjGZePRE0ZBWotvtrwxFNrNE9CUAGtplaDK5NNI/g= -google.golang.org/genproto v0.0.0-20230711160842-782d3b101e98/go.mod h1:S7mY02OqCJTD0E1OiQy1F72PWFB4bZJ87cAtLPYgDR0= +google.golang.org/genproto v0.0.0-20230526161137-0005af68ea54/go.mod h1:zqTuNwFlFRsw5zIts5VnzLQxSRqh+CGOTVMlYbY0Eyk= +google.golang.org/genproto v0.0.0-20230530153820-e85fd2cbaebc/go.mod h1:xZnkP7mREFX5MORlOPEzLMr+90PPZQ2QWzrVTWfAq64= +google.golang.org/genproto v0.0.0-20230629202037-9506855d4529/go.mod h1:xZnkP7mREFX5MORlOPEzLMr+90PPZQ2QWzrVTWfAq64= +google.golang.org/genproto v0.0.0-20230706204954-ccb25ca9f130/go.mod h1:O9kGHb51iE/nOGvQaDUuadVYqovW56s5emA88lQnj6Y= +google.golang.org/genproto v0.0.0-20230726155614-23370e0ffb3e/go.mod h1:0ggbjUrZYpy1q+ANUS30SEoGZ53cdfwtbuG7Ptgy108= google.golang.org/genproto v0.0.0-20230803162519-f966b187b2e5 h1:L6iMMGrtzgHsWofoFcihmDEMYeDR9KN/ThbPWGrh++g= google.golang.org/genproto v0.0.0-20230803162519-f966b187b2e5/go.mod h1:oH/ZOT02u4kWEp7oYBGYFFkCdKS/uYR9Z7+0/xuuFp8= +google.golang.org/genproto v0.0.0-20230815205213-6bfd019c3878/go.mod h1:yZTlhN0tQnXo3h00fuXNCxJdLdIdnVFVBaRJ5LWBbw4= +google.golang.org/genproto v0.0.0-20230822172742-b8732ec3820d h1:VBu5YqKPv6XiJ199exd8Br+Aetz+o08F+PLMnwJQHAY= +google.golang.org/genproto v0.0.0-20230822172742-b8732ec3820d/go.mod h1:yZTlhN0tQnXo3h00fuXNCxJdLdIdnVFVBaRJ5LWBbw4= google.golang.org/genproto/googleapis/api v0.0.0-20230525234020-1aefcd67740a/go.mod h1:ts19tUU+Z0ZShN1y3aPyq2+O3d5FUNNgT6FtOzmrNn8= -google.golang.org/genproto/googleapis/api v0.0.0-20230711160842-782d3b101e98 h1:FmF5cCW94Ij59cfpoLiwTgodWmm60eEV0CjlsVg2fuw= -google.golang.org/genproto/googleapis/api v0.0.0-20230711160842-782d3b101e98/go.mod h1:rsr7RhLuwsDKL7RmgDDCUc6yaGr1iqceVb5Wv6f6YvQ= +google.golang.org/genproto/googleapis/api v0.0.0-20230525234035-dd9d682886f9/go.mod h1:vHYtlOoi6TsQ3Uk2yxR7NI5z8uoV+3pZtR4jmHIkRig= +google.golang.org/genproto/googleapis/api v0.0.0-20230526203410-71b5a4ffd15e/go.mod h1:vHYtlOoi6TsQ3Uk2yxR7NI5z8uoV+3pZtR4jmHIkRig= +google.golang.org/genproto/googleapis/api v0.0.0-20230530153820-e85fd2cbaebc/go.mod h1:vHYtlOoi6TsQ3Uk2yxR7NI5z8uoV+3pZtR4jmHIkRig= +google.golang.org/genproto/googleapis/api v0.0.0-20230629202037-9506855d4529/go.mod h1:vHYtlOoi6TsQ3Uk2yxR7NI5z8uoV+3pZtR4jmHIkRig= +google.golang.org/genproto/googleapis/api v0.0.0-20230706204954-ccb25ca9f130/go.mod h1:mPBs5jNgx2GuQGvFwUvVKqtn6HsUw9nP64BedgvqEsQ= +google.golang.org/genproto/googleapis/api v0.0.0-20230726155614-23370e0ffb3e/go.mod h1:rsr7RhLuwsDKL7RmgDDCUc6yaGr1iqceVb5Wv6f6YvQ= google.golang.org/genproto/googleapis/api v0.0.0-20230803162519-f966b187b2e5 h1:nIgk/EEq3/YlnmVVXVnm14rC2oxgs1o0ong4sD/rd44= google.golang.org/genproto/googleapis/api v0.0.0-20230803162519-f966b187b2e5/go.mod h1:5DZzOUPCLYL3mNkQ0ms0F3EuUNZ7py1Bqeq6sxzI7/Q= +google.golang.org/genproto/googleapis/api v0.0.0-20230815205213-6bfd019c3878/go.mod h1:KjSP20unUpOx5kyQUFa7k4OJg0qeJ7DEZflGDu2p6Bk= +google.golang.org/genproto/googleapis/api v0.0.0-20230822172742-b8732ec3820d h1:DoPTO70H+bcDXcd39vOqb2viZxgqeBeSGtZ55yZU4/Q= +google.golang.org/genproto/googleapis/api v0.0.0-20230822172742-b8732ec3820d/go.mod h1:KjSP20unUpOx5kyQUFa7k4OJg0qeJ7DEZflGDu2p6Bk= +google.golang.org/genproto/googleapis/bytestream v0.0.0-20230530153820-e85fd2cbaebc/go.mod h1:ylj+BE99M198VPbBh6A8d9n3w8fChvyLK3wwBOjXBFA= google.golang.org/genproto/googleapis/rpc v0.0.0-20230525234015-3fc162c6f38a/go.mod h1:xURIpW9ES5+/GZhnV6beoEtxQrnkRGIfP5VQG2tCBLc= +google.golang.org/genproto/googleapis/rpc v0.0.0-20230525234030-28d5490b6b19/go.mod h1:66JfowdXAEgad5O9NnYcsNPLCPZJD++2L9X0PCMODrA= +google.golang.org/genproto/googleapis/rpc v0.0.0-20230526203410-71b5a4ffd15e/go.mod h1:66JfowdXAEgad5O9NnYcsNPLCPZJD++2L9X0PCMODrA= google.golang.org/genproto/googleapis/rpc v0.0.0-20230530153820-e85fd2cbaebc/go.mod h1:66JfowdXAEgad5O9NnYcsNPLCPZJD++2L9X0PCMODrA= -google.golang.org/genproto/googleapis/rpc v0.0.0-20230711160842-782d3b101e98 h1:bVf09lpb+OJbByTj913DRJioFFAjf/ZGxEz7MajTp2U= -google.golang.org/genproto/googleapis/rpc v0.0.0-20230711160842-782d3b101e98/go.mod h1:TUfxEVdsvPg18p6AslUXFoLdpED4oBnGwyqk3dV1XzM= +google.golang.org/genproto/googleapis/rpc v0.0.0-20230629202037-9506855d4529/go.mod h1:66JfowdXAEgad5O9NnYcsNPLCPZJD++2L9X0PCMODrA= +google.golang.org/genproto/googleapis/rpc v0.0.0-20230706204954-ccb25ca9f130/go.mod h1:8mL13HKkDa+IuJ8yruA3ci0q+0vsUz4m//+ottjwS5o= +google.golang.org/genproto/googleapis/rpc v0.0.0-20230731190214-cbb8c96f2d6d/go.mod h1:TUfxEVdsvPg18p6AslUXFoLdpED4oBnGwyqk3dV1XzM= google.golang.org/genproto/googleapis/rpc v0.0.0-20230803162519-f966b187b2e5 h1:eSaPbMR4T7WfH9FvABk36NBMacoTUKdWCvV0dx+KfOg= google.golang.org/genproto/googleapis/rpc v0.0.0-20230803162519-f966b187b2e5/go.mod h1:zBEcrKX2ZOcEkHWxBPAIvYUWOKKMIhYcmNiUIu2ji3I= +google.golang.org/genproto/googleapis/rpc v0.0.0-20230815205213-6bfd019c3878/go.mod h1:+Bk1OCOj40wS2hwAMA+aCW9ypzm63QTBBHp6lQ3p+9M= +google.golang.org/genproto/googleapis/rpc v0.0.0-20230822172742-b8732ec3820d h1:uvYuEyMHKNt+lT4K3bN6fGswmK8qSvcreM3BwjDh+y4= +google.golang.org/genproto/googleapis/rpc v0.0.0-20230822172742-b8732ec3820d/go.mod h1:+Bk1OCOj40wS2hwAMA+aCW9ypzm63QTBBHp6lQ3p+9M= google.golang.org/grpc v1.12.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= @@ -1988,10 +2178,10 @@ google.golang.org/grpc v1.50.0/go.mod h1:ZgQEeidpAuNRZ8iRrlBKXZQP1ghovWIVhdJRyCD google.golang.org/grpc v1.50.1/go.mod h1:ZgQEeidpAuNRZ8iRrlBKXZQP1ghovWIVhdJRyCDK+GI= google.golang.org/grpc v1.51.0/go.mod h1:wgNDFcnuBGmxLKI/qn4T+m5BtEBYXJPvibbUPsAIPww= google.golang.org/grpc v1.52.0/go.mod h1:pu6fVzoFb+NBYNAvQL08ic+lvB2IojljRYuun5vorUY= +google.golang.org/grpc v1.52.3/go.mod h1:pu6fVzoFb+NBYNAvQL08ic+lvB2IojljRYuun5vorUY= google.golang.org/grpc v1.53.0/go.mod h1:OnIrk0ipVdj4N5d9IUoFUx72/VlD7+jUsHwZgwSMQpw= google.golang.org/grpc v1.54.0/go.mod h1:PUSEXI6iWghWaB6lXM4knEgpJNu2qUcKfDtNci3EC2g= google.golang.org/grpc v1.55.0/go.mod h1:iYEXKGkEBhg1PjZQvoYEVPTDkHo1/bjTnfwTeGONTY8= -google.golang.org/grpc v1.56.2 h1:fVRFRnXvU+x6C4IlHZewvJOVHoOv1TUuQyoRsYnB4bI= google.golang.org/grpc v1.56.2/go.mod h1:I9bI3vqKfayGqPUAwGdOSu7kt6oIJLixfffKrpXqQ9s= google.golang.org/grpc v1.57.0 h1:kfzNeI/klCGD2YPMUlaGNT3pxvYfga7smW3Vth8Zsiw= google.golang.org/grpc v1.57.0/go.mod h1:Sd+9RMTACXwmub0zcNY2c4arhtrbBYD1AUHI/dt16Mo= @@ -2051,12 +2241,17 @@ lukechampine.com/uint128 v1.2.0/go.mod h1:c4eWIwlEGaxC/+H1VguhU4PHXNWDCDMUlWdIWl modernc.org/cc/v3 v3.36.0/go.mod h1:NFUHyPn4ekoC/JHeZFfZurN6ixxawE1BnVonP/oahEI= modernc.org/cc/v3 v3.36.2/go.mod h1:NFUHyPn4ekoC/JHeZFfZurN6ixxawE1BnVonP/oahEI= modernc.org/cc/v3 v3.36.3/go.mod h1:NFUHyPn4ekoC/JHeZFfZurN6ixxawE1BnVonP/oahEI= +modernc.org/cc/v3 v3.37.0/go.mod h1:vtL+3mdHx/wcj3iEGz84rQa8vEqR6XM84v5Lcvfph20= +modernc.org/cc/v3 v3.40.0/go.mod h1:/bTg4dnWkSXowUO6ssQKnOV0yMVxDYNIsIrzqTFDGH0= modernc.org/ccgo/v3 v3.0.0-20220428102840-41399a37e894/go.mod h1:eI31LL8EwEBKPpNpA4bU1/i+sKOwOrQy8D87zWUcRZc= modernc.org/ccgo/v3 v3.0.0-20220430103911-bc99d88307be/go.mod h1:bwdAnOoaIt8Ax9YdWGjxWsdkPcZyRPHqrOvJxaKAKGw= +modernc.org/ccgo/v3 v3.0.0-20220904174949-82d86e1b6d56/go.mod h1:YSXjPL62P2AMSxBphRHPn7IkzhVHqkvOnRKAKh+W6ZI= modernc.org/ccgo/v3 v3.16.4/go.mod h1:tGtX0gE9Jn7hdZFeU88slbTh1UtCYKusWOoCJuvkWsQ= modernc.org/ccgo/v3 v3.16.6/go.mod h1:tGtX0gE9Jn7hdZFeU88slbTh1UtCYKusWOoCJuvkWsQ= modernc.org/ccgo/v3 v3.16.8/go.mod h1:zNjwkizS+fIFDrDjIAgBSCLkWbJuHF+ar3QRn+Z9aws= modernc.org/ccgo/v3 v3.16.9/go.mod h1:zNMzC9A9xeNUepy6KuZBbugn3c0Mc9TeiJO4lgvkJDo= +modernc.org/ccgo/v3 v3.16.13-0.20221017192402-261537637ce8/go.mod h1:fUB3Vn0nVPReA+7IG7yZDfjv1TMWjhQP8gCxrFAtL5g= +modernc.org/ccgo/v3 v3.16.13/go.mod h1:2Quk+5YgpImhPjv2Qsob1DnZ/4som1lJTodubIcoUkY= modernc.org/ccorpus v1.11.6/go.mod h1:2gEUTrWqdpH2pXsmTM1ZkjeSrUWDpjMu2T6m29L/ErQ= modernc.org/httpfs v1.0.6/go.mod h1:7dosgurJGp0sPaRanU53W4xZYKh14wfzX420oZADeHM= modernc.org/libc v0.0.0-20220428101251-2d5f3daf273b/go.mod h1:p7Mg4+koNjc8jkqwcoFBJx7tXkpj00G77X7A72jXPXA= @@ -2066,19 +2261,31 @@ modernc.org/libc v1.16.17/go.mod h1:hYIV5VZczAmGZAnG15Vdngn5HSF5cSkbvfz2B7GRuVU= modernc.org/libc v1.16.19/go.mod h1:p7Mg4+koNjc8jkqwcoFBJx7tXkpj00G77X7A72jXPXA= modernc.org/libc v1.17.0/go.mod h1:XsgLldpP4aWlPlsjqKRdHPqCxCjISdHfM/yeWC5GyW0= modernc.org/libc v1.17.1/go.mod h1:FZ23b+8LjxZs7XtFMbSzL/EhPxNbfZbErxEHc7cbD9s= +modernc.org/libc v1.17.4/go.mod h1:WNg2ZH56rDEwdropAJeZPQkXmDwh+JCA1s/htl6r2fA= +modernc.org/libc v1.18.0/go.mod h1:vj6zehR5bfc98ipowQOM2nIDUZnVew/wNC/2tOGS+q0= +modernc.org/libc v1.20.3/go.mod h1:ZRfIaEkgrYgZDl6pa4W39HgN5G/yDW+NRmNKZBDFrk0= +modernc.org/libc v1.21.4/go.mod h1:przBsL5RDOZajTVslkugzLBj1evTue36jEomFQOoYuI= +modernc.org/libc v1.22.2/go.mod h1:uvQavJ1pZ0hIoC/jfqNoMLURIMhKzINIWypNM17puug= modernc.org/mathutil v1.2.2/go.mod h1:mZW8CKdRPY1v87qxC/wUdX5O1qDzXMP5TH3wjfpga6E= modernc.org/mathutil v1.4.1/go.mod h1:mZW8CKdRPY1v87qxC/wUdX5O1qDzXMP5TH3wjfpga6E= modernc.org/mathutil v1.5.0/go.mod h1:mZW8CKdRPY1v87qxC/wUdX5O1qDzXMP5TH3wjfpga6E= modernc.org/memory v1.1.1/go.mod h1:/0wo5ibyrQiaoUoH7f9D8dnglAmILJ5/cxZlRECf+Nw= modernc.org/memory v1.2.0/go.mod h1:/0wo5ibyrQiaoUoH7f9D8dnglAmILJ5/cxZlRECf+Nw= modernc.org/memory v1.2.1/go.mod h1:PkUhL0Mugw21sHPeskwZW4D6VscE/GQJOnIpCnW6pSU= +modernc.org/memory v1.3.0/go.mod h1:PkUhL0Mugw21sHPeskwZW4D6VscE/GQJOnIpCnW6pSU= +modernc.org/memory v1.4.0/go.mod h1:PkUhL0Mugw21sHPeskwZW4D6VscE/GQJOnIpCnW6pSU= +modernc.org/memory v1.5.0/go.mod h1:PkUhL0Mugw21sHPeskwZW4D6VscE/GQJOnIpCnW6pSU= modernc.org/opt v0.1.1/go.mod h1:WdSiB5evDcignE70guQKxYUl14mgWtbClRi5wmkkTX0= modernc.org/opt v0.1.3/go.mod h1:WdSiB5evDcignE70guQKxYUl14mgWtbClRi5wmkkTX0= modernc.org/sqlite v1.18.1/go.mod h1:6ho+Gow7oX5V+OiOQ6Tr4xeqbx13UZ6t+Fw9IRUG4d4= +modernc.org/sqlite v1.18.2/go.mod h1:kvrTLEWgxUcHa2GfHBQtanR1H9ht3hTJNtKpzH9k1u0= modernc.org/strutil v1.1.1/go.mod h1:DE+MQQ/hjKBZS2zNInV5hhcipt5rLPWkmpbGeW5mmdw= modernc.org/strutil v1.1.3/go.mod h1:MEHNA7PdEnEwLvspRMtWTNnp2nnyvMfkimT1NKNAGbw= modernc.org/tcl v1.13.1/go.mod h1:XOLfOwzhkljL4itZkK6T72ckMgvj0BDsnKNdZVUOecw= +modernc.org/tcl v1.13.2/go.mod h1:7CLiGIPo1M8Rv1Mitpv5akc2+8fxUd2y2UzC/MfMzy0= modernc.org/token v1.0.0/go.mod h1:UGzOrNV1mAFSEB63lOFHIpNRUVMvYTc6yu1SMY/XTDM= +modernc.org/token v1.0.1/go.mod h1:UGzOrNV1mAFSEB63lOFHIpNRUVMvYTc6yu1SMY/XTDM= +modernc.org/token v1.1.0/go.mod h1:UGzOrNV1mAFSEB63lOFHIpNRUVMvYTc6yu1SMY/XTDM= modernc.org/z v1.5.1/go.mod h1:eWFB510QWW5Th9YGZT81s+LwvaAs3Q2yr4sP0rmLkv8= rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4= diff --git a/flow/workflows/qrep_flow.go b/flow/workflows/qrep_flow.go index d5fbcf8745..9025fbe3ea 100644 --- a/flow/workflows/qrep_flow.go +++ b/flow/workflows/qrep_flow.go @@ -107,11 +107,12 @@ func (q *QRepFlowExecution) getPartitionWorkflowID(ctx workflow.Context) (string // startChildWorkflow starts a single child workflow. func (q *QRepFlowExecution) startChildWorkflow( + boundSelector *concurrency.BoundSelector, ctx workflow.Context, - partition *protos.QRepPartition) (workflow.Future, error) { + partition *protos.QRepPartition) error { wid, err := q.getPartitionWorkflowID(ctx) if err != nil { - return nil, fmt.Errorf("failed to get child workflow ID: %w", err) + return fmt.Errorf("failed to get child workflow ID: %w", err) } partFlowCtx := workflow.WithChildOptions(ctx, workflow.ChildWorkflowOptions{ WorkflowID: wid, @@ -121,8 +122,15 @@ func (q *QRepFlowExecution) startChildWorkflow( }, }) - return workflow.ExecuteChildWorkflow(partFlowCtx, QRepPartitionWorkflow, q.config, partition, - q.runUUID), nil + boundSelector.SpawnChild( + partFlowCtx, + QRepPartitionWorkflow, + q.config, + partition, + q.runUUID, + ) + + return nil } // processPartitions handles the logic for processing the partitions. @@ -131,22 +139,13 @@ func (q *QRepFlowExecution) processPartitions( maxParallelWorkers int, partitions []*protos.QRepPartition, ) error { - boundSelector := concurrency.NewBoundSelector(maxParallelWorkers, ctx) + boundSelector := concurrency.NewBoundSelector(maxParallelWorkers, len(partitions), ctx) for _, partition := range partitions { - future, err := q.startChildWorkflow(ctx, partition) + err := q.startChildWorkflow(boundSelector, ctx, partition) if err != nil { return err } - - boundSelector.AddFuture(future, func(f workflow.Future) error { - if err := f.Get(ctx, nil); err != nil { - q.logger.Error("failed to process partition", "error", err) - return err - } - - return nil - }) } err := boundSelector.Wait() @@ -238,7 +237,17 @@ func QRepFlowWorkflow( return fmt.Errorf("failed to register query handler: %w", err) } - q := NewQRepFlowExecution(ctx, config, uuid.New().String()) + // get qrep run uuid via side-effect + runUUIDSideEffect := workflow.SideEffect(ctx, func(ctx workflow.Context) interface{} { + return uuid.New().String() + }) + + var runUUID string + if err := runUUIDSideEffect.Get(&runUUID); err != nil { + return fmt.Errorf("failed to get run uuid: %w", err) + } + + q := NewQRepFlowExecution(ctx, config, runUUID) err = q.SetupMetadataTables(ctx) if err != nil { @@ -307,8 +316,12 @@ func QRepFlowWorkflow( } // QRepPartitionWorkflow replicate a single partition. -func QRepPartitionWorkflow(ctx workflow.Context, config *protos.QRepConfig, partition *protos.QRepPartition, - runUUID string) error { +func QRepPartitionWorkflow( + ctx workflow.Context, + config *protos.QRepConfig, + partition *protos.QRepPartition, + runUUID string, +) error { q := NewQRepFlowExecution(ctx, config, runUUID) return q.ReplicatePartition(ctx, partition) } diff --git a/flow/workflows/setup_flow.go b/flow/workflows/setup_flow.go index bc55d2ca6b..c5a47d6d38 100644 --- a/flow/workflows/setup_flow.go +++ b/flow/workflows/setup_flow.go @@ -2,6 +2,7 @@ package peerflow import ( "fmt" + "sort" "time" "github.com/PeerDB-io/peer-flow/activities" @@ -101,11 +102,14 @@ func (s *SetupFlowExecution) ensurePullability( }) tmpMap := make(map[uint32]string) + srcTblIdentifiers := maps.Keys(config.TableNameMapping) + sort.Strings(srcTblIdentifiers) + // create EnsurePullabilityInput for the srcTableName ensurePullabilityInput := &protos.EnsurePullabilityBatchInput{ PeerConnectionConfig: config.Source, FlowJobName: s.PeerFlowName, - SourceTableIdentifiers: maps.Keys(config.TableNameMapping), + SourceTableIdentifiers: srcTblIdentifiers, } future := workflow.ExecuteActivity(ctx, flowable.EnsurePullability, ensurePullabilityInput) @@ -115,7 +119,11 @@ func (s *SetupFlowExecution) ensurePullability( return fmt.Errorf("failed to ensure pullability for tables: %w", err) } - for tableName, tableIdentifier := range ensurePullabilityOutput.TableIdentifierMapping { + sortedTableNames := maps.Keys(ensurePullabilityOutput.TableIdentifierMapping) + sort.Strings(sortedTableNames) + + for _, tableName := range sortedTableNames { + tableIdentifier := ensurePullabilityOutput.TableIdentifierMapping[tableName] switch typedTableIdentifier := tableIdentifier.TableIdentifier.(type) { case *protos.TableIdentifier_PostgresTableIdentifier: tmpMap[typedTableIdentifier.PostgresTableIdentifier.RelId] = tableName @@ -163,10 +171,8 @@ func (s *SetupFlowExecution) fetchTableSchemaAndSetupNormalizedTables( HeartbeatTimeout: 5 * time.Minute, }) - sourceTables := make([]string, 0) - for srcTableName := range flowConnectionConfigs.TableNameMapping { - sourceTables = append(sourceTables, srcTableName) - } + sourceTables := maps.Keys(flowConnectionConfigs.TableNameMapping) + sort.Strings(sourceTables) tableSchemaInput := &protos.GetTableSchemaBatchInput{ PeerConnectionConfig: flowConnectionConfigs.Source, @@ -182,10 +188,13 @@ func (s *SetupFlowExecution) fetchTableSchemaAndSetupNormalizedTables( } tableNameSchemaMapping := tblSchemaOutput.TableNameSchemaMapping + sortedSourceTables := maps.Keys(tableNameSchemaMapping) + sort.Strings(sortedSourceTables) s.logger.Info("setting up normalized tables for peer flow - ", s.PeerFlowName) normalizedTableMapping := make(map[string]*protos.TableSchema) - for srcTableName, tableSchema := range tableNameSchemaMapping { + for _, srcTableName := range sortedSourceTables { + tableSchema := tableNameSchemaMapping[srcTableName] normalizedTableName := flowConnectionConfigs.TableNameMapping[srcTableName] normalizedTableMapping[normalizedTableName] = tableSchema s.logger.Info("normalized table schema: ", normalizedTableName, " -> ", tableSchema) diff --git a/flow/workflows/snapshot_flow.go b/flow/workflows/snapshot_flow.go index 299097eff2..8d4d5faf90 100644 --- a/flow/workflows/snapshot_flow.go +++ b/flow/workflows/snapshot_flow.go @@ -3,6 +3,7 @@ package peerflow import ( "fmt" "regexp" + "sort" "time" "github.com/PeerDB-io/peer-flow/concurrency" @@ -13,6 +14,7 @@ import ( "go.temporal.io/sdk/log" "go.temporal.io/sdk/temporal" "go.temporal.io/sdk/workflow" + "golang.org/x/exp/maps" ) type SnapshotFlowExecution struct { @@ -74,11 +76,12 @@ func (s *SnapshotFlowExecution) closeSlotKeepAlive( } func (s *SnapshotFlowExecution) cloneTable( + boundSelector *concurrency.BoundSelector, childCtx workflow.Context, snapshotName string, sourceTableName string, destinationTableName string, -) (workflow.Future, error) { +) error { flowName := s.config.FlowJobName srcName := sourceTableName dstName := destinationTableName @@ -95,8 +98,9 @@ func (s *SnapshotFlowExecution) cloneTable( "snapshotName": snapshotName, }).Errorf("failed to get child id for source table %s and destination table %s", srcName, dstName) - return nil, fmt.Errorf("failed to get child workflow ID: %w", err) + return fmt.Errorf("failed to get child workflow ID: %w", err) } + logrus.WithFields(logrus.Fields{ "flowName": flowName, "snapshotName": snapshotName, @@ -148,15 +152,8 @@ func (s *SnapshotFlowExecution) cloneTable( numPartitionsProcessed := 0 - qrepFuture := workflow.ExecuteChildWorkflow( - childCtx, - QRepFlowWorkflow, - config, - lastPartition, - numPartitionsProcessed, - ) - - return qrepFuture, nil + boundSelector.SpawnChild(childCtx, QRepFlowWorkflow, config, lastPartition, numPartitionsProcessed) + return nil } // startChildQrepWorkflow starts a child workflow for query based replication. @@ -167,31 +164,27 @@ func (s *SnapshotFlowExecution) cloneTables( ) { logrus.Infof("cloning tables for slot name %s and snapshotName %s", slotInfo.SlotName, slotInfo.SnapshotName) - boundSelector := concurrency.NewBoundSelector(maxParallelClones, ctx) - for srcTbl, dstTbl := range s.config.TableNameMapping { + srcTables := maps.Keys(s.config.TableNameMapping) + sort.Strings(srcTables) + + boundSelector := concurrency.NewBoundSelector(maxParallelClones, len(srcTables), ctx) + + for _, srcTbl := range srcTables { source := srcTbl - destination := dstTbl + destination := s.config.TableNameMapping[source] snapshotName := slotInfo.SnapshotName logrus.WithFields(logrus.Fields{ "snapshotName": snapshotName, - }).Infof("Cloning table with source table %s and destination table name %s", - source, destination) - future, err := s.cloneTable(ctx, snapshotName, source, destination) + }).Infof( + "Cloning table with source table %s and destination table name %s", + source, destination, + ) + err := s.cloneTable(boundSelector, ctx, snapshotName, source, destination) if err != nil { s.logger.Error("failed to start clone child workflow: ", err) continue } - boundSelector.AddFuture(future, func(f workflow.Future) error { - logrus.Infof("Adding future for clone table for source name %s and destination %s", - source, destination) - if err := f.Get(ctx, nil); err != nil { - s.logger.Error("failed to clone table", "table", source, "error", err) - return err - } - - return nil - }) } if err := boundSelector.Wait(); err != nil { From 783869223769899bdbc13b585bd415dd2f9fcb12 Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Mon, 28 Aug 2023 12:14:53 -0700 Subject: [PATCH 17/37] simplify bound selector always wait for the first future (#360) --- flow/concurrency/bound_selector.go | 65 ++++++++++++++---------------- 1 file changed, 30 insertions(+), 35 deletions(-) diff --git a/flow/concurrency/bound_selector.go b/flow/concurrency/bound_selector.go index 56f5b42891..29735c09d6 100644 --- a/flow/concurrency/bound_selector.go +++ b/flow/concurrency/bound_selector.go @@ -7,54 +7,49 @@ import ( ) type BoundSelector struct { - ctx workflow.Context - limit workflow.Channel - statusCh workflow.Channel - numFutures int + ctx workflow.Context + limit int + futures []workflow.Future + ferrors []error } func NewBoundSelector(limit int, total int, ctx workflow.Context) *BoundSelector { return &BoundSelector{ - ctx: ctx, - limit: workflow.NewBufferedChannel(ctx, limit), - statusCh: workflow.NewBufferedChannel(ctx, total), - numFutures: 0, + ctx: ctx, + limit: limit, } } func (s *BoundSelector) SpawnChild(chCtx workflow.Context, w interface{}, args ...interface{}) { - s.numFutures++ - workflow.Go(s.ctx, func(ctx workflow.Context) { - s.limit.Send(ctx, struct{}{}) - future := workflow.ExecuteChildWorkflow(chCtx, w, args...) - err := future.Get(ctx, nil) - s.statusCh.Send(ctx, err) - s.limit.Receive(ctx, nil) - }) + if len(s.futures) >= s.limit { + s.waitOne() + } + + future := workflow.ExecuteChildWorkflow(chCtx, w, args...) + s.futures = append(s.futures, future) +} + +func (s *BoundSelector) waitOne() { + if len(s.futures) == 0 { + return + } + + f := s.futures[0] + s.futures = s.futures[1:] + + err := f.Get(s.ctx, nil) + if err != nil { + s.ferrors = append(s.ferrors, err) + } } func (s *BoundSelector) Wait() error { - defer s.statusCh.Close() - defer s.limit.Close() - - ferrors := make([]error, 0) - doneCount := 0 - - for doneCount < s.numFutures { - selector := workflow.NewSelector(s.ctx) - selector.AddReceive(s.statusCh, func(c workflow.ReceiveChannel, more bool) { - var err error - c.Receive(s.ctx, &err) - if err != nil { - ferrors = append(ferrors, err) - } - doneCount++ - }) - selector.Select(s.ctx) + for len(s.futures) > 0 { + s.waitOne() } - if len(ferrors) > 0 { - return errors.Join(ferrors...) + if len(s.ferrors) > 0 { + return errors.Join(s.ferrors...) } return nil From 68f4ae8eec105babeec795302146c1091a53a8c9 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Tue, 29 Aug 2023 21:26:09 +0530 Subject: [PATCH 18/37] Relevant fields to logs for more info (#347) Adds fields such as flow job name, partition ID and other relevant fields in function logs. Fixes #337 --- flow/activities/flowable.go | 36 +++++-- flow/activities/snapshot_activity.go | 12 ++- flow/connectors/bigquery/qrep.go | 10 +- flow/connectors/bigquery/qrep_avro_sync.go | 34 ++++-- flow/connectors/bigquery/qrep_sync_method.go | 10 +- flow/connectors/eventhub/eventhub.go | 5 +- flow/connectors/eventhub/metadata.go | 8 +- flow/connectors/postgres/cdc.go | 12 ++- flow/connectors/postgres/client.go | 4 +- flow/connectors/postgres/postgres.go | 54 +++++++--- flow/connectors/postgres/qrep.go | 69 +++++++++--- flow/connectors/postgres/qrep_bench_test.go | 2 +- .../postgres/qrep_query_build_test.go | 2 +- .../postgres/qrep_query_executor.go | 101 +++++++++++++----- .../postgres/qrep_query_executor_test.go | 6 +- flow/connectors/postgres/qrep_sync_method.go | 39 +++++-- flow/connectors/s3/qrep.go | 5 +- flow/connectors/snowflake/qrep.go | 54 +++++++--- flow/connectors/snowflake/qrep_avro_sync.go | 89 ++++++++++++--- flow/connectors/snowflake/snowflake.go | 15 ++- flow/e2e/qrep_flow_test.go | 7 +- 21 files changed, 447 insertions(+), 127 deletions(-) diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index 776b43c9fd..5893b15859 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -176,13 +176,17 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, input *protos.StartFlo return nil, fmt.Errorf("failed to get destination connector: %w", err) } - log.Info("initializing table schema...") + log.WithFields(log.Fields{ + "flowName": input.FlowConnectionConfigs.FlowJobName, + }).Infof("initializing table schema...") err = dest.InitializeTableSchema(input.FlowConnectionConfigs.TableNameSchemaMapping) if err != nil { return nil, fmt.Errorf("failed to initialize table schema: %w", err) } - log.Info("pulling records...") + log.WithFields(log.Fields{ + "flowName": input.FlowConnectionConfigs.FlowJobName, + }).Info("pulling records...") startTime := time.Now() records, err := src.PullRecords(&model.PullRecordsRequest{ @@ -220,11 +224,15 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, input *protos.StartFlo // log the number of records numRecords := len(records.Records) - log.Printf("pulled %d records", numRecords) + log.WithFields(log.Fields{ + "flowName": input.FlowConnectionConfigs.FlowJobName, + }).Printf("pulled %d records", numRecords) activity.RecordHeartbeat(ctx, fmt.Sprintf("pulled %d records", numRecords)) if numRecords == 0 { - log.Info("no records to push") + log.WithFields(log.Fields{ + "flowName": input.FlowConnectionConfigs.FlowJobName, + }).Info("no records to push") return nil, nil } @@ -238,7 +246,9 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, input *protos.StartFlo log.Warnf("failed to push records: %v", err) return nil, fmt.Errorf("failed to push records: %w", err) } - log.Info("pushed records") + log.WithFields(log.Fields{ + "flowName": input.FlowConnectionConfigs.FlowJobName, + }).Infof("pushed %d records", res.NumRecordsSynced) err = a.CatalogMirrorMonitor. UpdateLatestLSNAtTargetForCDCFlow(ctx, input.FlowConnectionConfigs.FlowJobName, @@ -402,7 +412,9 @@ func (a *FlowableActivity) ReplicateQRepPartition(ctx context.Context, tmp, err := pgConn.PullQRepRecordStream(config, partition, stream) numRecords = int64(tmp) if err != nil { - log.Errorf("failed to pull records: %v", err) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Errorf("failed to pull records: %v", err) goroutineErr = err } err = a.CatalogMirrorMonitor.UpdatePullEndTimeAndRowsForPartition(ctx, runUUID, partition, numRecords) @@ -420,7 +432,9 @@ func (a *FlowableActivity) ReplicateQRepPartition(ctx context.Context, return fmt.Errorf("failed to pull records: %w", err) } numRecords = int64(recordBatch.NumRecords) - log.Printf("pulled %d records\n", len(recordBatch.Records)) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Printf("pulled %d records\n", len(recordBatch.Records)) err = a.CatalogMirrorMonitor.UpdatePullEndTimeAndRowsForPartition(ctx, runUUID, partition, numRecords) if err != nil { @@ -447,7 +461,9 @@ func (a *FlowableActivity) ReplicateQRepPartition(ctx context.Context, } if res == 0 { - log.Printf("no records to push for partition %s\n", partition.PartitionId) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Printf("no records to push for partition %s\n", partition.PartitionId) return nil } @@ -455,7 +471,9 @@ func (a *FlowableActivity) ReplicateQRepPartition(ctx context.Context, if goroutineErr != nil { return goroutineErr } - log.Printf("pushed %d records\n", res) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Printf("pushed %d records\n", res) err = a.CatalogMirrorMonitor.UpdateEndTimeForPartition(ctx, runUUID, partition) if err != nil { return err diff --git a/flow/activities/snapshot_activity.go b/flow/activities/snapshot_activity.go index 30d74f1d01..efd2950b59 100644 --- a/flow/activities/snapshot_activity.go +++ b/flow/activities/snapshot_activity.go @@ -54,17 +54,23 @@ func (a *SnapshotActivity) SetupReplication( pgConn := conn.(*connpostgres.PostgresConnector) err = pgConn.SetupReplication(slotSignal, config) if err != nil { - log.Errorf("failed to setup replication: %v", err) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Errorf("failed to setup replication: %v", err) replicationErr <- err return } }() - log.Info("waiting for slot to be created...") + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Info("waiting for slot to be created...") var slotInfo *connpostgres.SlotCreationResult select { case slotInfo = <-slotSignal.SlotCreated: - log.Infof("slot '%s' created", slotInfo.SlotName) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Infof("slot '%s' created", slotInfo.SlotName) case err := <-replicationErr: return nil, fmt.Errorf("failed to setup replication: %w", err) } diff --git a/flow/connectors/bigquery/qrep.go b/flow/connectors/bigquery/qrep.go index 9ae060a1fd..e1ee8cf3c5 100644 --- a/flow/connectors/bigquery/qrep.go +++ b/flow/connectors/bigquery/qrep.go @@ -44,9 +44,17 @@ func (c *BigQueryConnector) SyncQRepRecords( } if done { - log.Infof("Partition %s has already been synced", partition.PartitionId) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + "partitionID": partition.PartitionId, + }).Infof("Partition %s has already been synced", partition.PartitionId) return 0, nil } + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Infof("QRep sync function called and partition existence checked for"+ + " partition %s of destination table %s", + partition.PartitionId, destTable) syncMode := config.SyncMode switch syncMode { diff --git a/flow/connectors/bigquery/qrep_avro_sync.go b/flow/connectors/bigquery/qrep_avro_sync.go index ce90de6c14..a4cd92a82c 100644 --- a/flow/connectors/bigquery/qrep_avro_sync.go +++ b/flow/connectors/bigquery/qrep_avro_sync.go @@ -85,7 +85,11 @@ func (s *QRepAvroSyncMethod) SyncRecords( // drop the staging table if err := bqClient.Dataset(datasetID).Table(stagingTable).Delete(s.connector.ctx); err != nil { // just log the error this isn't fatal. - log.Errorf("failed to delete staging table %s: %v", stagingTable, err) + log.WithFields(log.Fields{ + "flowName": flowJobName, + "syncBatchID": syncBatchID, + "destinationTable": dstTableName, + }).Errorf("failed to delete staging table %s: %v", stagingTable, err) } log.Printf("loaded stage into %s.%s", @@ -108,7 +112,10 @@ func (s *QRepAvroSyncMethod) SyncQRepRecords( if err != nil { return 0, fmt.Errorf("failed to define Avro schema: %w", err) } - + log.WithFields(log.Fields{ + "flowName": flowJobName, + }).Infof("Obtained Avro schema for destination table %s and partition ID %s", + dstTableName, partition.PartitionId) fmt.Printf("Avro schema: %s\n", avroSchema) // create a staging table name with partitionID replace hyphens with underscores stagingTable := fmt.Sprintf("%s_%s_staging", dstTableName, strings.ReplaceAll(partition.PartitionId, "-", "_")) @@ -136,6 +143,10 @@ func (s *QRepAvroSyncMethod) SyncQRepRecords( if err != nil { return -1, fmt.Errorf("failed to create metadata insert statement: %v", err) } + log.WithFields(log.Fields{ + "flowName": flowJobName, + }).Infof("Performing transaction inside QRep sync function for partition ID %s", + partition.PartitionId) stmts = append(stmts, insertMetadataStmt) stmts = append(stmts, "COMMIT TRANSACTION;") // Execute the statements in a transaction @@ -150,10 +161,17 @@ func (s *QRepAvroSyncMethod) SyncQRepRecords( // drop the staging table if err := bqClient.Dataset(datasetID).Table(stagingTable).Delete(s.connector.ctx); err != nil { // just log the error this isn't fatal. - log.Errorf("failed to delete staging table %s: %v", stagingTable, err) + log.WithFields(log.Fields{ + "flowName": flowJobName, + "partitionID": partition.PartitionId, + "destinationTable": dstTableName, + }).Errorf("failed to delete staging table %s: %v", stagingTable, err) } - log.Printf("loaded stage into %s.%s", + log.WithFields(log.Fields{ + "flowName": flowJobName, + "partitionID": partition.PartitionId, + }).Infof("loaded stage into %s.%s", datasetID, dstTableName) return numRecords, nil } @@ -326,7 +344,9 @@ func (s *QRepAvroSyncMethod) writeToStage( schema, err := stream.Schema() if err != nil { - log.Errorf("failed to get schema from stream: %v", err) + log.WithFields(log.Fields{ + "partitonOrBatchID": syncID, + }).Errorf("failed to get schema from stream: %v", err) return 0, fmt.Errorf("failed to get schema from stream: %w", err) } @@ -344,7 +364,9 @@ func (s *QRepAvroSyncMethod) writeToStage( ) } if qRecordOrErr.Err != nil { - log.Errorf("[bq_avro] failed to get record from stream: %v", qRecordOrErr.Err) + log.WithFields(log.Fields{ + "batchOrPartitionID": syncID, + }).Errorf("[bq_avro] failed to get record from stream: %v", qRecordOrErr.Err) return 0, fmt.Errorf("[bq_avro] failed to get record from stream: %w", qRecordOrErr.Err) } diff --git a/flow/connectors/bigquery/qrep_sync_method.go b/flow/connectors/bigquery/qrep_sync_method.go index f68463d458..2a8eb0f5cd 100644 --- a/flow/connectors/bigquery/qrep_sync_method.go +++ b/flow/connectors/bigquery/qrep_sync_method.go @@ -73,7 +73,10 @@ func (s *QRepStagingTableSync) SyncQRepRecords( schema, err := stream.Schema() if err != nil { - log.Errorf("failed to get schema from stream: %v", err) + log.WithFields(log.Fields{ + "flowName": flowJobName, + "partitionID": partitionID, + }).Errorf("failed to get schema from stream: %v", err) return 0, fmt.Errorf("failed to get schema from stream: %w", err) } @@ -81,7 +84,10 @@ func (s *QRepStagingTableSync) SyncQRepRecords( valueSaverRecords := make([]bigquery.ValueSaver, 0) for qRecordOrErr := range stream.Records { if qRecordOrErr.Err != nil { - log.Errorf("[bq] failed to get record from stream: %v", qRecordOrErr.Err) + log.WithFields(log.Fields{ + "flowName": flowJobName, + "partitionID": partitionID, + }).Errorf("[bq] failed to get record from stream: %v", qRecordOrErr.Err) return 0, fmt.Errorf("[bq] failed to get record from stream: %w", qRecordOrErr.Err) } diff --git a/flow/connectors/eventhub/eventhub.go b/flow/connectors/eventhub/eventhub.go index 13e62b2897..f94b5b0d3d 100644 --- a/flow/connectors/eventhub/eventhub.go +++ b/flow/connectors/eventhub/eventhub.go @@ -232,7 +232,10 @@ func (c *EventHubConnector) CreateRawTable(req *protos.CreateRawTableInput) (*pr for _, table := range tableMap { err := c.ensureEventHub(c.ctx, table) if err != nil { - log.Errorf("failed to get event hub properties: %v", err) + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + "table": table, + }).Errorf("failed to get event hub properties: %v", err) return nil, err } } diff --git a/flow/connectors/eventhub/metadata.go b/flow/connectors/eventhub/metadata.go index 09cdb6c315..84175b0d27 100644 --- a/flow/connectors/eventhub/metadata.go +++ b/flow/connectors/eventhub/metadata.go @@ -137,7 +137,9 @@ func (c *EventHubConnector) GetLastOffset(jobName string) (*protos.LastSyncState } func (c *EventHubConnector) GetLastSyncBatchID(jobName string) (int64, error) { - log.Errorf("GetLastSyncBatchID not supported for EventHub") + log.WithFields(log.Fields{ + "flowName": jobName, + }).Errorf("GetLastSyncBatchID not supported for EventHub") return 0, fmt.Errorf("GetLastSyncBatchID not supported for EventHub connector") } @@ -162,7 +164,9 @@ func (c *EventHubConnector) UpdateLastOffset(jobName string, offset int64) error `, jobName, offset) if err != nil { - log.Errorf("failed to update last offset: %v", err) + log.WithFields(log.Fields{ + "flowName": jobName, + }).Errorf("failed to update last offset: %v", err) return err } diff --git a/flow/connectors/postgres/cdc.go b/flow/connectors/postgres/cdc.go index 92b1c94755..0ea09f923d 100644 --- a/flow/connectors/postgres/cdc.go +++ b/flow/connectors/postgres/cdc.go @@ -76,7 +76,9 @@ func (p *PostgresCDCSource) PullRecords(req *model.PullRecordsRequest) (*model.R defer replicationConn.Release() pgConn := replicationConn.Conn().PgConn() - log.Infof("created replication connection") + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + }).Infof("created replication connection") sysident, err := pglogrepl.IdentifySystem(p.ctx, pgConn) if err != nil { @@ -96,7 +98,9 @@ func (p *PostgresCDCSource) PullRecords(req *model.PullRecordsRequest) (*model.R if err != nil { return nil, fmt.Errorf("error starting replication at startLsn - %d: %w", p.startLSN, err) } - log.Infof("started replication on slot %s at startLSN: %d", replicationSlot, p.startLSN) + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + }).Infof("started replication on slot %s at startLSN: %d", p.slot, p.startLSN) return p.consumeStream(pgConn, req, p.startLSN) } @@ -119,7 +123,9 @@ func (p *PostgresCDCSource) consumeStream( defer func() { err := conn.Close(p.ctx) if err != nil { - log.Errorf("unexpected error closing replication connection: %v", err) + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + }).Errorf("unexpected error closing replication connection: %v", err) } }() diff --git a/flow/connectors/postgres/client.go b/flow/connectors/postgres/client.go index 0799917565..b10938f405 100644 --- a/flow/connectors/postgres/client.go +++ b/flow/connectors/postgres/client.go @@ -562,7 +562,9 @@ func (c *PostgresConnector) getApproxTableCounts(tables []string) (int64, error) var count int64 err := row.Scan(&count) if err != nil { - log.Errorf("error while scanning row: %v", err) + log.WithFields(log.Fields{ + "table": table, + }).Errorf("error while scanning row: %v", err) return fmt.Errorf("error while scanning row: %w", err) } totalCount += count diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index 46de9889b6..00b82705b7 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -154,7 +154,7 @@ func (c *PostgresConnector) GetLastOffset(jobName string) (*protos.LastSyncState defer rows.Close() if !rows.Next() { - log.Warnf("No row found for job %s, returning nil", jobName) + log.Infof("No row found for job %s, returning nil", jobName) return nil, nil } var result int64 @@ -193,14 +193,23 @@ func (c *PostgresConnector) PullRecords(req *model.PullRecordsRequest) (*model.R } if !exists.PublicationExists { - log.Warnf("publication %s does not exist", publicationName) + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + }).Warnf("publication %s does not exist", publicationName) publicationName = "" } if !exists.SlotExists { + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + }).Warnf("slot %s does not exist", slotName) return nil, fmt.Errorf("replication slot %s does not exist", slotName) } + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + }).Infof("PullRecords: performed checks for slot and publication") + cdc, err := NewPostgresCDCSource(&PostgresCDCConfig{ AppContext: c.ctx, Connection: c.replPool, @@ -242,7 +251,9 @@ func (c *PostgresConnector) PullRecords(req *model.PullRecordsRequest) (*model.R // SyncRecords pushes records to the destination. func (c *PostgresConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.SyncResponse, error) { rawTableIdentifier := getRawTableIdentifier(req.FlowJobName) - log.Printf("pushing %d records to Postgres table %s via COPY", len(req.Records.Records), rawTableIdentifier) + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + }).Printf("pushing %d records to Postgres table %s via COPY", len(req.Records.Records), rawTableIdentifier) syncBatchID, err := c.GetLastSyncBatchID(req.FlowJobName) if err != nil { @@ -338,7 +349,9 @@ func (c *PostgresConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S defer func() { deferErr := syncRecordsTx.Rollback(c.ctx) if deferErr != pgx.ErrTxClosed && deferErr != nil { - log.Errorf("unexpected error rolling back transaction for syncing records: %v", err) + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + }).Errorf("unexpected error rolling back transaction for syncing records: %v", err) } }() @@ -356,7 +369,9 @@ func (c *PostgresConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S } metrics.LogSyncMetrics(c.ctx, req.FlowJobName, syncedRecordsCount, time.Since(startTime)) - log.Printf("synced %d records to Postgres table %s via COPY", syncedRecordsCount, rawTableIdentifier) + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + }).Printf("synced %d records to Postgres table %s via COPY", syncedRecordsCount, rawTableIdentifier) // updating metadata with new offset and syncBatchID err = c.updateSyncMetadata(req.FlowJobName, lastCP, syncBatchID, syncRecordsTx) @@ -394,7 +409,9 @@ func (c *PostgresConnector) NormalizeRecords(req *model.NormalizeRecordsRequest) } // normalize has caught up with sync or no SyncFlow has run, chill until more records are loaded. if syncBatchID == normalizeBatchID || !jobMetadataExists { - log.Printf("no records to normalize: syncBatchID %d, normalizeBatchID %d", syncBatchID, normalizeBatchID) + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + }).Printf("no records to normalize: syncBatchID %d, normalizeBatchID %d", syncBatchID, normalizeBatchID) return &model.NormalizeResponse{ Done: true, StartBatchID: normalizeBatchID, @@ -414,7 +431,9 @@ func (c *PostgresConnector) NormalizeRecords(req *model.NormalizeRecordsRequest) defer func() { deferErr := normalizeRecordsTx.Rollback(c.ctx) if deferErr != pgx.ErrTxClosed && deferErr != nil { - log.Errorf("unexpected error rolling back transaction for normalizing records: %v", err) + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + }).Errorf("unexpected error rolling back transaction for normalizing records: %v", err) } }() @@ -443,7 +462,9 @@ func (c *PostgresConnector) NormalizeRecords(req *model.NormalizeRecordsRequest) return nil, fmt.Errorf("error executing merge statements: %w", err) } } - log.Printf("normalized %d records", totalRowsAffected) + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + }).Infof("normalized %d records", totalRowsAffected) if totalRowsAffected > 0 { totalRowsAtTarget, err := c.getApproxTableCounts(maps.Keys(unchangedToastColsMap)) if err != nil { @@ -487,7 +508,9 @@ func (c *PostgresConnector) CreateRawTable(req *protos.CreateRawTableInput) (*pr defer func() { deferErr := createRawTableTx.Rollback(c.ctx) if deferErr != pgx.ErrTxClosed && deferErr != nil { - log.Errorf("unexpected error rolling back transaction for creating raw table: %v", err) + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + }).Errorf("unexpected error rolling back transaction for creating raw table: %v", err) } }() @@ -579,10 +602,13 @@ func (c *PostgresConnector) SetupNormalizedTables(req *protos.SetupNormalizedTab if err != nil { return nil, fmt.Errorf("error starting transaction for creating raw table: %w", err) } + defer func() { deferErr := createNormalizedTablesTx.Rollback(c.ctx) if deferErr != pgx.ErrTxClosed && deferErr != nil { - log.Errorf("unexpected error rolling back transaction for creating raw table: %v", err) + log.WithFields(log.Fields{ + "tableMapping": req.TableNameSchemaMapping, + }).Errorf("unexpected error rolling back transaction for creating raw table: %v", err) } }() @@ -707,7 +733,9 @@ func (c *PostgresConnector) PullFlowCleanup(jobName string) error { defer func() { deferErr := pullFlowCleanupTx.Rollback(c.ctx) if deferErr != pgx.ErrTxClosed && deferErr != nil { - log.Errorf("unexpected error rolling back transaction for flow cleanup: %v", err) + log.WithFields(log.Fields{ + "flowName": jobName, + }).Errorf("unexpected error rolling back transaction for flow cleanup: %v", err) } }() @@ -737,7 +765,9 @@ func (c *PostgresConnector) SyncFlowCleanup(jobName string) error { defer func() { deferErr := syncFlowCleanupTx.Rollback(c.ctx) if deferErr != sql.ErrTxDone && deferErr != nil { - log.Errorf("unexpected error while rolling back transaction for flow cleanup: %v", deferErr) + log.WithFields(log.Fields{ + "flowName": jobName, + }).Errorf("unexpected error while rolling back transaction for flow cleanup: %v", deferErr) } }() diff --git a/flow/connectors/postgres/qrep.go b/flow/connectors/postgres/qrep.go index 5c11f7fac7..6d44e25bc9 100644 --- a/flow/connectors/postgres/qrep.go +++ b/flow/connectors/postgres/qrep.go @@ -44,7 +44,9 @@ func (c *PostgresConnector) GetQRepPartitions( defer func() { deferErr := tx.Rollback(c.ctx) if deferErr != pgx.ErrTxClosed && deferErr != nil { - log.Errorf("unexpected error rolling back transaction for get partitions: %v", err) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Errorf("unexpected error rolling back transaction for get partitions: %v", err) } }() @@ -187,7 +189,9 @@ func (c *PostgresConnector) getNumRowsPartitions( rows, err = tx.Query(c.ctx, partitionsQuery) } if err != nil { - log.Errorf("failed to query for partitions: %v", err) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Errorf("failed to query for partitions: %v", err) return nil, fmt.Errorf("failed to query for partitions: %w", err) } @@ -246,7 +250,9 @@ func (c *PostgresConnector) getMinMaxValues( minQuery := fmt.Sprintf("SELECT MIN(%[1]s) FROM %[2]s", quotedWatermarkColumn, config.WatermarkTable) row := tx.QueryRow(c.ctx, minQuery) if err := row.Scan(&minValue); err != nil { - log.Errorf("failed to query [%s] for min value: %v", minQuery, err) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Errorf("failed to query [%s] for min value: %v", minQuery, err) return nil, nil, fmt.Errorf("failed to query for min value: %w", err) } @@ -272,8 +278,11 @@ func (c *PostgresConnector) PullQRepRecords( config *protos.QRepConfig, partition *protos.QRepPartition) (*model.QRecordBatch, error) { if partition.FullTablePartition { - log.Infof("pulling full table partition for flow job %s", config.FlowJobName) - executor := NewQRepQueryExecutorSnapshot(c.pool, c.ctx, c.config.TransactionSnapshot) + log.WithFields(log.Fields{ + "partitionId": partition.PartitionId, + }).Infof("pulling full table partition for flow job %s", config.FlowJobName) + executor := NewQRepQueryExecutorSnapshot(c.pool, c.ctx, c.config.TransactionSnapshot, + config.FlowJobName, partition.PartitionId) query := config.Query return executor.ExecuteAndProcessQuery(query) } @@ -303,16 +312,22 @@ func (c *PostgresConnector) PullQRepRecords( default: return nil, fmt.Errorf("unknown range type: %v", x) } + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + "partition": partition.PartitionId, + }).Infof("Obtained ranges for partition for PullQRep") // Build the query to pull records within the range from the source table // Be sure to order the results by the watermark column to ensure consistency across pulls - query, err := BuildQuery(config.Query) + query, err := BuildQuery(config.Query, config.FlowJobName) if err != nil { return nil, err } - executor := NewQRepQueryExecutorSnapshot(c.pool, c.ctx, c.config.TransactionSnapshot) - records, err := executor.ExecuteAndProcessQuery(query, rangeStart, rangeEnd) + executor := NewQRepQueryExecutorSnapshot(c.pool, c.ctx, c.config.TransactionSnapshot, + config.FlowJobName, partition.PartitionId) + records, err := executor.ExecuteAndProcessQuery(query, + rangeStart, rangeEnd) if err != nil { return nil, err } @@ -331,12 +346,20 @@ func (c *PostgresConnector) PullQRepRecordStream( stream *model.QRecordStream, ) (int, error) { if partition.FullTablePartition { - log.Infof("pulling full table partition for flow job %s", config.FlowJobName) - executor := NewQRepQueryExecutorSnapshot(c.pool, c.ctx, c.config.TransactionSnapshot) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + "partitionId": partition.PartitionId, + }).Infof("pulling full table partition for flow job %s", config.FlowJobName) + executor := NewQRepQueryExecutorSnapshot(c.pool, c.ctx, c.config.TransactionSnapshot, + config.FlowJobName, partition.PartitionId) query := config.Query _, err := executor.ExecuteAndProcessQueryStream(stream, query) return 0, err } + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + "partition": partition.PartitionId, + }).Infof("Obtained ranges for partition for PullQRepStream") var rangeStart interface{} var rangeEnd interface{} @@ -366,12 +389,13 @@ func (c *PostgresConnector) PullQRepRecordStream( // Build the query to pull records within the range from the source table // Be sure to order the results by the watermark column to ensure consistency across pulls - query, err := BuildQuery(config.Query) + query, err := BuildQuery(config.Query, config.FlowJobName) if err != nil { return 0, err } - executor := NewQRepQueryExecutorSnapshot(c.pool, c.ctx, c.config.TransactionSnapshot) + executor := NewQRepQueryExecutorSnapshot(c.pool, c.ctx, c.config.TransactionSnapshot, + config.FlowJobName, partition.PartitionId) numRecords, err := executor.ExecuteAndProcessQueryStream(stream, query, rangeStart, rangeEnd) if err != nil { return 0, err @@ -382,7 +406,9 @@ func (c *PostgresConnector) PullQRepRecordStream( return 0, err } metrics.LogQRepPullMetrics(c.ctx, config.FlowJobName, numRecords, totalRecordsAtSource) - log.Infof("pulled %d records for flow job %s", numRecords, config.FlowJobName) + log.WithFields(log.Fields{ + "partition": partition.PartitionId, + }).Infof("pulled %d records for flow job %s", numRecords, config.FlowJobName) return numRecords, nil } @@ -411,9 +437,15 @@ func (c *PostgresConnector) SyncQRepRecords( } if done { - log.Infof("partition %s already synced", partition.PartitionId) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Infof("partition %s already synced", partition.PartitionId) return 0, nil } + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + "partition": partition.PartitionId, + }).Infof("SyncRecords called and initial checks complete.") syncMode := config.SyncMode switch syncMode { @@ -445,11 +477,14 @@ func (c *PostgresConnector) SetupQRepMetadataTables(config *protos.QRepConfig) e if err != nil { return fmt.Errorf("failed to create table %s: %w", qRepMetadataTableName, err) } + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Infof("Setup metadata table.") return nil } -func BuildQuery(query string) (string, error) { +func BuildQuery(query string, flowJobName string) (string, error) { tmpl, err := template.New("query").Parse(query) if err != nil { return "", err @@ -468,7 +503,9 @@ func BuildQuery(query string) (string, error) { } res := buf.String() - log.Infof("templated query: %s", res) + log.WithFields(log.Fields{ + "flowName": flowJobName, + }).Infof("templated query: %s", res) return res, nil } diff --git a/flow/connectors/postgres/qrep_bench_test.go b/flow/connectors/postgres/qrep_bench_test.go index 24e425830d..a8ee28ef15 100644 --- a/flow/connectors/postgres/qrep_bench_test.go +++ b/flow/connectors/postgres/qrep_bench_test.go @@ -21,7 +21,7 @@ func BenchmarkQRepQueryExecutor(b *testing.B) { defer pool.Close() // Create a new QRepQueryExecutor instance - qe := NewQRepQueryExecutor(pool, context.Background()) + qe := NewQRepQueryExecutor(pool, context.Background(), "test flow", "test part") // Run the benchmark b.ResetTimer() diff --git a/flow/connectors/postgres/qrep_query_build_test.go b/flow/connectors/postgres/qrep_query_build_test.go index f62fb2682b..25c1fef6e6 100644 --- a/flow/connectors/postgres/qrep_query_build_test.go +++ b/flow/connectors/postgres/qrep_query_build_test.go @@ -34,7 +34,7 @@ func TestBuildQuery(t *testing.T) { for _, tc := range testCases { t.Run(tc.name, func(t *testing.T) { - actual, err := BuildQuery(tc.query) + actual, err := BuildQuery(tc.query, "test_flow") if err != nil { t.Fatalf("Error returned by BuildQuery: %v", err) } diff --git a/flow/connectors/postgres/qrep_query_executor.go b/flow/connectors/postgres/qrep_query_executor.go index 8b5cbb7d3c..8857e8eb04 100644 --- a/flow/connectors/postgres/qrep_query_executor.go +++ b/flow/connectors/postgres/qrep_query_executor.go @@ -17,25 +17,37 @@ import ( ) type QRepQueryExecutor struct { - pool *pgxpool.Pool - ctx context.Context - snapshot string - testEnv bool + pool *pgxpool.Pool + ctx context.Context + snapshot string + testEnv bool + flowJobName string + partitionID string } -func NewQRepQueryExecutor(pool *pgxpool.Pool, ctx context.Context) *QRepQueryExecutor { +func NewQRepQueryExecutor(pool *pgxpool.Pool, ctx context.Context, + flowJobName string, partitionID string) *QRepQueryExecutor { return &QRepQueryExecutor{ - pool: pool, - ctx: ctx, - snapshot: "", + pool: pool, + ctx: ctx, + snapshot: "", + flowJobName: flowJobName, + partitionID: partitionID, } } -func NewQRepQueryExecutorSnapshot(pool *pgxpool.Pool, ctx context.Context, snapshot string) *QRepQueryExecutor { +func NewQRepQueryExecutorSnapshot(pool *pgxpool.Pool, ctx context.Context, snapshot string, + flowJobName string, partitionID string) *QRepQueryExecutor { + log.WithFields(log.Fields{ + "flowName": flowJobName, + "partitionID": partitionID, + }).Info("Declared new qrep executor for snapshot") return &QRepQueryExecutor{ - pool: pool, - ctx: ctx, - snapshot: snapshot, + pool: pool, + ctx: ctx, + snapshot: snapshot, + flowJobName: flowJobName, + partitionID: partitionID, } } @@ -52,6 +64,10 @@ func (qe *QRepQueryExecutor) ExecuteQuery(query string, args ...interface{}) (pg } func (qe *QRepQueryExecutor) executeQueryInTx(tx pgx.Tx, cursorName string, fetchSize int) (pgx.Rows, error) { + log.WithFields(log.Fields{ + "flowName": qe.flowJobName, + "partitionID": qe.partitionID, + }).Info("Executing query in transaction") q := fmt.Sprintf("FETCH %d FROM %s", fetchSize, cursorName) if !qe.testEnv { @@ -96,7 +112,10 @@ func (qe *QRepQueryExecutor) ProcessRows( ) (*model.QRecordBatch, error) { // Initialize the record slice records := make([]*model.QRecord, 0) - + log.WithFields(log.Fields{ + "flowName": qe.flowJobName, + "partitionID": qe.partitionID, + }).Info("Processing rows") // Iterate over the rows for rows.Next() { record, err := mapRowToQRecord(rows, fieldDescriptions) @@ -117,7 +136,10 @@ func (qe *QRepQueryExecutor) ProcessRows( Schema: fieldDescriptionsToSchema(fieldDescriptions), } - log.Infof("[postgres] pulled %d records", batch.NumRecords) + log.WithFields(log.Fields{ + "flowName": qe.flowJobName, + "partitionID": qe.partitionID, + }).Infof("[postgres] pulled %d records", batch.NumRecords) return batch, nil } @@ -154,7 +176,10 @@ func (qe *QRepQueryExecutor) processRowsStream( } qe.recordHeartbeat("cursor %s - fetch completed - %d records", cursorName, numRows) - + log.WithFields(log.Fields{ + "flowName": qe.flowJobName, + "partitionID": qe.partitionID, + }).Infof("processed row stream") return numRows, nil } @@ -179,7 +204,9 @@ func (qe *QRepQueryExecutor) processFetchedRows( stream.Records <- &model.QRecordOrError{ Err: err, } - log.Errorf("[pg_query_executor] failed to execute query in tx: %v", err) + log.WithFields(log.Fields{ + "query": query, + }).Errorf("[pg_query_executor] failed to execute query in tx: %v", err) return 0, fmt.Errorf("[pg_query_executor] failed to execute query in tx: %w", err) } @@ -215,10 +242,12 @@ func (qe *QRepQueryExecutor) ExecuteAndProcessQuery( args ...interface{}, ) (*model.QRecordBatch, error) { stream := model.NewQRecordStream(1024) - errors := make(chan error, 1) defer close(errors) - + log.WithFields(log.Fields{ + "flowName": qe.flowJobName, + "partitionID": qe.partitionID, + }).Infof("Executing and processing query '%s'", query) go func() { _, err := qe.ExecuteAndProcessQueryStream(stream, query, args...) if err != nil { @@ -256,6 +285,10 @@ func (qe *QRepQueryExecutor) ExecuteAndProcessQueryStream( query string, args ...interface{}, ) (int, error) { + log.WithFields(log.Fields{ + "flowName": qe.flowJobName, + "partitionID": qe.partitionID, + }).Infof("Executing and processing query stream '%s'", query) defer close(stream.Records) tx, err := qe.pool.BeginTx(qe.ctx, pgx.TxOptions{ @@ -263,7 +296,10 @@ func (qe *QRepQueryExecutor) ExecuteAndProcessQueryStream( IsoLevel: pgx.RepeatableRead, }) if err != nil { - log.Errorf("[pg_query_executor] failed to begin transaction: %v", err) + log.WithFields(log.Fields{ + "flowName": qe.flowJobName, + "partitionID": qe.partitionID, + }).Errorf("[pg_query_executor] failed to begin transaction: %v", err) return 0, fmt.Errorf("[pg_query_executor] failed to begin transaction: %w", err) } @@ -280,7 +316,11 @@ func (qe *QRepQueryExecutor) ExecuteAndProcessQueryStream( stream.Records <- &model.QRecordOrError{ Err: fmt.Errorf("failed to set snapshot: %w", err), } - log.Errorf("[pg_query_executor] failed to set snapshot: %v", err) + log.WithFields(log.Fields{ + "flowName": qe.flowJobName, + "partitionID": qe.partitionID, + "query": query, + }).Errorf("[pg_query_executor] failed to set snapshot: %v", err) return 0, fmt.Errorf("[pg_query_executor] failed to set snapshot: %w", err) } } @@ -295,18 +335,23 @@ func (qe *QRepQueryExecutor) ExecuteAndProcessQueryStream( cursorName := fmt.Sprintf("peerdb_cursor_%d", randomUint) fetchSize := shared.FetchAndChannelSize - cursorQuery := fmt.Sprintf("DECLARE %s CURSOR FOR %s", cursorName, query) _, err = tx.Exec(qe.ctx, cursorQuery, args...) if err != nil { stream.Records <- &model.QRecordOrError{ Err: fmt.Errorf("failed to declare cursor: %w", err), } - log.Errorf("[pg_query_executor] failed to declare cursor: %v", err) + log.WithFields(log.Fields{ + "flowName": qe.flowJobName, + "partitionID": qe.partitionID, + }).Infof("[pg_query_executor] failed to declare cursor with query %v: %v", cursorQuery, err) return 0, fmt.Errorf("[pg_query_executor] failed to declare cursor: %w", err) } - log.Infof("[pg_query_executor] declared cursor '%s' for query '%s'", cursorName, query) + log.WithFields(log.Fields{ + "flowName": qe.flowJobName, + "partitionID": qe.partitionID, + }).Infof("[pg_query_executor] declared cursor '%s' for query '%s'", cursorName, query) totalRecordsFetched := 0 numFetchOpsComplete := 0 @@ -316,7 +361,10 @@ func (qe *QRepQueryExecutor) ExecuteAndProcessQueryStream( return 0, err } - log.Infof("[pg_query_executor] fetched %d rows for query '%s'", numRows, query) + log.WithFields(log.Fields{ + "flowName": qe.flowJobName, + "partitionID": qe.partitionID, + }).Infof("[pg_query_executor] fetched %d rows for query '%s'", numRows, query) totalRecordsFetched += numRows if numRows == 0 { @@ -335,7 +383,10 @@ func (qe *QRepQueryExecutor) ExecuteAndProcessQueryStream( return 0, fmt.Errorf("[pg_query_executor] failed to commit transaction: %w", err) } - log.Infof("[pg_query_executor] committed transaction for query '%s', rows = %d", + log.WithFields(log.Fields{ + "flowName": qe.flowJobName, + "partitionID": qe.partitionID, + }).Infof("[pg_query_executor] committed transaction for query '%s', rows = %d", query, totalRecordsFetched) return totalRecordsFetched, nil } diff --git a/flow/connectors/postgres/qrep_query_executor_test.go b/flow/connectors/postgres/qrep_query_executor_test.go index c147a01f46..8b4ff0c116 100644 --- a/flow/connectors/postgres/qrep_query_executor_test.go +++ b/flow/connectors/postgres/qrep_query_executor_test.go @@ -49,7 +49,7 @@ func TestNewQRepQueryExecutor(t *testing.T) { defer teardownDB(t, pool, schema) ctx := context.Background() - qe := NewQRepQueryExecutor(pool, ctx) + qe := NewQRepQueryExecutor(pool, ctx, "test flow", "test part") if qe == nil { t.Fatalf("expected QRepQueryExecutor, got nil") @@ -64,7 +64,7 @@ func TestExecuteAndProcessQuery(t *testing.T) { ctx := context.Background() - qe := NewQRepQueryExecutor(pool, ctx) + qe := NewQRepQueryExecutor(pool, ctx, "test flow", "test part") qe.SetTestEnv(true) query := fmt.Sprintf("CREATE TABLE IF NOT EXISTS %s.test(id SERIAL PRIMARY KEY, data TEXT);", schemaName) @@ -104,7 +104,7 @@ func TestAllDataTypes(t *testing.T) { defer teardownDB(t, pool, schemaName) ctx := context.Background() - qe := NewQRepQueryExecutor(pool, ctx) + qe := NewQRepQueryExecutor(pool, ctx, "test flow", "test part") // Create a table that contains every data type we want to test query := fmt.Sprintf(` diff --git a/flow/connectors/postgres/qrep_sync_method.go b/flow/connectors/postgres/qrep_sync_method.go index 2f7b4030a4..1990dd95c5 100644 --- a/flow/connectors/postgres/qrep_sync_method.go +++ b/flow/connectors/postgres/qrep_sync_method.go @@ -54,7 +54,11 @@ func (s *QRepStagingTableSync) SyncQRepRecords( ) _, err = pool.Exec(context.Background(), tmpTableStmt) if err != nil { - log.Errorf( + log.WithFields(log.Fields{ + "flowName": flowJobName, + "partitionID": partitionID, + "destinationTable": dstTableName, + }).Errorf( "failed to create staging temporary table %s, statement: '%s'. Error: %v", stagingTable, tmpTableStmt, @@ -65,7 +69,11 @@ func (s *QRepStagingTableSync) SyncQRepRecords( schema, err := stream.Schema() if err != nil { - log.Errorf("failed to get schema from stream: %v", err) + log.WithFields(log.Fields{ + "flowName": flowJobName, + "destinationTable": dstTableName, + "partitionID": partitionID, + }).Errorf("failed to get schema from stream: %v", err) return 0, fmt.Errorf("failed to get schema from stream: %w", err) } @@ -94,7 +102,11 @@ func (s *QRepStagingTableSync) SyncQRepRecords( defer func() { if err := tx2.Rollback(context.Background()); err != nil { if err != pgx.ErrTxClosed { - log.Errorf("failed to rollback transaction tx2: %v", err) + log.WithFields(log.Fields{ + "flowName": flowJobName, + "partitionID": partitionID, + "destinationTable": dstTableName, + }).Errorf("failed to rollback transaction tx2: %v", err) } } }() @@ -105,7 +117,10 @@ func (s *QRepStagingTableSync) SyncQRepRecords( colNames[i] = fmt.Sprintf("\"%s\"", colName) } colNamesStr := strings.Join(colNames, ", ") - + log.WithFields(log.Fields{ + "flowName": flowJobName, + "partitionID": partitionID, + }).Infof("Obtained column names and quoted them in QRep sync") insertFromStagingStmt := fmt.Sprintf( "INSERT INTO %s (%s) SELECT %s FROM %s", dstTableName.String(), @@ -116,7 +131,11 @@ func (s *QRepStagingTableSync) SyncQRepRecords( _, err = tx2.Exec(context.Background(), insertFromStagingStmt) if err != nil { - log.Errorf("failed to execute statement '%s': %v", insertFromStagingStmt, err) + log.WithFields(log.Fields{ + "flowName": flowJobName, + "partitionID": partitionID, + "destinationTable": dstTableName, + }).Errorf("failed to execute statement '%s': %v", insertFromStagingStmt, err) return -1, fmt.Errorf("failed to execute statements in a transaction: %v", err) } @@ -131,6 +150,11 @@ func (s *QRepStagingTableSync) SyncQRepRecords( "INSERT INTO %s VALUES ($1, $2, $3, $4, $5);", qRepMetadataTableName, ) + log.WithFields(log.Fields{ + "flowName": flowJobName, + "partitionID": partitionID, + "destinationTable": dstTableName, + }).Infof("Executing transaction inside Qrep sync") rows, err := tx2.Exec( context.Background(), insertMetadataStmt, @@ -156,6 +180,9 @@ func (s *QRepStagingTableSync) SyncQRepRecords( } numRowsInserted := copySource.NumRecords() - log.Printf("pushed %d records to %s", numRowsInserted, dstTableName) + log.WithFields(log.Fields{ + "flowName": flowJobName, + "partitionID": partitionID, + }).Infof("pushed %d records to %s", numRowsInserted, dstTableName) return numRowsInserted, nil } diff --git a/flow/connectors/s3/qrep.go b/flow/connectors/s3/qrep.go index 29fbb22e20..ba051e0c66 100644 --- a/flow/connectors/s3/qrep.go +++ b/flow/connectors/s3/qrep.go @@ -29,7 +29,10 @@ func (c *S3Connector) SyncQRepRecords( ) (int, error) { schema, err := stream.Schema() if err != nil { - log.Errorf("failed to get schema from stream: %v", err) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + "partitionID": partition.PartitionId, + }).Errorf("failed to get schema from stream: %v", err) return 0, fmt.Errorf("failed to get schema from stream: %w", err) } diff --git a/flow/connectors/snowflake/qrep.go b/flow/connectors/snowflake/qrep.go index 16b2559010..3f795600ca 100644 --- a/flow/connectors/snowflake/qrep.go +++ b/flow/connectors/snowflake/qrep.go @@ -42,6 +42,12 @@ func (c *SnowflakeConnector) SyncQRepRecords( if err != nil { return 0, fmt.Errorf("failed to get schema of table %s: %w", destTable, err) } + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + "partition": partition.PartitionId, + }).Infof("Called QRep sync function and "+ + "obtained table schema for destination table %s", + destTable) done, err := c.isPartitionSynced(partition.PartitionId) if err != nil { @@ -49,7 +55,9 @@ func (c *SnowflakeConnector) SyncQRepRecords( } if done { - log.Infof("Partition %s has already been synced", partition.PartitionId) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Infof("Partition %s has already been synced", partition.PartitionId) return 0, nil } @@ -187,24 +195,32 @@ func (c *SnowflakeConnector) createStage(stageName string, config *protos.QRepCo // Execute the query _, err := c.database.Exec(createStageStmt) if err != nil { - log.Errorf("failed to create stage %s: %v", stageName, err) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Errorf("failed to create stage %s: %v", stageName, err) return fmt.Errorf("failed to create stage %s: %w", stageName, err) } - log.Infof("Created stage %s", stageName) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Infof("Created stage %s", stageName) return nil } func (c *SnowflakeConnector) createExternalStage(stageName string, config *protos.QRepConfig) (string, error) { awsCreds, err := utils.GetAWSSecrets() if err != nil { - log.Errorf("failed to get AWS secrets: %v", err) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Errorf("failed to get AWS secrets: %v", err) return "", fmt.Errorf("failed to get AWS secrets: %w", err) } s3o, err := utils.NewS3BucketAndPrefix(config.StagingPath) if err != nil { - log.Errorf("failed to extract S3 bucket and prefix: %v", err) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Errorf("failed to extract S3 bucket and prefix: %v", err) return "", fmt.Errorf("failed to extract S3 bucket and prefix: %w", err) } @@ -244,13 +260,17 @@ func (c *SnowflakeConnector) ConsolidateQRepPartitions(config *protos.QRepConfig case protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO: allCols, err := c.getColsFromTable(destTable) if err != nil { - log.Errorf("failed to get columns from table %s: %v", destTable, err) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Errorf("failed to get columns from table %s: %v", destTable, err) return fmt.Errorf("failed to get columns from table %s: %w", destTable, err) } err = CopyStageToDestination(c, config, destTable, stageName, allCols) if err != nil { - log.Errorf("failed to copy stage to destination: %v", err) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Errorf("failed to copy stage to destination: %v", err) return fmt.Errorf("failed to copy stage to destination: %w", err) } @@ -262,7 +282,9 @@ func (c *SnowflakeConnector) ConsolidateQRepPartitions(config *protos.QRepConfig // CleanupQRepFlow function for snowflake connector func (c *SnowflakeConnector) CleanupQRepFlow(config *protos.QRepConfig) error { - log.Infof("Cleaning up flow job %s", config.FlowJobName) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Infof("Cleaning up flow job %s", config.FlowJobName) return c.dropStage(config.StagingPath, config.FlowJobName) } @@ -316,7 +338,9 @@ func (c *SnowflakeConnector) dropStage(stagingPath string, job string) error { if strings.HasPrefix(stagingPath, "s3://") { s3o, err := utils.NewS3BucketAndPrefix(stagingPath) if err != nil { - log.Errorf("failed to create S3 bucket and prefix: %v", err) + log.WithFields(log.Fields{ + "flowName": job, + }).Errorf("failed to create S3 bucket and prefix: %v", err) return fmt.Errorf("failed to create S3 bucket and prefix: %w", err) } @@ -325,7 +349,9 @@ func (c *SnowflakeConnector) dropStage(stagingPath string, job string) error { // deleting the contents of the bucket with prefix s3svc, err := utils.CreateS3Client() if err != nil { - log.Errorf("failed to create S3 client: %v", err) + log.WithFields(log.Fields{ + "flowName": job, + }).Errorf("failed to create S3 client: %v", err) return fmt.Errorf("failed to create S3 client: %w", err) } @@ -338,14 +364,18 @@ func (c *SnowflakeConnector) dropStage(stagingPath string, job string) error { // Iterate through the objects in the bucket with the prefix and delete them s3Client := s3manager.NewBatchDeleteWithClient(s3svc) if err := s3Client.Delete(aws.BackgroundContext(), iter); err != nil { - log.Errorf("failed to delete objects from bucket: %v", err) + log.WithFields(log.Fields{ + "flowName": job, + }).Errorf("failed to delete objects from bucket: %v", err) return fmt.Errorf("failed to delete objects from bucket: %w", err) } log.Infof("Deleted contents of bucket %s with prefix %s/%s", s3o.Bucket, s3o.Prefix, job) } - log.Infof("Dropped stage %s", stageName) + log.WithFields(log.Fields{ + "flowName": job, + }).Infof("Dropped stage %s", stageName) return nil } diff --git a/flow/connectors/snowflake/qrep_avro_sync.go b/flow/connectors/snowflake/qrep_avro_sync.go index b1e96968de..6abfe71fa9 100644 --- a/flow/connectors/snowflake/qrep_avro_sync.go +++ b/flow/connectors/snowflake/qrep_avro_sync.go @@ -35,6 +35,7 @@ func NewSnowflakeAvroSyncMethod( func (s *SnowflakeAvroSyncMethod) SyncRecords( dstTableSchema []*sql.ColumnType, stream *model.QRecordStream, + flowJobName string, ) (int, error) { dstTableName := s.config.DestinationTableIdentifier @@ -43,22 +44,34 @@ func (s *SnowflakeAvroSyncMethod) SyncRecords( return -1, fmt.Errorf("failed to get schema from stream: %w", err) } - avroSchema, err := s.getAvroSchema(dstTableName, schema) + log.WithFields(log.Fields{ + "destinationTable": dstTableName, + "flowName": flowJobName, + }).Infof("sync function called and schema acquired") + + avroSchema, err := s.getAvroSchema(dstTableName, schema, flowJobName) if err != nil { return 0, err } - numRecords, localFilePath, err := s.writeToAvroFile(stream, avroSchema, "17") + numRecords, localFilePath, err := s.writeToAvroFile(stream, avroSchema, "17", flowJobName) if err != nil { return 0, err } - log.Infof("written %d records to Avro file", numRecords) + log.WithFields(log.Fields{ + "destinationTable": dstTableName, + "flowName": flowJobName, + }).Infof("written %d records to Avro file", numRecords) stage := s.connector.getStageNameForJob(s.config.FlowJobName) err = s.connector.createStage(stage, s.config) if err != nil { return 0, err } + log.WithFields(log.Fields{ + "destinationTable": dstTableName, + "flowName": flowJobName, + }).Infof("Created stage %s", stage) allCols, err := s.connector.getColsFromTable(s.config.DestinationTableIdentifier) if err != nil { @@ -69,13 +82,17 @@ func (s *SnowflakeAvroSyncMethod) SyncRecords( if err != nil { return 0, err } - log.Infof("pushed avro file to stage") + log.WithFields(log.Fields{ + "destinationTable": dstTableName, + }).Infof("pushed avro file to stage") err = CopyStageToDestination(s.connector, s.config, s.config.DestinationTableIdentifier, stage, allCols) if err != nil { return 0, err } - log.Infof("copying records into %s from stage %s", s.config.DestinationTableIdentifier, stage) + log.WithFields(log.Fields{ + "destinationTable": dstTableName, + }).Infof("copying records into %s from stage %s", s.config.DestinationTableIdentifier, stage) return numRecords, nil } @@ -93,13 +110,17 @@ func (s *SnowflakeAvroSyncMethod) SyncQRepRecords( if err != nil { return -1, fmt.Errorf("failed to get schema from stream: %w", err) } + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + "partitionID": partition.PartitionId, + }).Infof("sync function called and schema acquired") - avroSchema, err := s.getAvroSchema(dstTableName, schema) + avroSchema, err := s.getAvroSchema(dstTableName, schema, config.FlowJobName) if err != nil { return 0, err } - numRecords, localFilePath, err := s.writeToAvroFile(stream, avroSchema, partition.PartitionId) + numRecords, localFilePath, err := s.writeToAvroFile(stream, avroSchema, partition.PartitionId, config.FlowJobName) if err != nil { return 0, err } @@ -109,7 +130,11 @@ func (s *SnowflakeAvroSyncMethod) SyncQRepRecords( log.Infof("removing temp file %s", localFilePath) err := os.Remove(localFilePath) if err != nil { - log.Errorf("failed to remove temp file %s: %v", localFilePath, err) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + "partitionID": partition.PartitionId, + "destinationTable": dstTableName, + }).Errorf("failed to remove temp file %s: %v", localFilePath, err) } }() } @@ -121,6 +146,10 @@ func (s *SnowflakeAvroSyncMethod) SyncQRepRecords( if err != nil { return 0, err } + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + "partitionID": partition.PartitionId, + }).Infof("Put file to stage in Avro sync for snowflake") metrics.LogQRepSyncMetrics(s.connector.ctx, config.FlowJobName, int64(numRecords), time.Since(putFileStartTime)) @@ -137,13 +166,16 @@ func (s *SnowflakeAvroSyncMethod) SyncQRepRecords( func (s *SnowflakeAvroSyncMethod) getAvroSchema( dstTableName string, schema *model.QRecordSchema, + flowJobName string, ) (*model.QRecordAvroSchemaDefinition, error) { avroSchema, err := model.GetAvroSchemaDefinition(dstTableName, schema) if err != nil { return nil, fmt.Errorf("failed to define Avro schema: %w", err) } - log.Infof("Avro schema: %v\n", avroSchema) + log.WithFields(log.Fields{ + "flowName": flowJobName, + }).Infof("Avro schema: %v\n", avroSchema) return avroSchema, nil } @@ -151,6 +183,7 @@ func (s *SnowflakeAvroSyncMethod) writeToAvroFile( stream *model.QRecordStream, avroSchema *model.QRecordAvroSchemaDefinition, partitionID string, + flowJobName string, ) (int, string, error) { var numRecords int ocfWriter := avro.NewPeerDBOCFWriter(s.connector.ctx, stream, avroSchema) @@ -161,6 +194,10 @@ func (s *SnowflakeAvroSyncMethod) writeToAvroFile( } localFilePath := fmt.Sprintf("%s/%s.avro", tmpDir, partitionID) + log.WithFields(log.Fields{ + "flowName": flowJobName, + "partitionID": partitionID, + }).Infof("writing records to local file %s", localFilePath) numRecords, err = ocfWriter.WriteRecordsToAvroFile(localFilePath) if err != nil { return 0, "", fmt.Errorf("failed to write records to Avro file: %w", err) @@ -174,6 +211,10 @@ func (s *SnowflakeAvroSyncMethod) writeToAvroFile( } s3Key := fmt.Sprintf("%s/%s/%s.avro", s3o.Prefix, s.config.FlowJobName, partitionID) + log.WithFields(log.Fields{ + "flowName": flowJobName, + "partitionID": partitionID, + }).Infof("OCF: Writing records to S3") numRecords, err = ocfWriter.WriteRecordsToS3(s3o.Bucket, s3Key) if err != nil { return 0, "", fmt.Errorf("failed to write records to S3: %w", err) @@ -217,6 +258,9 @@ func CopyStageToDestination( stage string, allCols []string, ) error { + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Infof("Copying stage to destination %s", dstTableName) copyOpts := []string{ "FILE_FORMAT = (TYPE = AVRO)", "MATCH_BY_COLUMN_NAME='CASE_INSENSITIVE'", @@ -260,16 +304,25 @@ func (s *SnowflakeAvroSyncMethod) insertMetadata( ) error { insertMetadataStmt, err := s.connector.createMetadataInsertStatement(partition, flowJobName, startTime) if err != nil { - log.Errorf("failed to create metadata insert statement: %v", err) + log.WithFields(log.Fields{ + "flowName": flowJobName, + "partitionID": partition.PartitionId, + }).Errorf("failed to create metadata insert statement: %v", err) return fmt.Errorf("failed to create metadata insert statement: %v", err) } if _, err := s.connector.database.Exec(insertMetadataStmt); err != nil { - log.Errorf("failed to execute metadata insert statement '%s': %v", insertMetadataStmt, err) + log.WithFields(log.Fields{ + "flowName": flowJobName, + "partitionID": partition.PartitionId, + }).Errorf("failed to execute metadata insert statement '%s': %v", insertMetadataStmt, err) return fmt.Errorf("failed to execute metadata insert statement: %v", err) } - log.Infof("inserted metadata for partition %s", partition) + log.WithFields(log.Fields{ + "flowName": flowJobName, + "partitionID": partition.PartitionId, + }).Infof("inserted metadata for partition %s", partition) return nil } @@ -393,7 +446,9 @@ func (s *SnowflakeAvroWriteHandler) HandleUpsertMode( if _, err := s.connector.database.Exec(createTempTableCmd); err != nil { return fmt.Errorf("failed to create temp table: %w", err) } - log.Infof("created temp table %s", tempTableName) + log.WithFields(log.Fields{ + "flowName": flowJobName, + }).Infof("created temp table %s", tempTableName) //nolint:gosec copyCmd := fmt.Sprintf("COPY INTO %s FROM @%s %s", @@ -423,10 +478,14 @@ func (s *SnowflakeAvroWriteHandler) HandleUpsertMode( metrics.LogQRepNormalizeMetrics(s.connector.ctx, flowJobName, rowCount, time.Since(startTime), totalRowsAtTarget) } else { - log.Errorf("failed to get rows affected: %v", err) + log.WithFields(log.Fields{ + "flowName": flowJobName, + }).Errorf("failed to get rows affected: %v", err) } - log.Infof("merged data from temp table %s into destination table %s", + log.WithFields(log.Fields{ + "flowName": flowJobName, + }).Infof("merged data from temp table %s into destination table %s", tempTableName, s.dstTableName) return nil } diff --git a/flow/connectors/snowflake/snowflake.go b/flow/connectors/snowflake/snowflake.go index 4f79463d7c..7852eedc9c 100644 --- a/flow/connectors/snowflake/snowflake.go +++ b/flow/connectors/snowflake/snowflake.go @@ -396,7 +396,10 @@ func (c *SnowflakeConnector) SyncRecords(req *model.SyncRecordsRequest) (*model. defer func() { deferErr := syncRecordsTx.Rollback() if deferErr != sql.ErrTxDone && deferErr != nil { - log.Errorf("unexpected error while rolling back transaction for SyncRecords: %v", deferErr) + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + "syncBatchID": syncBatchID - 1, + }).Errorf("unexpected error while rolling back transaction for SyncRecords: %v", deferErr) } }() @@ -726,7 +729,7 @@ func (c *SnowflakeConnector) syncRecordsViaAvro(req *model.SyncRecordsRequest, r startTime := time.Now() close(recordStream.Records) - numRecords, err := avroSyncer.SyncRecords(destinationTableSchema, recordStream) + numRecords, err := avroSyncer.SyncRecords(destinationTableSchema, recordStream, req.FlowJobName) if err != nil { return nil, err } @@ -789,7 +792,9 @@ func (c *SnowflakeConnector) NormalizeRecords(req *model.NormalizeRecordsRequest defer func() { deferErr := normalizeRecordsTx.Rollback() if deferErr != sql.ErrTxDone && deferErr != nil { - log.Errorf("unexpected error while rolling back transaction for NormalizeRecords: %v", deferErr) + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + }).Errorf("unexpected error while rolling back transaction for NormalizeRecords: %v", deferErr) } }() @@ -890,7 +895,9 @@ func (c *SnowflakeConnector) SyncFlowCleanup(jobName string) error { defer func() { deferErr := syncFlowCleanupTx.Rollback() if deferErr != sql.ErrTxDone && deferErr != nil { - log.Errorf("unexpected error while rolling back transaction for flow cleanup: %v", deferErr) + log.WithFields(log.Fields{ + "flowName": jobName, + }).Errorf("unexpected error while rolling back transaction for flow cleanup: %v", deferErr) } }() diff --git a/flow/e2e/qrep_flow_test.go b/flow/e2e/qrep_flow_test.go index 4e04d1a17c..c79157bcfd 100644 --- a/flow/e2e/qrep_flow_test.go +++ b/flow/e2e/qrep_flow_test.go @@ -246,7 +246,7 @@ func (s *E2EPeerFlowTestSuite) createQRepWorkflowConfig( func (s *E2EPeerFlowTestSuite) compareTableContentsBQ(tableName string, colsString string) { // read rows from source table - pgQueryExecutor := connpostgres.NewQRepQueryExecutor(s.pool, context.Background()) + pgQueryExecutor := connpostgres.NewQRepQueryExecutor(s.pool, context.Background(), "testflow", "testpart") pgQueryExecutor.SetTestEnv(true) pgRows, err := pgQueryExecutor.ExecuteAndProcessQuery( @@ -266,7 +266,7 @@ func (s *E2EPeerFlowTestSuite) compareTableContentsBQ(tableName string, colsStri func (s *E2EPeerFlowTestSuite) compareTableContentsSF(tableName string, selector string, caseSensitive bool) { // read rows from source table - pgQueryExecutor := connpostgres.NewQRepQueryExecutor(s.pool, context.Background()) + pgQueryExecutor := connpostgres.NewQRepQueryExecutor(s.pool, context.Background(), "testflow", "testpart") pgQueryExecutor.SetTestEnv(true) pgRows, err := pgQueryExecutor.ExecuteAndProcessQuery( fmt.Sprintf("SELECT %s FROM e2e_test.%s ORDER BY id", selector, tableName), @@ -382,7 +382,8 @@ func (s *E2EPeerFlowTestSuite) Test_Complete_QRep_Flow_Avro() { query := fmt.Sprintf("SELECT * FROM e2e_test.%s WHERE updated_at BETWEEN {{.start}} AND {{.end}}", tblName) - qrepConfig := s.createQRepWorkflowConfig("test_qrep_flow_avro", + qrepConfig := s.createQRepWorkflowConfig( + "test_qrep_flow_avro", "e2e_test."+tblName, tblName, query, From 7e523da662571cfcd2ec8cbed0fa8ac3b09727be Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Tue, 29 Aug 2023 11:34:53 -0700 Subject: [PATCH 19/37] replicate partitions in batches (#361) --- flow/activities/flowable.go | 18 +++ flow/generated/protos/flow.pb.go | 250 +++++++++++++++++++----------- flow/workflows/qrep_flow.go | 82 +++++----- nexus/pt/src/peerdb_flow.rs | 8 + nexus/pt/src/peerdb_flow.serde.rs | 115 ++++++++++++++ protos/flow.proto | 5 + 6 files changed, 352 insertions(+), 126 deletions(-) diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index 5893b15859..a2d9014e73 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -372,6 +372,24 @@ func (a *FlowableActivity) GetQRepPartitions(ctx context.Context, }, nil } +// ReplicateQRepPartition replicates a QRepPartition from the source to the destination. +func (a *FlowableActivity) ReplicateQRepPartitions(ctx context.Context, + config *protos.QRepConfig, + partitions *protos.QRepPartitionBatch, + runUUID string, +) error { + log.Infof("replicating partitions for job - %s - batch %d\n", config.FlowJobName, partitions.BatchId) + for _, p := range partitions.Partitions { + log.Infof("replicating partition - %s\n", p.PartitionId) + err := a.ReplicateQRepPartition(ctx, config, p, runUUID) + if err != nil { + return err + } + } + + return nil +} + // ReplicateQRepPartition replicates a QRepPartition from the source to the destination. func (a *FlowableActivity) ReplicateQRepPartition(ctx context.Context, config *protos.QRepConfig, diff --git a/flow/generated/protos/flow.pb.go b/flow/generated/protos/flow.pb.go index be7e7ae680..00cb9fa2f6 100644 --- a/flow/generated/protos/flow.pb.go +++ b/flow/generated/protos/flow.pb.go @@ -2289,6 +2289,61 @@ func (x *QRepPartition) GetFullTablePartition() bool { return false } +type QRepPartitionBatch struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + BatchId int32 `protobuf:"varint,1,opt,name=batch_id,json=batchId,proto3" json:"batch_id,omitempty"` + Partitions []*QRepPartition `protobuf:"bytes,2,rep,name=partitions,proto3" json:"partitions,omitempty"` +} + +func (x *QRepPartitionBatch) Reset() { + *x = QRepPartitionBatch{} + if protoimpl.UnsafeEnabled { + mi := &file_flow_proto_msgTypes[33] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *QRepPartitionBatch) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QRepPartitionBatch) ProtoMessage() {} + +func (x *QRepPartitionBatch) ProtoReflect() protoreflect.Message { + mi := &file_flow_proto_msgTypes[33] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QRepPartitionBatch.ProtoReflect.Descriptor instead. +func (*QRepPartitionBatch) Descriptor() ([]byte, []int) { + return file_flow_proto_rawDescGZIP(), []int{33} +} + +func (x *QRepPartitionBatch) GetBatchId() int32 { + if x != nil { + return x.BatchId + } + return 0 +} + +func (x *QRepPartitionBatch) GetPartitions() []*QRepPartition { + if x != nil { + return x.Partitions + } + return nil +} + type QRepParitionResult struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -2300,7 +2355,7 @@ type QRepParitionResult struct { func (x *QRepParitionResult) Reset() { *x = QRepParitionResult{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[33] + mi := &file_flow_proto_msgTypes[34] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2313,7 +2368,7 @@ func (x *QRepParitionResult) String() string { func (*QRepParitionResult) ProtoMessage() {} func (x *QRepParitionResult) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[33] + mi := &file_flow_proto_msgTypes[34] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2326,7 +2381,7 @@ func (x *QRepParitionResult) ProtoReflect() protoreflect.Message { // Deprecated: Use QRepParitionResult.ProtoReflect.Descriptor instead. func (*QRepParitionResult) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{33} + return file_flow_proto_rawDescGZIP(), []int{34} } func (x *QRepParitionResult) GetPartitions() []*QRepPartition { @@ -2347,7 +2402,7 @@ type DropFlowInput struct { func (x *DropFlowInput) Reset() { *x = DropFlowInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[34] + mi := &file_flow_proto_msgTypes[35] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2360,7 +2415,7 @@ func (x *DropFlowInput) String() string { func (*DropFlowInput) ProtoMessage() {} func (x *DropFlowInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[34] + mi := &file_flow_proto_msgTypes[35] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2373,7 +2428,7 @@ func (x *DropFlowInput) ProtoReflect() protoreflect.Message { // Deprecated: Use DropFlowInput.ProtoReflect.Descriptor instead. func (*DropFlowInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{34} + return file_flow_proto_rawDescGZIP(), []int{35} } func (x *DropFlowInput) GetFlowName() string { @@ -2863,33 +2918,40 @@ var file_flow_proto_rawDesc = []byte{ 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, 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, 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, 0x47, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, - 0x65, 0x12, 0x1a, 0x0a, 0x16, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x5f, - 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x41, 0x50, 0x50, 0x45, 0x4e, 0x44, 0x10, 0x00, 0x12, 0x1a, 0x0a, - 0x16, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x5f, 0x4d, 0x4f, 0x44, 0x45, - 0x5f, 0x55, 0x50, 0x53, 0x45, 0x52, 0x54, 0x10, 0x01, 0x42, 0x76, 0x0a, 0x0f, 0x63, 0x6f, 0x6d, - 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x42, 0x09, 0x46, 0x6c, - 0x6f, 0x77, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x10, 0x67, 0x65, 0x6e, 0x65, 0x72, - 0x61, 0x74, 0x65, 0x64, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x73, 0xa2, 0x02, 0x03, 0x50, 0x58, - 0x58, 0xaa, 0x02, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0xca, 0x02, - 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0xe2, 0x02, 0x16, 0x50, 0x65, - 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, - 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, - 0x77, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x6b, + 0x0a, 0x12, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x42, + 0x61, 0x74, 0x63, 0x68, 0x12, 0x19, 0x0a, 0x08, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x62, 0x61, 0x74, 0x63, 0x68, 0x49, 0x64, 0x12, + 0x3a, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, + 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x50, 0x0a, 0x12, 0x51, + 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c, + 0x74, 0x12, 0x3a, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, + 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, + 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x2c, 0x0a, + 0x0d, 0x44, 0x72, 0x6f, 0x70, 0x46, 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x1b, + 0x0a, 0x09, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x08, 0x66, 0x6c, 0x6f, 0x77, 0x4e, 0x61, 0x6d, 0x65, 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, 0x47, 0x0a, + 0x0d, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1a, + 0x0a, 0x16, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x5f, 0x4d, 0x4f, 0x44, + 0x45, 0x5f, 0x41, 0x50, 0x50, 0x45, 0x4e, 0x44, 0x10, 0x00, 0x12, 0x1a, 0x0a, 0x16, 0x51, 0x52, + 0x45, 0x50, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x55, 0x50, + 0x53, 0x45, 0x52, 0x54, 0x10, 0x01, 0x42, 0x76, 0x0a, 0x0f, 0x63, 0x6f, 0x6d, 0x2e, 0x70, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x42, 0x09, 0x46, 0x6c, 0x6f, 0x77, 0x50, + 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x10, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, + 0x64, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x73, 0xa2, 0x02, 0x03, 0x50, 0x58, 0x58, 0xaa, 0x02, + 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0xca, 0x02, 0x0a, 0x50, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0xe2, 0x02, 0x16, 0x50, 0x65, 0x65, 0x72, 0x64, + 0x62, 0x46, 0x6c, 0x6f, 0x77, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, + 0x61, 0xea, 0x02, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0x62, 0x06, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -2905,7 +2967,7 @@ func file_flow_proto_rawDescGZIP() []byte { } var file_flow_proto_enumTypes = make([]protoimpl.EnumInfo, 2) -var file_flow_proto_msgTypes = make([]protoimpl.MessageInfo, 45) +var file_flow_proto_msgTypes = make([]protoimpl.MessageInfo, 46) var file_flow_proto_goTypes = []interface{}{ (QRepSyncMode)(0), // 0: peerdb_flow.QRepSyncMode (QRepWriteType)(0), // 1: peerdb_flow.QRepWriteType @@ -2942,79 +3004,81 @@ var file_flow_proto_goTypes = []interface{}{ (*QRepWriteMode)(nil), // 32: peerdb_flow.QRepWriteMode (*QRepConfig)(nil), // 33: peerdb_flow.QRepConfig (*QRepPartition)(nil), // 34: peerdb_flow.QRepPartition - (*QRepParitionResult)(nil), // 35: peerdb_flow.QRepParitionResult - (*DropFlowInput)(nil), // 36: peerdb_flow.DropFlowInput - nil, // 37: peerdb_flow.FlowConnectionConfigs.TableNameMappingEntry - nil, // 38: peerdb_flow.FlowConnectionConfigs.SrcTableIdNameMappingEntry - nil, // 39: peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry - nil, // 40: peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry - nil, // 41: peerdb_flow.SetupReplicationInput.TableNameMappingEntry - nil, // 42: peerdb_flow.CreateRawTableInput.TableNameMappingEntry - nil, // 43: peerdb_flow.TableSchema.ColumnsEntry - nil, // 44: peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry - nil, // 45: peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry - nil, // 46: peerdb_flow.SetupNormalizedTableBatchOutput.TableExistsMappingEntry - (*Peer)(nil), // 47: peerdb_peers.Peer - (*timestamppb.Timestamp)(nil), // 48: google.protobuf.Timestamp + (*QRepPartitionBatch)(nil), // 35: peerdb_flow.QRepPartitionBatch + (*QRepParitionResult)(nil), // 36: peerdb_flow.QRepParitionResult + (*DropFlowInput)(nil), // 37: peerdb_flow.DropFlowInput + nil, // 38: peerdb_flow.FlowConnectionConfigs.TableNameMappingEntry + nil, // 39: peerdb_flow.FlowConnectionConfigs.SrcTableIdNameMappingEntry + nil, // 40: peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry + nil, // 41: peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry + nil, // 42: peerdb_flow.SetupReplicationInput.TableNameMappingEntry + nil, // 43: peerdb_flow.CreateRawTableInput.TableNameMappingEntry + nil, // 44: peerdb_flow.TableSchema.ColumnsEntry + nil, // 45: peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry + nil, // 46: peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry + nil, // 47: peerdb_flow.SetupNormalizedTableBatchOutput.TableExistsMappingEntry + (*Peer)(nil), // 48: peerdb_peers.Peer + (*timestamppb.Timestamp)(nil), // 49: google.protobuf.Timestamp } var file_flow_proto_depIdxs = []int32{ - 47, // 0: peerdb_flow.FlowConnectionConfigs.source:type_name -> peerdb_peers.Peer - 47, // 1: peerdb_flow.FlowConnectionConfigs.destination:type_name -> peerdb_peers.Peer + 48, // 0: peerdb_flow.FlowConnectionConfigs.source:type_name -> peerdb_peers.Peer + 48, // 1: peerdb_flow.FlowConnectionConfigs.destination:type_name -> peerdb_peers.Peer 20, // 2: peerdb_flow.FlowConnectionConfigs.table_schema:type_name -> peerdb_flow.TableSchema - 37, // 3: peerdb_flow.FlowConnectionConfigs.table_name_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.TableNameMappingEntry - 38, // 4: peerdb_flow.FlowConnectionConfigs.src_table_id_name_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.SrcTableIdNameMappingEntry - 39, // 5: peerdb_flow.FlowConnectionConfigs.table_name_schema_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry - 47, // 6: peerdb_flow.FlowConnectionConfigs.metadata_peer:type_name -> peerdb_peers.Peer + 38, // 3: peerdb_flow.FlowConnectionConfigs.table_name_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.TableNameMappingEntry + 39, // 4: peerdb_flow.FlowConnectionConfigs.src_table_id_name_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.SrcTableIdNameMappingEntry + 40, // 5: peerdb_flow.FlowConnectionConfigs.table_name_schema_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry + 48, // 6: peerdb_flow.FlowConnectionConfigs.metadata_peer:type_name -> peerdb_peers.Peer 0, // 7: peerdb_flow.FlowConnectionConfigs.snapshot_sync_mode:type_name -> peerdb_flow.QRepSyncMode 0, // 8: peerdb_flow.FlowConnectionConfigs.cdc_sync_mode:type_name -> peerdb_flow.QRepSyncMode - 48, // 9: peerdb_flow.LastSyncState.last_synced_at:type_name -> google.protobuf.Timestamp + 49, // 9: peerdb_flow.LastSyncState.last_synced_at:type_name -> google.protobuf.Timestamp 6, // 10: peerdb_flow.StartFlowInput.last_sync_state:type_name -> peerdb_flow.LastSyncState 3, // 11: peerdb_flow.StartFlowInput.flow_connection_configs:type_name -> peerdb_flow.FlowConnectionConfigs 4, // 12: peerdb_flow.StartFlowInput.sync_flow_options:type_name -> peerdb_flow.SyncFlowOptions 3, // 13: peerdb_flow.StartNormalizeInput.flow_connection_configs:type_name -> peerdb_flow.FlowConnectionConfigs - 47, // 14: peerdb_flow.GetLastSyncedIDInput.peer_connection_config:type_name -> peerdb_peers.Peer - 47, // 15: peerdb_flow.EnsurePullabilityInput.peer_connection_config:type_name -> peerdb_peers.Peer - 47, // 16: peerdb_flow.EnsurePullabilityBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer + 48, // 14: peerdb_flow.GetLastSyncedIDInput.peer_connection_config:type_name -> peerdb_peers.Peer + 48, // 15: peerdb_flow.EnsurePullabilityInput.peer_connection_config:type_name -> peerdb_peers.Peer + 48, // 16: peerdb_flow.EnsurePullabilityBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer 12, // 17: peerdb_flow.TableIdentifier.postgres_table_identifier:type_name -> peerdb_flow.PostgresTableIdentifier 13, // 18: peerdb_flow.EnsurePullabilityOutput.table_identifier:type_name -> peerdb_flow.TableIdentifier - 40, // 19: peerdb_flow.EnsurePullabilityBatchOutput.table_identifier_mapping:type_name -> peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry - 47, // 20: peerdb_flow.SetupReplicationInput.peer_connection_config:type_name -> peerdb_peers.Peer - 41, // 21: peerdb_flow.SetupReplicationInput.table_name_mapping:type_name -> peerdb_flow.SetupReplicationInput.TableNameMappingEntry - 47, // 22: peerdb_flow.SetupReplicationInput.destination_peer:type_name -> peerdb_peers.Peer - 47, // 23: peerdb_flow.CreateRawTableInput.peer_connection_config:type_name -> peerdb_peers.Peer - 42, // 24: peerdb_flow.CreateRawTableInput.table_name_mapping:type_name -> peerdb_flow.CreateRawTableInput.TableNameMappingEntry + 41, // 19: peerdb_flow.EnsurePullabilityBatchOutput.table_identifier_mapping:type_name -> peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry + 48, // 20: peerdb_flow.SetupReplicationInput.peer_connection_config:type_name -> peerdb_peers.Peer + 42, // 21: peerdb_flow.SetupReplicationInput.table_name_mapping:type_name -> peerdb_flow.SetupReplicationInput.TableNameMappingEntry + 48, // 22: peerdb_flow.SetupReplicationInput.destination_peer:type_name -> peerdb_peers.Peer + 48, // 23: peerdb_flow.CreateRawTableInput.peer_connection_config:type_name -> peerdb_peers.Peer + 43, // 24: peerdb_flow.CreateRawTableInput.table_name_mapping:type_name -> peerdb_flow.CreateRawTableInput.TableNameMappingEntry 0, // 25: peerdb_flow.CreateRawTableInput.cdc_sync_mode:type_name -> peerdb_flow.QRepSyncMode - 43, // 26: peerdb_flow.TableSchema.columns:type_name -> peerdb_flow.TableSchema.ColumnsEntry - 47, // 27: peerdb_flow.GetTableSchemaBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer - 44, // 28: peerdb_flow.GetTableSchemaBatchOutput.table_name_schema_mapping:type_name -> peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry - 47, // 29: peerdb_flow.SetupNormalizedTableInput.peer_connection_config:type_name -> peerdb_peers.Peer + 44, // 26: peerdb_flow.TableSchema.columns:type_name -> peerdb_flow.TableSchema.ColumnsEntry + 48, // 27: peerdb_flow.GetTableSchemaBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer + 45, // 28: peerdb_flow.GetTableSchemaBatchOutput.table_name_schema_mapping:type_name -> peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry + 48, // 29: peerdb_flow.SetupNormalizedTableInput.peer_connection_config:type_name -> peerdb_peers.Peer 20, // 30: peerdb_flow.SetupNormalizedTableInput.source_table_schema:type_name -> peerdb_flow.TableSchema - 47, // 31: peerdb_flow.SetupNormalizedTableBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer - 45, // 32: peerdb_flow.SetupNormalizedTableBatchInput.table_name_schema_mapping:type_name -> peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry - 46, // 33: peerdb_flow.SetupNormalizedTableBatchOutput.table_exists_mapping:type_name -> peerdb_flow.SetupNormalizedTableBatchOutput.TableExistsMappingEntry - 48, // 34: peerdb_flow.TimestampPartitionRange.start:type_name -> google.protobuf.Timestamp - 48, // 35: peerdb_flow.TimestampPartitionRange.end:type_name -> google.protobuf.Timestamp + 48, // 31: peerdb_flow.SetupNormalizedTableBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer + 46, // 32: peerdb_flow.SetupNormalizedTableBatchInput.table_name_schema_mapping:type_name -> peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry + 47, // 33: peerdb_flow.SetupNormalizedTableBatchOutput.table_exists_mapping:type_name -> peerdb_flow.SetupNormalizedTableBatchOutput.TableExistsMappingEntry + 49, // 34: peerdb_flow.TimestampPartitionRange.start:type_name -> google.protobuf.Timestamp + 49, // 35: peerdb_flow.TimestampPartitionRange.end:type_name -> google.protobuf.Timestamp 29, // 36: peerdb_flow.TIDPartitionRange.start:type_name -> peerdb_flow.TID 29, // 37: peerdb_flow.TIDPartitionRange.end:type_name -> peerdb_flow.TID 27, // 38: peerdb_flow.PartitionRange.int_range:type_name -> peerdb_flow.IntPartitionRange 28, // 39: peerdb_flow.PartitionRange.timestamp_range:type_name -> peerdb_flow.TimestampPartitionRange 30, // 40: peerdb_flow.PartitionRange.tid_range:type_name -> peerdb_flow.TIDPartitionRange 1, // 41: peerdb_flow.QRepWriteMode.write_type:type_name -> peerdb_flow.QRepWriteType - 47, // 42: peerdb_flow.QRepConfig.source_peer:type_name -> peerdb_peers.Peer - 47, // 43: peerdb_flow.QRepConfig.destination_peer:type_name -> peerdb_peers.Peer + 48, // 42: peerdb_flow.QRepConfig.source_peer:type_name -> peerdb_peers.Peer + 48, // 43: peerdb_flow.QRepConfig.destination_peer:type_name -> peerdb_peers.Peer 0, // 44: peerdb_flow.QRepConfig.sync_mode:type_name -> peerdb_flow.QRepSyncMode 32, // 45: peerdb_flow.QRepConfig.write_mode:type_name -> peerdb_flow.QRepWriteMode 31, // 46: peerdb_flow.QRepPartition.range:type_name -> peerdb_flow.PartitionRange - 34, // 47: peerdb_flow.QRepParitionResult.partitions:type_name -> peerdb_flow.QRepPartition - 20, // 48: peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry.value:type_name -> peerdb_flow.TableSchema - 13, // 49: peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry.value:type_name -> peerdb_flow.TableIdentifier - 20, // 50: peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry.value:type_name -> peerdb_flow.TableSchema - 20, // 51: peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry.value:type_name -> peerdb_flow.TableSchema - 52, // [52:52] is the sub-list for method output_type - 52, // [52:52] is the sub-list for method input_type - 52, // [52:52] is the sub-list for extension type_name - 52, // [52:52] is the sub-list for extension extendee - 0, // [0:52] is the sub-list for field type_name + 34, // 47: peerdb_flow.QRepPartitionBatch.partitions:type_name -> peerdb_flow.QRepPartition + 34, // 48: peerdb_flow.QRepParitionResult.partitions:type_name -> peerdb_flow.QRepPartition + 20, // 49: peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry.value:type_name -> peerdb_flow.TableSchema + 13, // 50: peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry.value:type_name -> peerdb_flow.TableIdentifier + 20, // 51: peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry.value:type_name -> peerdb_flow.TableSchema + 20, // 52: peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry.value:type_name -> peerdb_flow.TableSchema + 53, // [53:53] is the sub-list for method output_type + 53, // [53:53] is the sub-list for method input_type + 53, // [53:53] is the sub-list for extension type_name + 53, // [53:53] is the sub-list for extension extendee + 0, // [0:53] is the sub-list for field type_name } func init() { file_flow_proto_init() } @@ -3421,7 +3485,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*QRepParitionResult); i { + switch v := v.(*QRepPartitionBatch); i { case 0: return &v.state case 1: @@ -3433,6 +3497,18 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*QRepParitionResult); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_flow_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*DropFlowInput); i { case 0: return &v.state @@ -3459,7 +3535,7 @@ func file_flow_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_flow_proto_rawDesc, NumEnums: 2, - NumMessages: 45, + NumMessages: 46, NumExtensions: 0, NumServices: 0, }, diff --git a/flow/workflows/qrep_flow.go b/flow/workflows/qrep_flow.go index 9025fbe3ea..5a3b704e50 100644 --- a/flow/workflows/qrep_flow.go +++ b/flow/workflows/qrep_flow.go @@ -3,10 +3,8 @@ package peerflow import ( "fmt" - "math/rand" "time" - "github.com/PeerDB-io/peer-flow/concurrency" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/google/uuid" "go.temporal.io/api/enums/v1" @@ -70,10 +68,8 @@ func (q *QRepFlowExecution) GetPartitions( return partitions, nil } -// ReplicateParititon replicates the given partition. -func (q *QRepFlowExecution) ReplicatePartition(ctx workflow.Context, partition *protos.QRepPartition) error { - q.logger.Info("replicating partition - ", partition.PartitionId) - +// ReplicatePartitions replicates the partition batch. +func (q *QRepFlowExecution) ReplicatePartitions(ctx workflow.Context, partitions *protos.QRepPartitionBatch) error { ctx = workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ StartToCloseTimeout: 24 * 5 * time.Hour, RetryPolicy: &temporal.RetryPolicy{ @@ -82,12 +78,12 @@ func (q *QRepFlowExecution) ReplicatePartition(ctx workflow.Context, partition * HeartbeatTimeout: 1 * time.Hour, }) + q.logger.Info("replicating partition", "partition", partitions.BatchId) if err := workflow.ExecuteActivity(ctx, - flowable.ReplicateQRepPartition, q.config, partition, q.runUUID).Get(ctx, nil); err != nil { + flowable.ReplicateQRepPartitions, q.config, partitions, q.runUUID).Get(ctx, nil); err != nil { return fmt.Errorf("failed to replicate partition: %w", err) } - q.logger.Info("replicated partition - ", partition.PartitionId) return nil } @@ -107,12 +103,11 @@ func (q *QRepFlowExecution) getPartitionWorkflowID(ctx workflow.Context) (string // startChildWorkflow starts a single child workflow. func (q *QRepFlowExecution) startChildWorkflow( - boundSelector *concurrency.BoundSelector, ctx workflow.Context, - partition *protos.QRepPartition) error { + partitions *protos.QRepPartitionBatch) (workflow.Future, error) { wid, err := q.getPartitionWorkflowID(ctx) if err != nil { - return fmt.Errorf("failed to get child workflow ID: %w", err) + return nil, fmt.Errorf("failed to get child workflow ID: %w", err) } partFlowCtx := workflow.WithChildOptions(ctx, workflow.ChildWorkflowOptions{ WorkflowID: wid, @@ -122,15 +117,10 @@ func (q *QRepFlowExecution) startChildWorkflow( }, }) - boundSelector.SpawnChild( - partFlowCtx, - QRepPartitionWorkflow, - q.config, - partition, - q.runUUID, - ) + future := workflow.ExecuteChildWorkflow( + partFlowCtx, QRepPartitionWorkflow, q.config, partitions, q.runUUID) - return nil + return future, nil } // processPartitions handles the logic for processing the partitions. @@ -139,18 +129,42 @@ func (q *QRepFlowExecution) processPartitions( maxParallelWorkers int, partitions []*protos.QRepPartition, ) error { - boundSelector := concurrency.NewBoundSelector(maxParallelWorkers, len(partitions), ctx) + chunkSize := len(partitions) / maxParallelWorkers + if chunkSize == 0 { + chunkSize = 1 + } + + batches := make([][]*protos.QRepPartition, 0) + for i := 0; i < len(partitions); i += chunkSize { + end := i + chunkSize + if end > len(partitions) { + end = len(partitions) + } + + batches = append(batches, partitions[i:end]) + } + + q.logger.Info("processing partitions in batches", "num batches", len(batches)) - for _, partition := range partitions { - err := q.startChildWorkflow(boundSelector, ctx, partition) + futures := make([]workflow.Future, 0) + for i, parts := range batches { + batch := &protos.QRepPartitionBatch{ + Partitions: parts, + BatchId: int32(i), + } + future, err := q.startChildWorkflow(ctx, batch) if err != nil { - return err + return fmt.Errorf("failed to start child workflow: %w", err) } + + futures = append(futures, future) } - err := boundSelector.Wait() - if err != nil { - return fmt.Errorf("failed to process partitions: %w", err) + // wait for all the child workflows to complete + for _, future := range futures { + if err := future.Get(ctx, nil); err != nil { + return fmt.Errorf("failed to wait for child workflow: %w", err) + } } q.logger.Info("all partitions in batch processed") @@ -261,16 +275,6 @@ func QRepFlowWorkflow( return fmt.Errorf("failed to get partitions: %w", err) } - workflow.SideEffect(ctx, func(ctx workflow.Context) interface{} { - numPartitions := len(partitions.Partitions) - if numPartitions > 0 { - rand.Shuffle(len(partitions.Partitions), func(i, j int) { - partitions.Partitions[i], partitions.Partitions[j] = partitions.Partitions[j], partitions.Partitions[i] - }) - } - return nil - }) - logger.Info("partitions to replicate - ", len(partitions.Partitions)) if err = q.processPartitions(ctx, maxParallelWorkers, partitions.Partitions); err != nil { return err @@ -315,13 +319,13 @@ func QRepFlowWorkflow( return workflow.NewContinueAsNewError(ctx, QRepFlowWorkflow, config, lastPartition, numPartitionsProcessed) } -// QRepPartitionWorkflow replicate a single partition. +// QRepPartitionWorkflow replicate a partition batch func QRepPartitionWorkflow( ctx workflow.Context, config *protos.QRepConfig, - partition *protos.QRepPartition, + partitions *protos.QRepPartitionBatch, runUUID string, ) error { q := NewQRepFlowExecution(ctx, config, runUUID) - return q.ReplicatePartition(ctx, partition) + return q.ReplicatePartitions(ctx, partitions) } diff --git a/nexus/pt/src/peerdb_flow.rs b/nexus/pt/src/peerdb_flow.rs index 3da275341f..6e996b372f 100644 --- a/nexus/pt/src/peerdb_flow.rs +++ b/nexus/pt/src/peerdb_flow.rs @@ -375,6 +375,14 @@ pub struct QRepPartition { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct QRepPartitionBatch { + #[prost(int32, tag="1")] + pub batch_id: i32, + #[prost(message, repeated, tag="2")] + pub partitions: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct QRepParitionResult { #[prost(message, repeated, tag="1")] pub partitions: ::prost::alloc::vec::Vec, diff --git a/nexus/pt/src/peerdb_flow.serde.rs b/nexus/pt/src/peerdb_flow.serde.rs index 0ec540285a..2d2dc32e9f 100644 --- a/nexus/pt/src/peerdb_flow.serde.rs +++ b/nexus/pt/src/peerdb_flow.serde.rs @@ -2743,6 +2743,121 @@ impl<'de> serde::Deserialize<'de> for QRepPartition { deserializer.deserialize_struct("peerdb_flow.QRepPartition", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for QRepPartitionBatch { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.batch_id != 0 { + len += 1; + } + if !self.partitions.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_flow.QRepPartitionBatch", len)?; + if self.batch_id != 0 { + struct_ser.serialize_field("batchId", &self.batch_id)?; + } + if !self.partitions.is_empty() { + struct_ser.serialize_field("partitions", &self.partitions)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for QRepPartitionBatch { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "batch_id", + "batchId", + "partitions", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + BatchId, + Partitions, + __SkipField__, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "batchId" | "batch_id" => Ok(GeneratedField::BatchId), + "partitions" => Ok(GeneratedField::Partitions), + _ => Ok(GeneratedField::__SkipField__), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = QRepPartitionBatch; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct peerdb_flow.QRepPartitionBatch") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut batch_id__ = None; + let mut partitions__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::BatchId => { + if batch_id__.is_some() { + return Err(serde::de::Error::duplicate_field("batchId")); + } + batch_id__ = + Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::Partitions => { + if partitions__.is_some() { + return Err(serde::de::Error::duplicate_field("partitions")); + } + partitions__ = Some(map.next_value()?); + } + GeneratedField::__SkipField__ => { + let _ = map.next_value::()?; + } + } + } + Ok(QRepPartitionBatch { + batch_id: batch_id__.unwrap_or_default(), + partitions: partitions__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("peerdb_flow.QRepPartitionBatch", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for QRepSyncMode { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result diff --git a/protos/flow.proto b/protos/flow.proto index d76be176a7..a279a91914 100644 --- a/protos/flow.proto +++ b/protos/flow.proto @@ -252,6 +252,11 @@ message QRepPartition { bool full_table_partition = 4; } +message QRepPartitionBatch { + int32 batch_id = 1; + repeated QRepPartition partitions = 2; +} + message QRepParitionResult { repeated QRepPartition partitions = 1; } From 1592cee0562c195e2f09122f4b5f3ab1f1204152 Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Tue, 29 Aug 2023 12:19:58 -0700 Subject: [PATCH 20/37] better logging for partition batches (#362) --- flow/activities/flowable.go | 16 ++++++++++------ flow/workflows/qrep_flow.go | 5 +++-- 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index a2d9014e73..49e2f6a27a 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -378,10 +378,12 @@ func (a *FlowableActivity) ReplicateQRepPartitions(ctx context.Context, partitions *protos.QRepPartitionBatch, runUUID string, ) error { - log.Infof("replicating partitions for job - %s - batch %d\n", config.FlowJobName, partitions.BatchId) - for _, p := range partitions.Partitions { - log.Infof("replicating partition - %s\n", p.PartitionId) - err := a.ReplicateQRepPartition(ctx, config, p, runUUID) + numPartitions := len(partitions.Partitions) + log.Infof("replicating partitions for job - %s - batch %d - size: %d\n", + config.FlowJobName, partitions.BatchId, numPartitions) + for i, p := range partitions.Partitions { + log.Infof("batch-%d - replicating partition - %s\n", partitions.BatchId, p.PartitionId) + err := a.replicateQRepPartition(ctx, config, i+1, numPartitions, p, runUUID) if err != nil { return err } @@ -391,8 +393,10 @@ func (a *FlowableActivity) ReplicateQRepPartitions(ctx context.Context, } // ReplicateQRepPartition replicates a QRepPartition from the source to the destination. -func (a *FlowableActivity) ReplicateQRepPartition(ctx context.Context, +func (a *FlowableActivity) replicateQRepPartition(ctx context.Context, config *protos.QRepConfig, + idx int, + total int, partition *protos.QRepPartition, runUUID string, ) error { @@ -466,7 +470,7 @@ func (a *FlowableActivity) ReplicateQRepPartition(ctx context.Context, } shutdown := utils.HeartbeatRoutine(ctx, 5*time.Minute, func() string { - return fmt.Sprintf("syncing partition - %s", partition.PartitionId) + return fmt.Sprintf("syncing partition - %s: %d of %d total.", partition.PartitionId, idx, total) }) defer func() { diff --git a/flow/workflows/qrep_flow.go b/flow/workflows/qrep_flow.go index 5a3b704e50..10d3e089d0 100644 --- a/flow/workflows/qrep_flow.go +++ b/flow/workflows/qrep_flow.go @@ -78,7 +78,8 @@ func (q *QRepFlowExecution) ReplicatePartitions(ctx workflow.Context, partitions HeartbeatTimeout: 1 * time.Hour, }) - q.logger.Info("replicating partition", "partition", partitions.BatchId) + msg := fmt.Sprintf("replicating partition batch - %d", partitions.BatchId) + q.logger.Info(msg) if err := workflow.ExecuteActivity(ctx, flowable.ReplicateQRepPartitions, q.config, partitions, q.runUUID).Get(ctx, nil); err != nil { return fmt.Errorf("failed to replicate partition: %w", err) @@ -150,7 +151,7 @@ func (q *QRepFlowExecution) processPartitions( for i, parts := range batches { batch := &protos.QRepPartitionBatch{ Partitions: parts, - BatchId: int32(i), + BatchId: int32(i + 1), } future, err := q.startChildWorkflow(ctx, batch) if err != nil { From 16702bb2ce488dbc033ca4de4db0d23391586d5f Mon Sep 17 00:00:00 2001 From: Arunprasad Rajkumar Date: Wed, 30 Aug 2023 20:26:42 +0530 Subject: [PATCH 21/37] Fix Linux docker-compose error and add TEMPORAL_CSRF_COOKIE_INSECURE=true (#363) This addition helps cancel workflows from the temporal UI, avoiding the "Missing csrf token in request header" error. Reference: [1] [1] https://community.temporal.io/t/missing-csrf-token-in-request-header/7410 Signed-off-by: Arunprasad Rajkumar --- docker-compose.yml | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/docker-compose.yml b/docker-compose.yml index d902a71b4f..8f65b14b4d 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -13,11 +13,11 @@ x-flow-worker-env: &flow-worker-env AWS_SECRET_ACCESS_KEY: ${AWS_SECRET_ACCESS_KEY:-""} AWS_REGION: ${AWS_REGION:-""} # enables worker profiling using Go's pprof - ENABLE_PROFILING: true + ENABLE_PROFILING: "true" # enables exporting of mirror metrics to Prometheus for visualization using Grafana - ENABLE_METRICS: true + ENABLE_METRICS: "true" # enables exporting of mirror metrics to Catalog in the PEERDB_STATS schema. - ENABLE_STATS: true + ENABLE_STATS: "true" services: catalog: @@ -83,6 +83,7 @@ services: environment: - TEMPORAL_ADDRESS=temporal:7233 - TEMPORAL_CORS_ORIGINS=http://localhost:3000 + - TEMPORAL_CSRF_COOKIE_INSECURE=true image: temporalio/ui:2.17.2 ports: - 8085:8080 From f52b4dc5257b39e151c7498be20c203feb074b2d Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Wed, 30 Aug 2023 20:54:21 +0530 Subject: [PATCH 22/37] QRep Flow: 5 min timeout and removes retry (#364) --- flow/workflows/qrep_flow.go | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/flow/workflows/qrep_flow.go b/flow/workflows/qrep_flow.go index 10d3e089d0..bceb1fa7f7 100644 --- a/flow/workflows/qrep_flow.go +++ b/flow/workflows/qrep_flow.go @@ -72,10 +72,7 @@ func (q *QRepFlowExecution) GetPartitions( func (q *QRepFlowExecution) ReplicatePartitions(ctx workflow.Context, partitions *protos.QRepPartitionBatch) error { ctx = workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ StartToCloseTimeout: 24 * 5 * time.Hour, - RetryPolicy: &temporal.RetryPolicy{ - MaximumAttempts: 20, - }, - HeartbeatTimeout: 1 * time.Hour, + HeartbeatTimeout: 5 * time.Minute, }) msg := fmt.Sprintf("replicating partition batch - %d", partitions.BatchId) From c7acbf0bdadbefc37201405df028dcf3705aa2e6 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Wed, 30 Aug 2023 21:16:27 +0530 Subject: [PATCH 23/37] Changes to ubuntu-latest in stable debian file (#365) --- .github/workflows/stable-debian.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/stable-debian.yml b/.github/workflows/stable-debian.yml index 96c14812dd..6f337c1dc9 100644 --- a/.github/workflows/stable-debian.yml +++ b/.github/workflows/stable-debian.yml @@ -8,7 +8,7 @@ on: jobs: release: name: build and release - runs-on: ubuntu-latest-64-core + runs-on: ubuntu-latest permissions: contents: write steps: From 87a54a1b71a4af88839c34cd1529e8f3995874da Mon Sep 17 00:00:00 2001 From: Kevin K Biju <52661649+heavycrystal@users.noreply.github.com> Date: Wed, 30 Aug 2023 23:37:05 +0530 Subject: [PATCH 24/37] optimizing flow tests by separating and parallelizing test suites (#356) 1. Split each connector into its own test suite, and rewrote them to allow running multiple test suites in parallel. 2. Better separation on source PG instance [shared by all test suites of a run] and destination BQ and SF instances [shared by all runs]. 3. some code cleanup and fixing some tests up. 4. Switched to using a vanilla PG instance, but configured with higher logical replication limits. --- .github/workflows/flow.yml | 20 +- flow/e2e/{ => bigquery}/bigquery_helper.go | 5 +- flow/e2e/bigquery/peer_flow_bq_test.go | 925 +++++++++ flow/e2e/bigquery/qrep_flow_bq_test.go | 123 ++ flow/e2e/congen.go | 99 + flow/e2e/{ => eventhub}/eventhub_helper.go | 2 +- flow/e2e/{ => eventhub}/peer_flow_eh_test.go | 74 +- flow/e2e/peer_flow_s3_test.go | 153 -- flow/e2e/peer_flow_test.go | 1809 ----------------- flow/e2e/postgres/qrep_flow_pg_test.go | 147 ++ flow/e2e/qrep_flow_test.go | 625 ------ flow/e2e/s3/qrep_flow_s3_test.go | 183 ++ flow/e2e/{ => s3}/s3_helper.go | 2 +- flow/e2e/snowflake/peer_flow_sf_test.go | 800 ++++++++ flow/e2e/snowflake/qrep_flow_sf_test.go | 232 +++ flow/e2e/snowflake/snowflake_helper.go | 131 ++ flow/e2e/snowflake_helper.go | 104 - .../qrep_flow_sqlserver_test.go | 102 +- flow/e2e/{ => sqlserver}/sqlserver_helper.go | 2 +- flow/e2e/test_utils.go | 290 ++- 20 files changed, 3098 insertions(+), 2730 deletions(-) rename flow/e2e/{ => bigquery}/bigquery_helper.go (99%) create mode 100644 flow/e2e/bigquery/peer_flow_bq_test.go create mode 100644 flow/e2e/bigquery/qrep_flow_bq_test.go rename flow/e2e/{ => eventhub}/eventhub_helper.go (99%) rename flow/e2e/{ => eventhub}/peer_flow_eh_test.go (57%) delete mode 100644 flow/e2e/peer_flow_s3_test.go delete mode 100644 flow/e2e/peer_flow_test.go create mode 100644 flow/e2e/postgres/qrep_flow_pg_test.go delete mode 100644 flow/e2e/qrep_flow_test.go create mode 100644 flow/e2e/s3/qrep_flow_s3_test.go rename flow/e2e/{ => s3}/s3_helper.go (99%) create mode 100644 flow/e2e/snowflake/peer_flow_sf_test.go create mode 100644 flow/e2e/snowflake/qrep_flow_sf_test.go create mode 100644 flow/e2e/snowflake/snowflake_helper.go delete mode 100644 flow/e2e/snowflake_helper.go rename flow/e2e/{ => sqlserver}/qrep_flow_sqlserver_test.go (53%) rename flow/e2e/{ => sqlserver}/sqlserver_helper.go (99%) diff --git a/.github/workflows/flow.yml b/.github/workflows/flow.yml index b7e2085b47..5f1623eba3 100644 --- a/.github/workflows/flow.yml +++ b/.github/workflows/flow.yml @@ -1,10 +1,10 @@ name: Flow build and test on: - push: - branches: [main] pull_request: branches: [main] + push: + branches: [main] jobs: flow_test: @@ -12,7 +12,7 @@ jobs: timeout-minutes: 30 services: pg_cdc: - image: debezium/postgres:14-alpine + image: postgres:15.4-alpine ports: - 7132:5432 env: @@ -20,6 +20,7 @@ jobs: POSTGRES_PASSWORD: postgres POSTGRES_DB: postgres options: >- + --name pg_cdc --health-cmd pg_isready --health-interval 10s --health-timeout 5s @@ -55,18 +56,21 @@ jobs: name: "snowflake_creds.json" json: ${{ secrets.SNOWFLAKE_GH_CI_PKEY }} - - name: create hstore extension + - name: create hstore extension and increase logical replication limits run: | - sudo apt-get update - sudo apt-get install -y postgresql-client - psql -h localhost -p 7132 -U postgres -c "CREATE EXTENSION hstore;" + docker exec pg_cdc psql -h localhost -p 5432 -U postgres -c "CREATE EXTENSION hstore;" + docker exec pg_cdc psql -h localhost -p 5432 -U postgres -c "ALTER SYSTEM SET wal_level=logical;" + docker exec pg_cdc psql -h localhost -p 5432 -U postgres -c "ALTER SYSTEM SET max_replication_slots=100;" + docker exec pg_cdc psql -h localhost -p 5432 -U postgres -c "ALTER SYSTEM SET max_wal_senders=100;" + docker restart pg_cdc working-directory: ./flow env: + PG_CDC: PGPASSWORD: postgres - name: run tests run: | - gotestsum --format testname -- -p 1 ./... -timeout 1200s + gotestsum --format testname -- -p 4 ./... -timeout 1200s working-directory: ./flow env: AWS_ACCESS_KEY_ID: ${{ secrets.AWS_ACCESS_KEY_ID }} diff --git a/flow/e2e/bigquery_helper.go b/flow/e2e/bigquery/bigquery_helper.go similarity index 99% rename from flow/e2e/bigquery_helper.go rename to flow/e2e/bigquery/bigquery_helper.go index 5e9083ec98..72e8b3edea 100644 --- a/flow/e2e/bigquery_helper.go +++ b/flow/e2e/bigquery/bigquery_helper.go @@ -1,4 +1,4 @@ -package e2e +package e2e_bigquery import ( "context" @@ -12,6 +12,7 @@ import ( "cloud.google.com/go/bigquery" "cloud.google.com/go/civil" peer_bq "github.com/PeerDB-io/peer-flow/connectors/bigquery" + "github.com/PeerDB-io/peer-flow/e2e" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" @@ -45,7 +46,7 @@ func NewBigQueryTestHelper() (*BigQueryTestHelper, error) { return nil, fmt.Errorf("TEST_BQ_CREDS env var not set") } - content, err := readFileToBytes(jsonPath) + content, err := e2e.ReadFileToBytes(jsonPath) if err != nil { return nil, fmt.Errorf("failed to read file: %w", err) } diff --git a/flow/e2e/bigquery/peer_flow_bq_test.go b/flow/e2e/bigquery/peer_flow_bq_test.go new file mode 100644 index 0000000000..655a317854 --- /dev/null +++ b/flow/e2e/bigquery/peer_flow_bq_test.go @@ -0,0 +1,925 @@ +package e2e_bigquery + +import ( + "context" + "fmt" + "testing" + + "github.com/PeerDB-io/peer-flow/e2e" + "github.com/PeerDB-io/peer-flow/generated/protos" + peerflow "github.com/PeerDB-io/peer-flow/workflows" + "github.com/jackc/pgx/v5/pgxpool" + "github.com/joho/godotenv" + log "github.com/sirupsen/logrus" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + "go.temporal.io/sdk/testsuite" +) + +const bigquerySuffix = "bigquery" + +type PeerFlowE2ETestSuiteBQ struct { + suite.Suite + testsuite.WorkflowTestSuite + + pool *pgxpool.Pool + bqHelper *BigQueryTestHelper +} + +func TestPeerFlowE2ETestSuiteBQ(t *testing.T) { + suite.Run(t, new(PeerFlowE2ETestSuiteBQ)) +} + +func (s *PeerFlowE2ETestSuiteBQ) attachSchemaSuffix(tableName string) string { + return fmt.Sprintf("e2e_test_%s.%s", bigquerySuffix, tableName) +} + +func (s *PeerFlowE2ETestSuiteBQ) attachSuffix(input string) string { + return fmt.Sprintf("%s_%s", input, bigquerySuffix) +} + +// setupBigQuery sets up the bigquery connection. +func (s *PeerFlowE2ETestSuiteBQ) setupBigQuery() error { + bqHelper, err := NewBigQueryTestHelper() + if err != nil { + return fmt.Errorf("failed to create bigquery helper: %w", err) + } + + err = bqHelper.RecreateDataset() + if err != nil { + return fmt.Errorf("failed to recreate bigquery dataset: %w", err) + } + + s.bqHelper = bqHelper + return nil +} + +// Implement SetupAllSuite interface to setup the test suite +func (s *PeerFlowE2ETestSuiteBQ) SetupSuite() { + err := godotenv.Load() + if err != nil { + // it's okay if the .env file is not present + // we will use the default values + log.Infof("Unable to load .env file, using default values from env") + } + + log.SetReportCaller(true) + + pool, err := e2e.SetupPostgres(bigquerySuffix) + if err != nil { + s.Fail("failed to setup postgres", err) + } + s.pool = pool + + err = s.setupBigQuery() + if err != nil { + s.Fail("failed to setup bigquery", err) + } +} + +// Implement TearDownAllSuite interface to tear down the test suite +func (s *PeerFlowE2ETestSuiteBQ) TearDownSuite() { + err := e2e.TearDownPostgres(s.pool, bigquerySuffix) + if err != nil { + s.Fail("failed to drop Postgres schema", err) + } + + err = s.bqHelper.DropDataset() + if err != nil { + s.Fail("failed to drop bigquery dataset", err) + } +} + +func (s *PeerFlowE2ETestSuiteBQ) Test_Invalid_Connection_Config() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + // TODO (kaushikiska): ensure flow name can only be alpha numeric and underscores. + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 1, + MaxBatchSize: 1, + } + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, nil, &limits, nil) + + // Verify workflow completes + s.True(env.IsWorkflowCompleted()) + err := env.GetWorkflowError() + + // assert that error contains "invalid connection configs" + s.Error(err) + s.Contains(err.Error(), "invalid connection configs") + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteBQ) Test_Complete_Flow_No_Data() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_no_data") + dstTableName := "test_no_data" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id SERIAL PRIMARY KEY, + key TEXT NOT NULL, + value VARCHAR(255) NOT NULL + ); + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_complete_flow_no_data"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.bqHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 1, + MaxBatchSize: 1, + } + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteBQ) Test_Char_ColType_Error() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_char_coltype") + dstTableName := "test_char_coltype" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id SERIAL PRIMARY KEY, + key TEXT NOT NULL, + value CHAR(255) NOT NULL + ); + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_char_table"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.bqHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 1, + MaxBatchSize: 1, + } + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + env.AssertExpectations(s.T()) +} + +// Test_Complete_Simple_Flow_BQ tests a complete flow with data in the source table. +// The test inserts 10 rows into the source table and verifies that the data is +// correctly synced to the destination table after sync flow completes. +func (s *PeerFlowE2ETestSuiteBQ) Test_Complete_Simple_Flow_BQ() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_simple_flow_bq") + dstTableName := "test_simple_flow_bq" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id SERIAL PRIMARY KEY, + key TEXT NOT NULL, + value TEXT NOT NULL + ); + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_complete_simple_flow"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.bqHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 2, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and then insert 10 rows into the source table + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + // insert 10 rows into the source table + for i := 0; i < 10; i++ { + testKey := fmt.Sprintf("test_key_%d", i) + testValue := fmt.Sprintf("test_value_%d", i) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(key, value) VALUES ($1, $2) + `, srcTableName), testKey, testValue) + s.NoError(err) + } + fmt.Println("Inserted 10 rows into the source table") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + count, err := s.bqHelper.CountRows(dstTableName) + s.NoError(err) + s.Equal(10, count) + + // TODO: verify that the data is correctly synced to the destination table + // on the bigquery side + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteBQ) Test_Toast_BQ() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_toast_bq_1") + dstTableName := "test_toast_bq_1" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id SERIAL PRIMARY KEY, + t1 text, + t2 text, + k int + );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_toast_bq_1"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.bqHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 1, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and execute a transaction touching toast columns + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + /* + Executing a transaction which + 1. changes both toast column + 2. changes no toast column + 2. changes 1 toast column + */ + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + BEGIN; + INSERT INTO %s(t1,t2,k) SELECT random_string(9000),random_string(9000), + 1 FROM generate_series(1,2); + UPDATE %s SET k=102 WHERE id=1; + UPDATE %s SET t1='dummy' WHERE id=2; + END; + `, srcTableName, srcTableName, srcTableName)) + s.NoError(err) + fmt.Println("Executed a transaction touching toast columns") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + s.compareTableContentsBQ(dstTableName, "id,t1,t2,k") + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteBQ) Test_Toast_Nochanges_BQ() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_toast_bq_2") + dstTableName := "test_toast_bq_2" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id SERIAL PRIMARY KEY, + t1 text, + t2 text, + k int + );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_toast_bq_2"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.bqHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 1, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and execute a transaction touching toast columns + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + /* transaction updating no rows */ + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + BEGIN; + UPDATE %s SET k=102 WHERE id=1; + UPDATE %s SET t1='dummy' WHERE id=2; + END; + `, srcTableName, srcTableName)) + s.NoError(err) + fmt.Println("Executed a transaction touching toast columns") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + s.compareTableContentsBQ(dstTableName, "id,t1,t2,k") + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_1_BQ() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_toast_bq_3") + dstTableName := "test_toast_bq_3" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id SERIAL PRIMARY KEY, + t1 text, + t2 text, + k int + );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_toast_bq_3"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.bqHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 1, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and execute a transaction touching toast columns + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + //complex transaction with random DMLs on a table with toast columns + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + BEGIN; + INSERT INTO %s(t1,t2,k) SELECT random_string(9000),random_string(9000), + 1 FROM generate_series(1,2); + UPDATE %s SET k=102 WHERE id=1; + UPDATE %s SET t1='dummy' WHERE id=2; + UPDATE %s SET t2='dummy' WHERE id=2; + DELETE FROM %s WHERE id=1; + INSERT INTO %s(t1,t2,k) SELECT random_string(9000),random_string(9000), + 1 FROM generate_series(1,2); + UPDATE %s SET k=1 WHERE id=1; + UPDATE %s SET t1='dummy1',t2='dummy2' WHERE id=1; + UPDATE %s SET t1='dummy3' WHERE id=3; + DELETE FROM %s WHERE id=2; + DELETE FROM %s WHERE id=3; + DELETE FROM %s WHERE id=2; + END; + `, srcTableName, srcTableName, srcTableName, srcTableName, srcTableName, srcTableName, + srcTableName, srcTableName, srcTableName, srcTableName, srcTableName, srcTableName)) + s.NoError(err) + fmt.Println("Executed a transaction touching toast columns") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + s.compareTableContentsBQ(dstTableName, "id,t1,t2,k") + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_2_BQ() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_toast_bq_4") + dstTableName := "test_toast_bq_4" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE %s ( + id SERIAL PRIMARY KEY, + t1 text, + k int + );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_toast_bq_4"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.bqHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 1, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and execute a transaction touching toast columns + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + //complex transaction with random DMLs on a table with toast columns + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + BEGIN; + INSERT INTO %s(t1,k) SELECT random_string(9000), + 1 FROM generate_series(1,1); + UPDATE %s SET t1=sub.t1 FROM (SELECT random_string(9000) t1 + FROM generate_series(1,1) ) sub WHERE id=1; + UPDATE %s SET k=2 WHERE id=1; + UPDATE %s SET k=3 WHERE id=1; + UPDATE %s SET t1=sub.t1 FROM (SELECT random_string(9000) t1 + FROM generate_series(1,1)) sub WHERE id=1; + UPDATE %s SET k=4 WHERE id=1; + END; + `, srcTableName, srcTableName, srcTableName, srcTableName, srcTableName, srcTableName)) + s.NoError(err) + fmt.Println("Executed a transaction touching toast columns") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + s.compareTableContentsBQ(dstTableName, "id,t1,k") + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_3_BQ() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_toast_bq_5") + dstTableName := "test_toast_bq_5" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id SERIAL PRIMARY KEY, + t1 text, + t2 text, + k int + );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_toast_bq_5"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.bqHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 1, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and execute a transaction touching toast columns + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + /* + transaction updating a single row + multiple times with changed/unchanged toast columns + */ + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + BEGIN; + INSERT INTO %s(t1,t2,k) SELECT random_string(9000),random_string(9000), + 1 FROM generate_series(1,1); + UPDATE %s SET k=102 WHERE id=1; + UPDATE %s SET t1='dummy' WHERE id=1; + UPDATE %s SET t2='dummy' WHERE id=1; + END; + `, srcTableName, srcTableName, srcTableName, srcTableName)) + s.NoError(err) + fmt.Println("Executed a transaction touching toast columns") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + s.compareTableContentsBQ(dstTableName, "id,t1,t2,k") + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteBQ) Test_Types_BQ() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_types_bq") + dstTableName := "test_types_bq" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s (id serial PRIMARY KEY,c1 BIGINT,c2 BIT,c3 VARBIT,c4 BOOLEAN, + c6 BYTEA,c7 CHARACTER,c8 varchar,c9 CIDR,c11 DATE,c12 FLOAT,c13 DOUBLE PRECISION, + c14 INET,c15 INTEGER,c16 INTERVAL,c17 JSON,c18 JSONB,c21 MACADDR,c22 MONEY, + c23 NUMERIC,c24 OID,c28 REAL,c29 SMALLINT,c30 SMALLSERIAL,c31 SERIAL,c32 TEXT, + c33 TIMESTAMP,c34 TIMESTAMPTZ,c35 TIME, c36 TIMETZ,c37 TSQUERY,c38 TSVECTOR, + c39 TXID_SNAPSHOT,c40 UUID,c41 XML, c42 INT[], c43 FLOAT[], c44 TEXT[]); + CREATE OR REPLACE FUNCTION random_bytea(bytea_length integer) + RETURNS bytea AS $body$ + SELECT decode(string_agg(lpad(to_hex(width_bucket(random(), 0, 1, 256)-1),2,'0') ,''), 'hex') + FROM generate_series(1, $1); + $body$ + LANGUAGE 'sql' + VOLATILE + SET search_path = 'pg_catalog'; + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_types_bq"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.bqHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + + TotalSyncFlows: 1, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and execute a transaction touching toast columns + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + /* test inserting various types*/ + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s SELECT 2,2,b'1',b'101', + true,random_bytea(32),'s','test','1.1.10.2'::cidr, + CURRENT_DATE,1.23,1.234,'192.168.1.5'::inet,1, + '5 years 2 months 29 days 1 minute 2 seconds 200 milliseconds 20000 microseconds'::interval, + '{"sai":1}'::json,'{"sai":1}'::jsonb,'08:00:2b:01:02:03'::macaddr, + 1.2,1.23,4::oid,1.23,1,1,1,'test',now(),now(),now()::time,now()::timetz, + 'fat & rat'::tsquery,'a fat cat sat on a mat and ate a fat rat'::tsvector, + txid_current_snapshot(), + '66073c38-b8df-4bdb-bbca-1c97596b8940'::uuid,xmlcomment('hello'), + ARRAY[10299301,2579827], + ARRAY[0.0003, 8902.0092], + ARRAY['hello','bye']; + `, srcTableName)) + s.NoError(err) + fmt.Println("Executed an insert with all types") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + noNulls, err := s.bqHelper.CheckNull(dstTableName, []string{"c41", "c1", "c2", "c3", "c4", + "c6", "c39", "c40", "id", "c9", "c11", "c12", "c13", "c14", "c15", "c16", "c17", "c18", + "c21", "c22", "c23", "c24", "c28", "c29", "c30", "c31", "c33", "c34", "c35", "c36", + "c37", "c38", "c7", "c8", "c32", "c42", "c43", "c44"}) + if err != nil { + fmt.Println("error %w", err) + } + // Make sure that there are no nulls + s.True(noNulls) + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteBQ) Test_Types_Avro_BQ() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_types_avro_bq") + dstTableName := "test_types_avro_bq" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s (id serial PRIMARY KEY,c1 BIGINT,c2 BIT,c3 VARBIT,c4 BOOLEAN, + c6 BYTEA,c7 CHARACTER,c8 varchar,c9 CIDR,c11 DATE,c12 FLOAT,c13 DOUBLE PRECISION, + c14 INET,c15 INTEGER,c16 INTERVAL,c17 JSON,c18 JSONB,c21 MACADDR,c22 MONEY, + c23 NUMERIC,c24 OID,c28 REAL,c29 SMALLINT,c30 SMALLSERIAL,c31 SERIAL,c32 TEXT, + c33 TIMESTAMP,c34 TIMESTAMPTZ,c35 TIME, c36 TIMETZ,c37 TSQUERY,c38 TSVECTOR, + c39 TXID_SNAPSHOT,c40 UUID,c41 XML, c42 INT[], c43 FLOAT[], c44 TEXT[]); + CREATE OR REPLACE FUNCTION random_bytea(bytea_length integer) + RETURNS bytea AS $body$ + SELECT decode(string_agg(lpad(to_hex(width_bucket(random(), 0, 1, 256)-1),2,'0') ,''), 'hex') + FROM generate_series(1, $1); + $body$ + LANGUAGE 'sql' + VOLATILE + SET search_path = 'pg_catalog'; + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_types_avro_bq"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.bqHelper.Peer, + CDCSyncMode: protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, + CdcStagingPath: "peerdb_staging", + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + + TotalSyncFlows: 1, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and execute a transaction touching toast columns + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + /* test inserting various types*/ + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s SELECT 2,2,b'1',b'101', + true,random_bytea(32),'s','test','1.1.10.2'::cidr, + CURRENT_DATE,1.23,1.234,'192.168.1.5'::inet,1, + '5 years 2 months 29 days 1 minute 2 seconds 200 milliseconds 20000 microseconds'::interval, + '{"sai":1}'::json,'{"sai":1}'::jsonb,'08:00:2b:01:02:03'::macaddr, + 1.2,1.23,4::oid,1.23,1,1,1,'test',now(),now(),now()::time,now()::timetz, + 'fat & rat'::tsquery,'a fat cat sat on a mat and ate a fat rat'::tsvector, + txid_current_snapshot(), + '66073c38-b8df-4bdb-bbca-1c97596b8940'::uuid,xmlcomment('hello'), + ARRAY[9301,239827], + ARRAY[0.0003, 1039.0034], + ARRAY['hello','bye']; + `, srcTableName)) + s.NoError(err) + fmt.Println("Executed an insert with all types") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + noNulls, err := s.bqHelper.CheckNull(dstTableName, []string{"c41", "c1", "c2", "c3", "c4", + "c6", "c39", "c40", "id", "c9", "c11", "c12", "c13", "c14", "c15", "c16", "c17", "c18", + "c21", "c22", "c23", "c24", "c28", "c29", "c30", "c31", "c33", "c34", "c35", "c36", + "c37", "c38", "c7", "c8", "c32", "c42", "c43", "c44"}) + if err != nil { + fmt.Println("error %w", err) + } + // Make sure that there are no nulls + s.True(noNulls) + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteBQ) Test_Simple_Flow_BQ_Avro_CDC() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_simple_flow_bq_avro_cdc") + dstTableName := "test_simple_flow_bq_avro_cdc" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id SERIAL PRIMARY KEY, + key TEXT NOT NULL, + value TEXT NOT NULL + ); + `, srcTableName)) + s.NoError(err) + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_simple_flow_bq_avro_cdc"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.bqHelper.Peer, + CDCSyncMode: protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, + CdcStagingPath: "peerdb_staging", + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 2, + MaxBatchSize: 100, + } + + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + for i := 0; i < 10; i++ { + testKey := fmt.Sprintf("test_key_%d", i) + testValue := fmt.Sprintf("test_value_%d", i) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s (key, value) VALUES ($1, $2) + `, srcTableName), testKey, testValue) + s.NoError(err) + } + fmt.Println("Inserted 10 rows into the source table") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + count, err := s.bqHelper.CountRows(dstTableName) + s.NoError(err) + s.Equal(10, count) + + // TODO: verify that the data is correctly synced to the destination table + // on the bigquery side + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteBQ) Test_Multi_Table_BQ() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTable1Name := s.attachSchemaSuffix("test1_bq") + dstTable1Name := "test1_bq" + srcTable2Name := s.attachSchemaSuffix("test2_bq") + dstTable2Name := "test2_bq" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE %s (id serial primary key, c1 int, c2 text); + CREATE TABLE %s(id serial primary key, c1 int, c2 text); + `, srcTable1Name, srcTable2Name)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_multi_table_bq"), + TableNameMapping: map[string]string{srcTable1Name: dstTable1Name, srcTable2Name: dstTable2Name}, + PostgresPort: e2e.PostgresPort, + Destination: s.bqHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 1, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and execute a transaction touching toast columns + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + /* inserting across multiple tables*/ + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s (c1,c2) VALUES (1,'dummy_1'); + INSERT INTO %s (c1,c2) VALUES (-1,'dummy_-1'); + `, srcTable1Name, srcTable2Name)) + s.NoError(err) + fmt.Println("Executed an insert on two tables") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + require.True(s.T(), env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + count1, err := s.bqHelper.CountRows(dstTable1Name) + s.NoError(err) + count2, err := s.bqHelper.CountRows(dstTable2Name) + s.NoError(err) + + s.Equal(1, count1) + s.Equal(1, count2) + + env.AssertExpectations(s.T()) +} diff --git a/flow/e2e/bigquery/qrep_flow_bq_test.go b/flow/e2e/bigquery/qrep_flow_bq_test.go new file mode 100644 index 0000000000..5e6374cc1a --- /dev/null +++ b/flow/e2e/bigquery/qrep_flow_bq_test.go @@ -0,0 +1,123 @@ +package e2e_bigquery + +import ( + "context" + "fmt" + + connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" + "github.com/PeerDB-io/peer-flow/e2e" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/stretchr/testify/require" +) + +func (s *PeerFlowE2ETestSuiteBQ) setupSourceTable(tableName string, rowCount int) { + err := e2e.CreateSourceTableQRep(s.pool, bigquerySuffix, tableName) + s.NoError(err) + err = e2e.PopulateSourceTable(s.pool, bigquerySuffix, tableName, rowCount) + s.NoError(err) +} + +func (s *PeerFlowE2ETestSuiteBQ) setupBQDestinationTable(dstTable string) { + schema := e2e.GetOwnersSchema() + err := s.bqHelper.CreateTable(dstTable, schema) + + // fail if table creation fails + require.NoError(s.T(), err) + + fmt.Printf("created table on bigquery: %s.%s. %v\n", s.bqHelper.Config.DatasetId, dstTable, err) +} + +func (s *PeerFlowE2ETestSuiteBQ) compareTableContentsBQ(tableName string, colsString string) { + // read rows from source table + pgQueryExecutor := connpostgres.NewQRepQueryExecutor(s.pool, context.Background(), "testflow", "testpart") + pgQueryExecutor.SetTestEnv(true) + + pgRows, err := pgQueryExecutor.ExecuteAndProcessQuery( + fmt.Sprintf("SELECT %s FROM e2e_test_%s.%s ORDER BY id", colsString, bigquerySuffix, tableName), + ) + s.NoError(err) + + // read rows from destination table + qualifiedTableName := fmt.Sprintf("`%s.%s`", s.bqHelper.Config.DatasetId, tableName) + bqRows, err := s.bqHelper.ExecuteAndProcessQuery( + fmt.Sprintf("SELECT %s FROM %s ORDER BY id", colsString, qualifiedTableName), + ) + s.NoError(err) + + s.True(pgRows.Equals(bqRows), "rows from source and destination tables are not equal") +} + +func (s *PeerFlowE2ETestSuiteBQ) Test_Complete_QRep_Flow_Avro() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + numRows := 10 + + tblName := "test_qrep_flow_avro" + s.setupSourceTable(tblName, numRows) + s.setupBQDestinationTable(tblName) + + query := fmt.Sprintf("SELECT * FROM e2e_test_%s.%s WHERE updated_at BETWEEN {{.start}} AND {{.end}}", + bigquerySuffix, tblName) + + qrepConfig, err := e2e.CreateQRepWorkflowConfig("test_qrep_flow_avro", + fmt.Sprintf("e2e_test_%s.%s", bigquerySuffix, tblName), + tblName, + query, + protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, + s.bqHelper.Peer, + "peerdb_staging") + s.NoError(err) + e2e.RunQrepFlowWorkflow(env, qrepConfig) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + + // assert that error contains "invalid connection configs" + err = env.GetWorkflowError() + s.NoError(err) + + s.compareTableContentsBQ(tblName, "*") + + env.AssertExpectations(s.T()) +} + +// NOTE: Disabled due to large JSON tests being added: https://github.com/PeerDB-io/peerdb/issues/309 + +// Test_Complete_QRep_Flow tests a complete flow with data in the source table. +// The test inserts 10 rows into the source table and verifies that the data is +// // correctly synced to the destination table this runs a QRep Flow. +// func (s *E2EPeerFlowTestSuite) Test_Complete_QRep_Flow_Multi_Insert() { +// env := s.NewTestWorkflowEnvironment() +// registerWorkflowsAndActivities(env) + +// numRows := 10 + +// tblName := "test_qrep_flow_multi_insert" +// s.setupSourceTable(tblName, numRows) +// s.setupBQDestinationTable(tblName) + +// query := fmt.Sprintf("SELECT * FROM e2e_test.%s WHERE updated_at BETWEEN {{.start}} AND {{.end}}", tblName) + +// qrepConfig := s.createQRepWorkflowConfig("test_qrep_flow_mi", +// "e2e_test."+tblName, +// tblName, +// query, +// protos.QRepSyncMode_QREP_SYNC_MODE_MULTI_INSERT, +// s.bqHelper.Peer) +// runQrepFlowWorkflow(env, qrepConfig) + +// // Verify workflow completes without error +// s.True(env.IsWorkflowCompleted()) + +// // assert that error contains "invalid connection configs" +// err := env.GetWorkflowError() +// s.NoError(err) + +// count, err := s.bqHelper.CountRows(tblName) +// s.NoError(err) + +// s.Equal(numRows, count) + +// env.AssertExpectations(s.T()) +// } diff --git a/flow/e2e/congen.go b/flow/e2e/congen.go index f82912947e..14725d7899 100644 --- a/flow/e2e/congen.go +++ b/flow/e2e/congen.go @@ -1,9 +1,108 @@ package e2e import ( + "context" + "fmt" + + "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/jackc/pgx/v5/pgxpool" +) + +const ( + postgresHost = "localhost" + postgresUser = "postgres" + postgresPassword = "postgres" + postgresDatabase = "postgres" + PostgresPort = 7132 ) +func GetTestPostgresConf() *protos.PostgresConfig { + return &protos.PostgresConfig{ + Host: postgresHost, + Port: uint32(PostgresPort), + User: postgresUser, + Password: postgresPassword, + Database: postgresDatabase, + } +} + +func cleanPostgres(pool *pgxpool.Pool, suffix string) error { + // drop the e2e_test schema with the given suffix if it exists + _, err := pool.Exec(context.Background(), fmt.Sprintf("DROP SCHEMA IF EXISTS e2e_test_%s CASCADE", suffix)) + if err != nil { + return fmt.Errorf("failed to drop e2e_test schema: %w", err) + } + + // drop all open slots with the given suffix + _, err = pool.Exec( + context.Background(), + "SELECT pg_drop_replication_slot(slot_name) FROM pg_replication_slots WHERE slot_name LIKE $1", + fmt.Sprintf("%%_%s", suffix), + ) + if err != nil { + return fmt.Errorf("failed to drop replication slots: %w", err) + } + + // list all publications from pg_publication table + rows, err := pool.Query(context.Background(), + "SELECT pubname FROM pg_publication WHERE pubname LIKE $1", + fmt.Sprintf("%%_%s", suffix), + ) + if err != nil { + return fmt.Errorf("failed to list publications: %w", err) + } + + // drop all publications with the given suffix + for rows.Next() { + var pubName string + err = rows.Scan(&pubName) + if err != nil { + return fmt.Errorf("failed to scan publication name: %w", err) + } + + _, err = pool.Exec(context.Background(), fmt.Sprintf("DROP PUBLICATION %s", pubName)) + if err != nil { + return fmt.Errorf("failed to drop publication %s: %w", pubName, err) + } + } + + return nil +} + +// setupPostgres sets up the postgres connection pool. +func SetupPostgres(suffix string) (*pgxpool.Pool, error) { + pool, err := pgxpool.New(context.Background(), utils.GetPGConnectionString(GetTestPostgresConf())) + if err != nil { + return nil, fmt.Errorf("failed to create postgres connection pool: %w", err) + } + + err = cleanPostgres(pool, suffix) + if err != nil { + return nil, err + } + + // create an e2e_test schema + _, err = pool.Exec(context.Background(), fmt.Sprintf("CREATE SCHEMA e2e_test_%s", suffix)) + if err != nil { + return nil, fmt.Errorf("failed to create e2e_test schema: %w", err) + } + + return pool, nil +} + +func TearDownPostgres(pool *pgxpool.Pool, suffix string) error { + // drop the e2e_test schema + if pool != nil { + err := cleanPostgres(pool, suffix) + if err != nil { + return err + } + pool.Close() + } + return nil +} + // GeneratePostgresPeer generates a postgres peer config for testing. func GeneratePostgresPeer(postgresPort int) *protos.Peer { ret := &protos.Peer{} diff --git a/flow/e2e/eventhub_helper.go b/flow/e2e/eventhub/eventhub_helper.go similarity index 99% rename from flow/e2e/eventhub_helper.go rename to flow/e2e/eventhub/eventhub_helper.go index 2457361cb6..e488da4287 100644 --- a/flow/e2e/eventhub_helper.go +++ b/flow/e2e/eventhub/eventhub_helper.go @@ -1,4 +1,4 @@ -package e2e +package e2e_eventhub import ( "context" diff --git a/flow/e2e/peer_flow_eh_test.go b/flow/e2e/eventhub/peer_flow_eh_test.go similarity index 57% rename from flow/e2e/peer_flow_eh_test.go rename to flow/e2e/eventhub/peer_flow_eh_test.go index 23a85abde8..687f17face 100644 --- a/flow/e2e/peer_flow_eh_test.go +++ b/flow/e2e/eventhub/peer_flow_eh_test.go @@ -1,23 +1,44 @@ -package e2e +package e2e_eventhub import ( "context" "fmt" "os" + "testing" "time" + "github.com/PeerDB-io/peer-flow/e2e" util "github.com/PeerDB-io/peer-flow/utils" peerflow "github.com/PeerDB-io/peer-flow/workflows" + "github.com/jackc/pgx/v5/pgxpool" + "github.com/joho/godotenv" + log "github.com/sirupsen/logrus" "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + "go.temporal.io/sdk/testsuite" ) -func (s *E2EPeerFlowTestSuite) setupEventHub() error { +const eventhubSuffix = "eventhub" + +type PeerFlowE2ETestSuiteEH struct { + suite.Suite + testsuite.WorkflowTestSuite + + pool *pgxpool.Pool + ehHelper *EventHubTestHelper +} + +func TestPeerFlowE2ETestSuiteEH(t *testing.T) { + suite.Run(t, new(PeerFlowE2ETestSuiteEH)) +} + +func (s *PeerFlowE2ETestSuiteEH) setupEventHub() error { enableEHT := os.Getenv("ENABLE_EVENT_HUB_TESTS") if enableEHT == "" { return nil } - pgConf := GetTestPostgresConf() + pgConf := e2e.GetTestPostgresConf() helper, err := NewEventHubTestHelper(pgConf) if err != nil { return err @@ -27,13 +48,50 @@ func (s *E2EPeerFlowTestSuite) setupEventHub() error { return nil } -func (s *E2EPeerFlowTestSuite) Test_Complete_Simple_Flow_EH() { +func (s *PeerFlowE2ETestSuiteEH) SetupSuite() { + err := godotenv.Load() + if err != nil { + // it's okay if the .env file is not present + // we will use the default values + log.Infof("Unable to load .env file, using default values from env") + } + + log.SetReportCaller(true) + + pool, err := e2e.SetupPostgres(eventhubSuffix) + if err != nil { + s.Fail("failed to setup postgres", err) + } + s.pool = pool + + err = s.setupEventHub() + if err != nil { + s.Fail("failed to setup eventhub", err) + } +} + +// Implement TearDownAllSuite interface to tear down the test suite +func (s *PeerFlowE2ETestSuiteEH) TearDownSuite() { + err := e2e.TearDownPostgres(s.pool, eventhubSuffix) + if err != nil { + s.Fail("failed to drop Postgres schema", err) + } + + if s.ehHelper != nil { + err = s.ehHelper.CleanUp() + if err != nil { + s.Fail("failed to clean up eventhub", err) + } + } +} + +func (s *PeerFlowE2ETestSuiteEH) Test_Complete_Simple_Flow_EH() { if s.ehHelper == nil { s.T().Skip("Skipping EventHub test") } env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env) ru, err := util.RandomUInt64() s.NoError(err) @@ -49,10 +107,10 @@ func (s *E2EPeerFlowTestSuite) Test_Complete_Simple_Flow_EH() { `) s.NoError(err) - connectionGen := FlowConnectionGenerationConfig{ + connectionGen := e2e.FlowConnectionGenerationConfig{ FlowJobName: jobName, TableNameMapping: map[string]string{schemaQualifiedName: jobName}, - PostgresPort: postgresPort, + PostgresPort: e2e.PostgresPort, Destination: s.ehHelper.GetPeer(), } @@ -67,7 +125,7 @@ func (s *E2EPeerFlowTestSuite) Test_Complete_Simple_Flow_EH() { // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup // and then insert 10 rows into the source table go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) + e2e.SetupPeerFlowStatusQuery(env, connectionGen) // insert 10 rows into the source table for i := 0; i < 10; i++ { testKey := fmt.Sprintf("test_key_%d", i) diff --git a/flow/e2e/peer_flow_s3_test.go b/flow/e2e/peer_flow_s3_test.go deleted file mode 100644 index 7cc6e7e387..0000000000 --- a/flow/e2e/peer_flow_s3_test.go +++ /dev/null @@ -1,153 +0,0 @@ -package e2e - -import ( - "context" - "fmt" - "time" - - "github.com/PeerDB-io/peer-flow/generated/protos" - util "github.com/PeerDB-io/peer-flow/utils" - "github.com/stretchr/testify/require" -) - -func (s *E2EPeerFlowTestSuite) setupS3() error { - helper, err := NewS3TestHelper() - if err != nil { - return err - } - - s.s3Helper = helper - return nil -} - -func (s *E2EPeerFlowTestSuite) Test_Complete_QRep_Flow_S3() { - if s.s3Helper == nil { - s.T().Skip("Skipping S3 test") - } - - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - ru, err := util.RandomUInt64() - s.NoError(err) - - jobName := fmt.Sprintf("test_complete_flow_s3_%d", ru) - schemaQualifiedName := fmt.Sprintf("e2e_test.%s", jobName) - _, err = s.pool.Exec(context.Background(), ` - CREATE TABLE `+schemaQualifiedName+` ( - id SERIAL PRIMARY KEY, - key TEXT NOT NULL, - value TEXT NOT NULL - ); - `) - s.NoError(err) - - tblName := "test_qrep_flow_s3_1" - s.setupSourceTable(tblName, 10) - query := fmt.Sprintf("SELECT * FROM e2e_test.%s WHERE updated_at >= {{.start}} AND updated_at < {{.end}}", tblName) - qrepConfig := s.createQRepWorkflowConfig( - jobName, - "e2e_test."+tblName, - "e2e_dest_1", - query, - protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, - s.s3Helper.GetPeer(), - "stage", - ) - qrepConfig.StagingPath = s.s3Helper.s3Config.Url - - runQrepFlowWorkflow(env, qrepConfig) - - go func() { - // insert 10 rows into the source table - for i := 0; i < 10; i++ { - testKey := fmt.Sprintf("test_key_%d", i) - testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.pool.Exec(context.Background(), ` - INSERT INTO `+schemaQualifiedName+` (key, value) VALUES ($1, $2) - `, testKey, testValue) - s.NoError(err) - } - fmt.Println("Inserted 10 rows into the source table") - }() - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - s.NoError(err) - - // Verify destination has 1 file - // make context with timeout - ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) - defer cancel() - - files, err := s.s3Helper.ListAllFiles(ctx, jobName) - - require.NoError(s.T(), err) - - require.Equal(s.T(), 1, len(files)) - - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Complete_QRep_Flow_S3_CTID() { - if s.s3Helper == nil { - s.T().Skip("Skipping S3 test") - } - - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - ru, err := util.RandomUInt64() - s.NoError(err) - - jobName := fmt.Sprintf("test_complete_flow_s3_ctid_%d", ru) - schemaQualifiedName := fmt.Sprintf("e2e_test.%s", jobName) - _, err = s.pool.Exec(context.Background(), ` - CREATE TABLE `+schemaQualifiedName+` ( - id SERIAL PRIMARY KEY, - key TEXT NOT NULL, - value TEXT NOT NULL - ); - `) - s.NoError(err) - - tblName := "test_qrep_flow_s3_ctid" - s.setupSourceTable(tblName, 20000) - query := fmt.Sprintf("SELECT * FROM e2e_test.%s WHERE ctid BETWEEN {{.start}} AND {{.end}}", tblName) - qrepConfig := s.createQRepWorkflowConfig( - jobName, - "e2e_test."+tblName, - "e2e_dest_ctid", - query, - protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, - s.s3Helper.GetPeer(), - "stage", - ) - qrepConfig.StagingPath = s.s3Helper.s3Config.Url - qrepConfig.NumRowsPerPartition = 2000 - qrepConfig.InitialCopyOnly = true - qrepConfig.WatermarkColumn = "ctid" - - runQrepFlowWorkflow(env, qrepConfig) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - s.NoError(err) - - // Verify destination has 1 file - // make context with timeout - ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) - defer cancel() - - files, err := s.s3Helper.ListAllFiles(ctx, jobName) - - require.NoError(s.T(), err) - - require.Equal(s.T(), 10, len(files)) - - env.AssertExpectations(s.T()) -} diff --git a/flow/e2e/peer_flow_test.go b/flow/e2e/peer_flow_test.go deleted file mode 100644 index 4df215cd3f..0000000000 --- a/flow/e2e/peer_flow_test.go +++ /dev/null @@ -1,1809 +0,0 @@ -package e2e - -import ( - "context" - "fmt" - "math/rand" - "os" - "testing" - "time" - - "github.com/PeerDB-io/peer-flow/activities" - "github.com/PeerDB-io/peer-flow/connectors/utils" - "github.com/PeerDB-io/peer-flow/generated/protos" - util "github.com/PeerDB-io/peer-flow/utils" - peerflow "github.com/PeerDB-io/peer-flow/workflows" - "github.com/jackc/pgx/v5/pgxpool" - "github.com/joho/godotenv" - log "github.com/sirupsen/logrus" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" - "go.temporal.io/sdk/testsuite" -) - -type E2EPeerFlowTestSuite struct { - suite.Suite - testsuite.WorkflowTestSuite - - pgConnStr string - pool *pgxpool.Pool - - bqHelper *BigQueryTestHelper - sfHelper *SnowflakeTestHelper - ehHelper *EventHubTestHelper - s3Helper *S3TestHelper - sqlsHelper *SQLServerHelper -} - -func TestE2EPeerFlowTestSuite(t *testing.T) { - suite.Run(t, new(E2EPeerFlowTestSuite)) -} - -const ( - postgresHost = "localhost" - postgresUser = "postgres" - postgresPassword = "postgres" - postgresDatabase = "postgres" - postgresPort = 7132 -) - -func GetTestPostgresConf() *protos.PostgresConfig { - return &protos.PostgresConfig{ - Host: postgresHost, - Port: uint32(postgresPort), - User: postgresUser, - Password: postgresPassword, - Database: postgresDatabase, - } -} - -// setupPostgres sets up the postgres connection pool. -func (s *E2EPeerFlowTestSuite) setupPostgres() error { - s.pgConnStr = utils.GetPGConnectionString(GetTestPostgresConf()) - pool, err := pgxpool.New(context.Background(), s.pgConnStr) - if err != nil { - return fmt.Errorf("failed to create postgres connection pool: %w", err) - } - - s.pool = pool - - // drop the e2e_test schema if it exists - _, err = s.pool.Exec(context.Background(), "DROP SCHEMA IF EXISTS e2e_test CASCADE") - if err != nil { - return fmt.Errorf("failed to drop e2e_test schema: %w", err) - } - - // create an e2e_test schema - _, err = s.pool.Exec(context.Background(), "CREATE SCHEMA e2e_test") - if err != nil { - return fmt.Errorf("failed to create e2e_test schema: %w", err) - } - - // drop all open slots - _, err = s.pool.Exec( - context.Background(), - "SELECT pg_drop_replication_slot(slot_name) FROM pg_replication_slots", - ) - if err != nil { - return fmt.Errorf("failed to drop replication slots: %w", err) - } - - // list all publications from pg_publication table - rows, err := s.pool.Query(context.Background(), "SELECT pubname FROM pg_publication") - if err != nil { - return fmt.Errorf("failed to list publications: %w", err) - } - - // drop all publications - for rows.Next() { - var pubName string - err = rows.Scan(&pubName) - if err != nil { - return fmt.Errorf("failed to scan publication name: %w", err) - } - - _, err = s.pool.Exec(context.Background(), fmt.Sprintf("DROP PUBLICATION %s", pubName)) - if err != nil { - return fmt.Errorf("failed to drop publication %s: %w", pubName, err) - } - } - - return nil -} - -// setupBigQuery sets up the bigquery connection. -func (s *E2EPeerFlowTestSuite) setupBigQuery() error { - bqHelper, err := NewBigQueryTestHelper() - if err != nil { - return fmt.Errorf("failed to create bigquery helper: %w", err) - } - - err = bqHelper.RecreateDataset() - if err != nil { - return fmt.Errorf("failed to recreate bigquery dataset: %w", err) - } - - s.bqHelper = bqHelper - return nil -} - -// setupSnowflake sets up the snowflake connection. -func (s *E2EPeerFlowTestSuite) setupSnowflake() error { - runID, err := util.RandomUInt64() - if err != nil { - return fmt.Errorf("failed to generate random uint64: %w", err) - } - - testSchemaName := fmt.Sprintf("e2e_test_%d", runID) - - sfHelper, err := NewSnowflakeTestHelper(testSchemaName) - if err != nil { - return fmt.Errorf("failed to create snowflake helper: %w", err) - } - - err = sfHelper.RecreateSchema() - if err != nil { - return fmt.Errorf("failed to recreate snowflake schema: %w", err) - } - s.sfHelper = sfHelper - - // for every test, drop the _PEERDB_INTERNAL schema - err = s.sfHelper.client.DropSchema("_PEERDB_INTERNAL") - require.NoError(s.T(), err) - - return nil -} - -// setup sql server connection -func (s *E2EPeerFlowTestSuite) setupSQLServer() { - env := os.Getenv("ENABLE_SQLSERVER_TESTS") - if env != "true" { - s.sqlsHelper = nil - return - } - - sqlsHelper, err := NewSQLServerHelper("test_sqlserver_peer") - require.NoError(s.T(), err) - s.sqlsHelper = sqlsHelper -} - -// Implement SetupAllSuite interface to setup the test suite -func (s *E2EPeerFlowTestSuite) SetupSuite() { - err := godotenv.Load() - if err != nil { - // it's okay if the .env file is not present - // we will use the default values - log.Infof("Unable to load .env file, using default values from env") - } - - log.SetReportCaller(true) - - // seed the random number generator with current time - rand.Seed(time.Now().UnixNano()) - - err = s.setupPostgres() - if err != nil { - s.Fail("failed to setup postgres", err) - } - - err = s.setupBigQuery() - if err != nil { - s.Fail("failed to setup bigquery", err) - } - - err = s.setupSnowflake() - if err != nil { - s.Fail("failed to setup snowflake", err) - } - - err = s.setupEventHub() - if err != nil { - s.Fail("failed to setup eventhub", err) - } - - err = s.setupS3() - if err != nil { - s.Fail("failed to setup s3", err) - } - - s.setupSQLServer() -} - -// Implement TearDownAllSuite interface to tear down the test suite -func (s *E2EPeerFlowTestSuite) TearDownSuite() { - // drop the e2e_test schema - _, err := s.pool.Exec(context.Background(), "DROP SCHEMA e2e_test CASCADE") - if err != nil { - s.Fail("failed to drop e2e_test schema", err) - } - - if s.pool != nil { - s.pool.Close() - } - - err = s.bqHelper.DropDataset() - if err != nil { - s.Fail("failed to drop bigquery dataset", err) - } - - if s.sfHelper != nil { - err = s.sfHelper.DropSchema() - if err != nil { - s.Fail("failed to drop snowflake schema", err) - } - } else { - s.Fail("snowflake helper is nil, unable to drop snowflake schema") - } - - if s.ehHelper != nil { - err = s.ehHelper.CleanUp() - if err != nil { - s.Fail("failed to clean up eventhub", err) - } - } - - if s.s3Helper != nil { - err = s.s3Helper.CleanUp() - if err != nil { - s.Fail("failed to clean up s3", err) - } - } - - if s.sqlsHelper != nil { - err = s.sqlsHelper.CleanUp() - if err != nil { - s.Fail("failed to clean up sqlserver", err) - } - } -} - -func (s *E2EPeerFlowTestSuite) TearDownTest() { - // clear all replication slots - _, err := s.pool.Exec( - context.Background(), - "SELECT pg_drop_replication_slot(slot_name) FROM pg_replication_slots", - ) - if err != nil { - s.Fail("failed to drop replication slots", err) - } -} - -func registerWorkflowsAndActivities(env *testsuite.TestWorkflowEnvironment) { - // set a 300 second timeout for the workflow to execute a few runs. - env.SetTestTimeout(300 * time.Second) - - env.RegisterWorkflow(peerflow.PeerFlowWorkflow) - env.RegisterWorkflow(peerflow.PeerFlowWorkflowWithConfig) - env.RegisterWorkflow(peerflow.SyncFlowWorkflow) - env.RegisterWorkflow(peerflow.SetupFlowWorkflow) - env.RegisterWorkflow(peerflow.SnapshotFlowWorkflow) - env.RegisterWorkflow(peerflow.NormalizeFlowWorkflow) - env.RegisterWorkflow(peerflow.QRepFlowWorkflow) - env.RegisterWorkflow(peerflow.QRepPartitionWorkflow) - env.RegisterActivity(&activities.FetchConfigActivity{}) - env.RegisterActivity(&activities.FlowableActivity{}) - env.RegisterActivity(&activities.SnapshotActivity{}) -} - -func (s *E2EPeerFlowTestSuite) Test_Invalid_Connection_Config() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - // TODO (kaushikiska): ensure flow name can only be alpha numeric and underscores. - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 1, - MaxBatchSize: 1, - } - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, nil, &limits, nil) - - // Verify workflow completes - s.True(env.IsWorkflowCompleted()) - err := env.GetWorkflowError() - - // assert that error contains "invalid connection configs" - s.Error(err) - s.Contains(err.Error(), "invalid connection configs") - - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Complete_Flow_No_Data() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - CREATE TABLE e2e_test.test ( - id SERIAL PRIMARY KEY, - key TEXT NOT NULL, - value VARCHAR(255) NOT NULL - ); - `) - s.NoError(err) - - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_complete_flow_no_data", - TableNameMapping: map[string]string{"e2e_test.test": "test"}, - PostgresPort: postgresPort, - Destination: s.bqHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 1, - MaxBatchSize: 1, - } - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Char_ColType_Error() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - CREATE TABLE e2e_test.test_char_table ( - id SERIAL PRIMARY KEY, - key TEXT NOT NULL, - value CHAR(255) NOT NULL - ); - `) - s.NoError(err) - - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_char_table", - TableNameMapping: map[string]string{"e2e_test.test_char_table": "test"}, - PostgresPort: postgresPort, - Destination: s.bqHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 1, - MaxBatchSize: 1, - } - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - env.AssertExpectations(s.T()) -} - -// Test_Complete_Simple_Flow_BQ tests a complete flow with data in the source table. -// The test inserts 10 rows into the source table and verifies that the data is -// correctly synced to the destination table after sync flow completes. -func (s *E2EPeerFlowTestSuite) Test_Complete_Simple_Flow_BQ() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - CREATE TABLE e2e_test.test_simple_flow_bq ( - id SERIAL PRIMARY KEY, - key TEXT NOT NULL, - value TEXT NOT NULL - ); - `) - s.NoError(err) - - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_complete_single_col_flow_bq", - TableNameMapping: map[string]string{"e2e_test.test_simple_flow_bq": "test_simple_flow_bq"}, - PostgresPort: postgresPort, - Destination: s.bqHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 2, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and then insert 10 rows into the source table - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - // insert 10 rows into the source table - for i := 0; i < 10; i++ { - testKey := fmt.Sprintf("test_key_%d", i) - testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.pool.Exec(context.Background(), ` - INSERT INTO e2e_test.test_simple_flow_bq (key, value) VALUES ($1, $2) - `, testKey, testValue) - s.NoError(err) - } - fmt.Println("Inserted 10 rows into the source table") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - // TODO: verify that the data is correctly synced to the destination table - // on the bigquery side - - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Toast_BQ() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - - CREATE TABLE e2e_test.test_toast_bq_1 ( - id SERIAL PRIMARY KEY, - t1 text, - t2 text, - k int - );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; - `) - s.NoError(err) - - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_toast_bq_1", - TableNameMapping: map[string]string{"e2e_test.test_toast_bq_1": "test_toast_bq_1"}, - PostgresPort: postgresPort, - Destination: s.bqHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 1, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and execute a transaction touching toast columns - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - /* - Executing a transaction which - 1. changes both toast column - 2. changes no toast column - 2. changes 1 toast column - */ - _, err = s.pool.Exec(context.Background(), ` - BEGIN; - INSERT INTO e2e_test.test_toast_bq_1(t1,t2,k) SELECT random_string(9000),random_string(9000), - 1 FROM generate_series(1,2); - UPDATE e2e_test.test_toast_bq_1 SET k=102 WHERE id=1; - UPDATE e2e_test.test_toast_bq_1 SET t1='dummy' WHERE id=2; - END; - `) - s.NoError(err) - fmt.Println("Executed a transaction touching toast columns") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - s.compareTableContentsBQ("test_toast_bq_1", "id,t1,t2,k") - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Toast_Nochanges_BQ() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - - CREATE TABLE e2e_test.test_toast_bq_2 ( - id SERIAL PRIMARY KEY, - t1 text, - t2 text, - k int - );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; - `) - s.NoError(err) - - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_toast_bq_2", - TableNameMapping: map[string]string{"e2e_test.test_toast_bq_2": "test_toast_bq_2"}, - PostgresPort: postgresPort, - Destination: s.bqHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 1, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and execute a transaction touching toast columns - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - /* transaction updating no rows */ - _, err = s.pool.Exec(context.Background(), ` - BEGIN; - UPDATE e2e_test.test_toast_bq_2 SET k=102 WHERE id=1; - UPDATE e2e_test.test_toast_bq_2 SET t1='dummy' WHERE id=2; - END; - `) - s.NoError(err) - fmt.Println("Executed a transaction touching toast columns") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - s.compareTableContentsBQ("test_toast_bq_2", "id,t1,t2,k") - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Toast_Advance_1_BQ() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - - CREATE TABLE e2e_test.test_toast_bq_3 ( - id SERIAL PRIMARY KEY, - t1 text, - t2 text, - k int - );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; - `) - s.NoError(err) - - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_toast_bq_3", - TableNameMapping: map[string]string{"e2e_test.test_toast_bq_3": "test_toast_bq_3"}, - PostgresPort: postgresPort, - Destination: s.bqHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 1, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and execute a transaction touching toast columns - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - //complex transaction with random DMLs on a table with toast columns - _, err = s.pool.Exec(context.Background(), ` - BEGIN; - INSERT INTO e2e_test.test_toast_bq_3(t1,t2,k) SELECT random_string(9000),random_string(9000), - 1 FROM generate_series(1,2); - UPDATE e2e_test.test_toast_bq_3 SET k=102 WHERE id=1; - UPDATE e2e_test.test_toast_bq_3 SET t1='dummy' WHERE id=2; - UPDATE e2e_test.test_toast_bq_3 SET t2='dummy' WHERE id=2; - DELETE FROM e2e_test.test_toast_bq_3 WHERE id=1; - INSERT INTO e2e_test.test_toast_bq_3(t1,t2,k) SELECT random_string(9000),random_string(9000), - 1 FROM generate_series(1,2); - UPDATE e2e_test.test_toast_bq_3 SET k=1 WHERE id=1; - UPDATE e2e_test.test_toast_bq_3 SET t1='dummy1',t2='dummy2' WHERE id=1; - UPDATE e2e_test.test_toast_bq_3 SET t1='dummy3' WHERE id=3; - DELETE FROM e2e_test.test_toast_bq_3 WHERE id=2; - DELETE FROM e2e_test.test_toast_bq_3 WHERE id=3; - DELETE FROM e2e_test.test_toast_bq_3 WHERE id=2; - END; - `) - s.NoError(err) - fmt.Println("Executed a transaction touching toast columns") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - s.compareTableContentsBQ("test_toast_bq_3", "id,t1,t2,k") - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) SetupPeerFlowStatusQuery(env *testsuite.TestWorkflowEnvironment, - connectionGen FlowConnectionGenerationConfig) { - // wait for PeerFlowStatusQuery to finish setup - // sleep for 5 second to allow the workflow to start - time.Sleep(5 * time.Second) - for { - response, err := env.QueryWorkflow( - peerflow.PeerFlowStatusQuery, - connectionGen.FlowJobName, - ) - if err == nil { - var state peerflow.PeerFlowState - err = response.Get(&state) - s.NoError(err) - - if state.SetupComplete { - fmt.Println("query indicates setup is complete") - break - } - } else { - // log the error for informational purposes - fmt.Println(err) - } - time.Sleep(1 * time.Second) - } -} - -func (s *E2EPeerFlowTestSuite) Test_Toast_Advance_2_BQ() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - - CREATE TABLE e2e_test.test_toast_bq_4 ( - id SERIAL PRIMARY KEY, - t1 text, - k int - );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; - `) - s.NoError(err) - - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_toast_bq_4", - TableNameMapping: map[string]string{"e2e_test.test_toast_bq_4": "test_toast_bq_4"}, - PostgresPort: postgresPort, - Destination: s.bqHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 1, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and execute a transaction touching toast columns - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - //complex transaction with random DMLs on a table with toast columns - _, err = s.pool.Exec(context.Background(), ` - BEGIN; - INSERT INTO e2e_test.test_toast_bq_4(t1,k) SELECT random_string(9000), - 1 FROM generate_series(1,1); - UPDATE e2e_test.test_toast_bq_4 SET t1=sub.t1 FROM (SELECT random_string(9000) t1 - FROM generate_series(1,1) ) sub WHERE id=1; - UPDATE e2e_test.test_toast_bq_4 SET k=2 WHERE id=1; - UPDATE e2e_test.test_toast_bq_4 SET k=3 WHERE id=1; - UPDATE e2e_test.test_toast_bq_4 SET t1=sub.t1 FROM (SELECT random_string(9000) t1 - FROM generate_series(1,1)) sub WHERE id=1; - UPDATE e2e_test.test_toast_bq_4 SET k=4 WHERE id=1; - END; - `) - s.NoError(err) - fmt.Println("Executed a transaction touching toast columns") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - s.compareTableContentsBQ("test_toast_bq_4", "id,t1,k") - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Toast_Advance_3_BQ() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - - CREATE TABLE e2e_test.test_toast_bq_5 ( - id SERIAL PRIMARY KEY, - t1 text, - t2 text, - k int - );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; - `) - s.NoError(err) - - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_toast_bq_5", - TableNameMapping: map[string]string{"e2e_test.test_toast_bq_5": "test_toast_bq_5"}, - PostgresPort: postgresPort, - Destination: s.bqHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 1, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and execute a transaction touching toast columns - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - /* - transaction updating a single row - multiple times with changed/unchanged toast columns - */ - _, err = s.pool.Exec(context.Background(), ` - BEGIN; - INSERT INTO e2e_test.test_toast_bq_5(t1,t2,k) SELECT random_string(9000),random_string(9000), - 1 FROM generate_series(1,1); - UPDATE e2e_test.test_toast_bq_5 SET k=102 WHERE id=1; - UPDATE e2e_test.test_toast_bq_5 SET t1='dummy' WHERE id=1; - UPDATE e2e_test.test_toast_bq_5 SET t2='dummy' WHERE id=1; - END; - `) - s.NoError(err) - fmt.Println("Executed a transaction touching toast columns") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - s.compareTableContentsBQ("test_toast_bq_5", "id,t1,t2,k") - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Types_BQ() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - - CREATE TABLE e2e_test.test_types_bq(id serial PRIMARY KEY,c1 BIGINT,c2 BIT,c3 VARBIT,c4 BOOLEAN, - c6 BYTEA,c7 CHARACTER,c8 varchar,c9 CIDR,c11 DATE,c12 FLOAT,c13 DOUBLE PRECISION, - c14 INET,c15 INTEGER,c16 INTERVAL,c17 JSON,c18 JSONB,c21 MACADDR,c22 MONEY, - c23 NUMERIC,c24 OID,c28 REAL,c29 SMALLINT,c30 SMALLSERIAL,c31 SERIAL,c32 TEXT, - c33 TIMESTAMP,c34 TIMESTAMPTZ,c35 TIME, c36 TIMETZ,c37 TSQUERY,c38 TSVECTOR, - c39 TXID_SNAPSHOT,c40 UUID,c41 XML, c42 INT[], c43 FLOAT[]); - CREATE OR REPLACE FUNCTION random_bytea(bytea_length integer) - RETURNS bytea AS $body$ - SELECT decode(string_agg(lpad(to_hex(width_bucket(random(), 0, 1, 256)-1),2,'0') ,''), 'hex') - FROM generate_series(1, $1); - $body$ - LANGUAGE 'sql' - VOLATILE - SET search_path = 'pg_catalog'; - `) - s.NoError(err) - - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_types_bq", - TableNameMapping: map[string]string{"e2e_test.test_types_bq": "test_types_bq"}, - PostgresPort: postgresPort, - Destination: s.bqHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - - TotalSyncFlows: 1, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and execute a transaction touching toast columns - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - /* test inserting various types*/ - _, err = s.pool.Exec(context.Background(), ` - INSERT INTO e2e_test.test_types_bq SELECT 2,2,b'1',b'101', - true,random_bytea(32),'s','test','1.1.10.2'::cidr, - CURRENT_DATE,1.23,1.234,'192.168.1.5'::inet,1, - '5 years 2 months 29 days 1 minute 2 seconds 200 milliseconds 20000 microseconds'::interval, - '{"sai":1}'::json,'{"sai":1}'::jsonb,'08:00:2b:01:02:03'::macaddr, - 1.2,1.23,4::oid,1.23,1,1,1,'test',now(),now(),now()::time,now()::timetz, - 'fat & rat'::tsquery,'a fat cat sat on a mat and ate a fat rat'::tsvector, - txid_current_snapshot(), - '66073c38-b8df-4bdb-bbca-1c97596b8940'::uuid,xmlcomment('hello'), - ARRAY[10299301,2579827], - ARRAY[0.0003, 8902.0092]; - `) - s.NoError(err) - fmt.Println("Executed an insert with all types") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - noNulls, err := s.bqHelper.CheckNull("test_types_bq", []string{"c41", "c1", "c2", "c3", "c4", - "c6", "c39", "c40", "id", "c9", "c11", "c12", "c13", "c14", "c15", "c16", "c17", "c18", - "c21", "c22", "c23", "c24", "c28", "c29", "c30", "c31", "c33", "c34", "c35", "c36", - "c37", "c38", "c7", "c8", "c32", "c42", "c43"}) - if err != nil { - fmt.Println("error %w", err) - } - // Make sure that there are no nulls - s.Equal(noNulls, true) - - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Types_Avro_BQ() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - - CREATE TABLE e2e_test.test_types_avro_bq(id serial PRIMARY KEY,c1 BIGINT,c2 BIT,c3 VARBIT,c4 BOOLEAN, - c6 BYTEA,c7 CHARACTER,c8 varchar,c9 CIDR,c11 DATE,c12 FLOAT,c13 DOUBLE PRECISION, - c14 INET,c15 INTEGER,c16 INTERVAL,c17 JSON,c18 JSONB,c21 MACADDR,c22 MONEY, - c23 NUMERIC,c24 OID,c28 REAL,c29 SMALLINT,c30 SMALLSERIAL,c31 SERIAL,c32 TEXT, - c33 TIMESTAMP,c34 TIMESTAMPTZ,c35 TIME, c36 TIMETZ,c37 TSQUERY,c38 TSVECTOR, - c39 TXID_SNAPSHOT,c40 UUID,c41 XML, c42 INT[], c43 FLOAT[], c44 TEXT[]); - CREATE OR REPLACE FUNCTION random_bytea(bytea_length integer) - RETURNS bytea AS $body$ - SELECT decode(string_agg(lpad(to_hex(width_bucket(random(), 0, 1, 256)-1),2,'0') ,''), 'hex') - FROM generate_series(1, $1); - $body$ - LANGUAGE 'sql' - VOLATILE - SET search_path = 'pg_catalog'; - `) - s.NoError(err) - - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_types_avro_bq", - TableNameMapping: map[string]string{"e2e_test.test_types_avro_bq": "test_types_avro_bq"}, - PostgresPort: postgresPort, - Destination: s.bqHelper.Peer, - CDCSyncMode: protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, - CdcStagingPath: "peerdb_staging", - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - - TotalSyncFlows: 1, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and execute a transaction touching toast columns - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - /* test inserting various types*/ - _, err = s.pool.Exec(context.Background(), ` - INSERT INTO e2e_test.test_types_avro_bq SELECT 2,2,b'1',b'101', - true,random_bytea(32),'s','test','1.1.10.2'::cidr, - CURRENT_DATE,1.23,1.234,'192.168.1.5'::inet,1, - '5 years 2 months 29 days 1 minute 2 seconds 200 milliseconds 20000 microseconds'::interval, - '{"sai":1}'::json,'{"sai":1}'::jsonb,'08:00:2b:01:02:03'::macaddr, - 1.2,1.23,4::oid,1.23,1,1,1,'test',now(),now(),now()::time,now()::timetz, - 'fat & rat'::tsquery,'a fat cat sat on a mat and ate a fat rat'::tsvector, - txid_current_snapshot(), - '66073c38-b8df-4bdb-bbca-1c97596b8940'::uuid,xmlcomment('hello'), - ARRAY[9301,239827], - ARRAY[0.0003, 1039.0034], - ARRAY['hello','bye']; - `) - s.NoError(err) - fmt.Println("Executed an insert with all types") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - noNulls, err := s.bqHelper.CheckNull("test_types_avro_bq", []string{"c41", "c1", "c2", "c3", "c4", - "c6", "c39", "c40", "id", "c9", "c11", "c12", "c13", "c14", "c15", "c16", "c17", "c18", - "c21", "c22", "c23", "c24", "c28", "c29", "c30", "c31", "c33", "c34", "c35", "c36", - "c37", "c38", "c7", "c8", "c32", "c42", "c43"}) - if err != nil { - fmt.Println("error %w", err) - } - // Make sure that there are no nulls - s.Equal(noNulls, true) - - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Simple_Flow_BQ_Avro_CDC() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - CREATE TABLE e2e_test.test_simple_flow_bq_avro_cdc ( - id SERIAL PRIMARY KEY, - key TEXT NOT NULL, - value TEXT NOT NULL - ); - `) - s.NoError(err) - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_simple_flow_bq_avro_cdc", - TableNameMapping: map[string]string{"e2e_test.test_simple_flow_bq_avro_cdc": "test_simple_flow_bq_avro_cdc"}, - PostgresPort: postgresPort, - Destination: s.bqHelper.Peer, - CDCSyncMode: protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, - CdcStagingPath: "peerdb_staging", - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 2, - MaxBatchSize: 100, - } - - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - for i := 0; i < 10; i++ { - testKey := fmt.Sprintf("test_key_%d", i) - testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.pool.Exec(context.Background(), ` - INSERT INTO e2e_test.test_simple_flow_bq_avro_cdc (key, value) VALUES ($1, $2) - `, testKey, testValue) - s.NoError(err) - } - fmt.Println("Inserted 10 rows into the source table") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - count, err := s.bqHelper.CountRows("test_simple_flow_bq") - s.NoError(err) - s.Equal(10, count) - - // TODO: verify that the data is correctly synced to the destination table - // on the bigquery side - - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Multi_Table_BQ() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - CREATE TABLE e2e_test.test1_bq(id serial primary key, c1 int, c2 text); - CREATE TABLE e2e_test.test2_bq(id serial primary key, c1 int, c2 text); - `) - s.NoError(err) - - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_multi_table_bq", - TableNameMapping: map[string]string{"e2e_test.test1_bq": "test1_bq", "e2e_test.test2_bq": "test2_bq"}, - PostgresPort: postgresPort, - Destination: s.bqHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 1, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and execute a transaction touching toast columns - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - /* inserting across multiple tables*/ - _, err = s.pool.Exec(context.Background(), ` - INSERT INTO e2e_test.test1_bq(c1,c2) VALUES (1,'dummy_1'); - INSERT INTO e2e_test.test2_bq(c1,c2) VALUES (-1,'dummy_-1'); - `) - s.NoError(err) - fmt.Println("Executed an insert with all types") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - require.True(s.T(), env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - count1, err := s.bqHelper.CountRows("test1_bq") - s.NoError(err) - count2, err := s.bqHelper.CountRows("test2_bq") - s.NoError(err) - - s.Equal(1, count1) - s.Equal(1, count2) - - env.AssertExpectations(s.T()) -} - -// tests for snowflake - -func (s *E2EPeerFlowTestSuite) Test_Complete_Simple_Flow_SF() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - CREATE TABLE e2e_test.test_simple_flow_sf ( - id SERIAL PRIMARY KEY, - key TEXT NOT NULL, - value TEXT NOT NULL - ); - `) - s.NoError(err) - tableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_simple_flow_sf") - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_complete_single_col_flow_sf", - TableNameMapping: map[string]string{"e2e_test.test_simple_flow_sf": tableName}, - PostgresPort: postgresPort, - Destination: s.sfHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 2, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and then insert 10 rows into the source table - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - // insert 10 rows into the source table - for i := 0; i < 10; i++ { - testKey := fmt.Sprintf("test_key_%d", i) - testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.pool.Exec(context.Background(), ` - INSERT INTO e2e_test.test_simple_flow_sf (key, value) VALUES ($1, $2) - `, testKey, testValue) - s.NoError(err) - } - fmt.Println("Inserted 10 rows into the source table") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - count, err := s.sfHelper.CountRows("test_simple_flow_sf") - s.NoError(err) - s.Equal(10, count) - - // TODO: verify that the data is correctly synced to the destination table - // on the bigquery side - - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Complete_Simple_Flow_SF_Avro_CDC() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - CREATE TABLE e2e_test.test_simple_flow_sf_avro_cdc ( - id SERIAL PRIMARY KEY, - key TEXT NOT NULL, - value TEXT NOT NULL - ); - `) - s.NoError(err) - tableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_simple_flow_sf_avro_cdc") - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_complete_single_col_flow_sf", - TableNameMapping: map[string]string{"e2e_test.test_simple_flow_sf_avro_cdc": tableName}, - PostgresPort: postgresPort, - Destination: s.sfHelper.Peer, - CDCSyncMode: protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 2, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and then insert 10 rows into the source table - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - // insert 10 rows into the source table - for i := 0; i < 10; i++ { - testKey := fmt.Sprintf("test_key_%d", i) - testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.pool.Exec(context.Background(), ` - INSERT INTO e2e_test.test_simple_flow_sf_avro_cdc (key, value) VALUES ($1, $2) - `, testKey, testValue) - s.NoError(err) - } - fmt.Println("Inserted 10 rows into the source table") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - count, err := s.sfHelper.CountRows("test_simple_flow_sf") - s.NoError(err) - s.Equal(10, count) - - // TODO: verify that the data is correctly synced to the destination table - // on the bigquery side - - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Toast_SF() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - - CREATE TABLE e2e_test.test_toast_sf_1 ( - id SERIAL PRIMARY KEY, - t1 text, - t2 text, - k int - );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; - `) - s.NoError(err) - - tableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_1") - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_toast_sf_1", - TableNameMapping: map[string]string{"e2e_test.test_toast_sf_1": tableName}, - PostgresPort: postgresPort, - Destination: s.sfHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 1, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and execute a transaction touching toast columns - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - /* - Executing a transaction which - 1. changes both toast column - 2. changes no toast column - 2. changes 1 toast column - */ - _, err = s.pool.Exec(context.Background(), ` - BEGIN; - INSERT INTO e2e_test.test_toast_sf_1(t1,t2,k) SELECT random_string(9000),random_string(9000), - 1 FROM generate_series(1,2); - UPDATE e2e_test.test_toast_sf_1 SET k=102 WHERE id=1; - UPDATE e2e_test.test_toast_sf_1 SET t1='dummy' WHERE id=2; - END; - `) - s.NoError(err) - fmt.Println("Executed a transaction touching toast columns") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - s.compareTableContentsSF("test_toast_sf_1", `id,t1,t2,k`, false) - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Toast_Nochanges_SF() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - - CREATE TABLE e2e_test.test_toast_sf_2 ( - id SERIAL PRIMARY KEY, - t1 text, - t2 text, - k int - );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; - `) - s.NoError(err) - - tableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_2") - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_toast_sf_2", - TableNameMapping: map[string]string{"e2e_test.test_toast_sf_2": tableName}, - PostgresPort: postgresPort, - Destination: s.sfHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 1, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and execute a transaction touching toast columns - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - /* transaction updating no rows */ - _, err = s.pool.Exec(context.Background(), ` - BEGIN; - UPDATE e2e_test.test_toast_sf_2 SET k=102 WHERE id=1; - UPDATE e2e_test.test_toast_sf_2 SET t1='dummy' WHERE id=2; - END; - `) - s.NoError(err) - fmt.Println("Executed a transaction touching toast columns") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - s.compareTableContentsSF("test_toast_sf_2", `id,t1,t2,k`, false) - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Toast_Advance_1_SF() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - - CREATE TABLE e2e_test.test_toast_sf_3 ( - id SERIAL PRIMARY KEY, - t1 text, - t2 text, - k int - );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; - `) - s.NoError(err) - - tableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_3") - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_toast_sf_3", - TableNameMapping: map[string]string{"e2e_test.test_toast_sf_3": tableName}, - PostgresPort: postgresPort, - Destination: s.sfHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 2, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and execute a transaction touching toast columns - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - //complex transaction with random DMLs on a table with toast columns - _, err = s.pool.Exec(context.Background(), ` - BEGIN; - INSERT INTO e2e_test.test_toast_sf_3(t1,t2,k) SELECT random_string(9000),random_string(9000), - 1 FROM generate_series(1,2); - UPDATE e2e_test.test_toast_sf_3 SET k=102 WHERE id=1; - UPDATE e2e_test.test_toast_sf_3 SET t1='dummy' WHERE id=2; - UPDATE e2e_test.test_toast_sf_3 SET t2='dummy' WHERE id=2; - DELETE FROM e2e_test.test_toast_sf_3 WHERE id=1; - INSERT INTO e2e_test.test_toast_sf_3(t1,t2,k) SELECT random_string(9000),random_string(9000), - 1 FROM generate_series(1,2); - UPDATE e2e_test.test_toast_sf_3 SET k=1 WHERE id=1; - UPDATE e2e_test.test_toast_sf_3 SET t1='dummy1',t2='dummy2' WHERE id=1; - UPDATE e2e_test.test_toast_sf_3 SET t1='dummy3' WHERE id=3; - DELETE FROM e2e_test.test_toast_sf_3 WHERE id=2; - DELETE FROM e2e_test.test_toast_sf_3 WHERE id=3; - DELETE FROM e2e_test.test_toast_sf_3 WHERE id=2; - END; - `) - s.NoError(err) - fmt.Println("Executed a transaction touching toast columns") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - s.compareTableContentsSF("test_toast_sf_3", `id,t1,t2,k`, false) - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Toast_Advance_2_SF() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - - CREATE TABLE e2e_test.test_toast_sf_4 ( - id SERIAL PRIMARY KEY, - t1 text, - k int - );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; - `) - s.NoError(err) - - tableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_4") - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_toast_sf_4", - TableNameMapping: map[string]string{"e2e_test.test_toast_sf_4": tableName}, - PostgresPort: postgresPort, - Destination: s.sfHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 1, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and execute a transaction touching toast columns - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - //complex transaction with random DMLs on a table with toast columns - _, err = s.pool.Exec(context.Background(), ` - BEGIN; - INSERT INTO e2e_test.test_toast_sf_4(t1,k) SELECT random_string(9000), - 1 FROM generate_series(1,1); - UPDATE e2e_test.test_toast_sf_4 SET t1=sub.t1 FROM (SELECT random_string(9000) t1 - FROM generate_series(1,1) ) sub WHERE id=1; - UPDATE e2e_test.test_toast_sf_4 SET k=2 WHERE id=1; - UPDATE e2e_test.test_toast_sf_4 SET k=3 WHERE id=1; - UPDATE e2e_test.test_toast_sf_4 SET t1=sub.t1 FROM (SELECT random_string(9000) t1 - FROM generate_series(1,1)) sub WHERE id=1; - UPDATE e2e_test.test_toast_sf_4 SET k=4 WHERE id=1; - END; - `) - s.NoError(err) - fmt.Println("Executed a transaction touching toast columns") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - s.compareTableContentsSF("test_toast_sf_4", `id,t1,k`, false) - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Toast_Advance_3_SF() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - - CREATE TABLE e2e_test.test_toast_sf_5 ( - id SERIAL PRIMARY KEY, - t1 text, - t2 text, - k int - );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; - `) - s.NoError(err) - - tableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_5") - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_toast_sf_5", - TableNameMapping: map[string]string{"e2e_test.test_toast_sf_5": tableName}, - PostgresPort: postgresPort, - Destination: s.sfHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 1, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and execute a transaction touching toast columns - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - /* - transaction updating a single row - multiple times with changed/unchanged toast columns - */ - _, err = s.pool.Exec(context.Background(), ` - BEGIN; - INSERT INTO e2e_test.test_toast_sf_5(t1,t2,k) SELECT random_string(9000),random_string(9000), - 1 FROM generate_series(1,1); - UPDATE e2e_test.test_toast_sf_5 SET k=102 WHERE id=1; - UPDATE e2e_test.test_toast_sf_5 SET t1='dummy' WHERE id=1; - UPDATE e2e_test.test_toast_sf_5 SET t2='dummy' WHERE id=1; - END; - `) - s.NoError(err) - fmt.Println("Executed a transaction touching toast columns") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - s.compareTableContentsSF("test_toast_sf_5", `id,t1,t2,k`, false) - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Types_SF() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - - CREATE TABLE e2e_test.test_types_sf(id serial PRIMARY KEY,c1 BIGINT,c2 BIT,c3 VARBIT,c4 BOOLEAN, - c6 BYTEA,c7 CHARACTER,c8 varchar,c9 CIDR,c11 DATE,c12 FLOAT,c13 DOUBLE PRECISION, - c14 INET,c15 INTEGER,c16 INTERVAL,c17 JSON,c18 JSONB,c21 MACADDR,c22 MONEY, - c23 NUMERIC,c24 OID,c28 REAL,c29 SMALLINT,c30 SMALLSERIAL,c31 SERIAL,c32 TEXT, - c33 TIMESTAMP,c34 TIMESTAMPTZ,c35 TIME, c36 TIMETZ,c37 TSQUERY,c38 TSVECTOR, - c39 TXID_SNAPSHOT,c40 UUID,c41 XML); - CREATE OR REPLACE FUNCTION random_bytea(bytea_length integer) - RETURNS bytea AS $body$ - SELECT decode(string_agg(lpad(to_hex(width_bucket(random(), 0, 1, 256)-1),2,'0') ,''), 'hex') - FROM generate_series(1, $1); - $body$ - LANGUAGE 'sql' - VOLATILE - SET search_path = 'pg_catalog'; - `) - s.NoError(err) - - tableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_types_sf") - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_types_sf", - TableNameMapping: map[string]string{"e2e_test.test_types_sf": tableName}, - PostgresPort: postgresPort, - Destination: s.sfHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 1, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and execute a transaction touching toast columns - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - /* test inserting various types*/ - _, err = s.pool.Exec(context.Background(), ` - INSERT INTO e2e_test.test_types_sf SELECT 2,2,b'1',b'101', - true,random_bytea(32),'s','test','1.1.10.2'::cidr, - CURRENT_DATE,1.23,1.234,'192.168.1.5'::inet,1, - '5 years 2 months 29 days 1 minute 2 seconds 200 milliseconds 20000 microseconds'::interval, - '{"sai":1}'::json,'{"sai":1}'::jsonb,'08:00:2b:01:02:03'::macaddr, - 1.2,1.23,4::oid,1.23,1,1,1,'test',now(),now(),now()::time,now()::timetz, - 'fat & rat'::tsquery,'a fat cat sat on a mat and ate a fat rat'::tsvector, - txid_current_snapshot(), - '66073c38-b8df-4bdb-bbca-1c97596b8940'::uuid,xmlcomment('hello'); - `) - s.NoError(err) - fmt.Println("Executed an insert with all types") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - noNulls, err := s.sfHelper.CheckNull("test_types_sf", []string{"c41", "c1", "c2", "c3", "c4", - "c6", "c39", "c40", "id", "c9", "c11", "c12", "c13", "c14", "c15", "c16", "c17", "c18", - "c21", "c22", "c23", "c24", "c28", "c29", "c30", "c31", "c33", "c34", "c35", "c36", - "c37", "c38", "c7", "c8", "c32"}) - if err != nil { - fmt.Println("error %w", err) - } - // Make sure that there are no nulls - s.Equal(noNulls, true) - - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Types_SF_Avro_CDC() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - - CREATE TABLE e2e_test.test_types_sf_avro_cdc(id serial PRIMARY KEY,c1 BIGINT,c2 BIT,c3 VARBIT,c4 BOOLEAN, - c6 BYTEA,c7 CHARACTER,c8 varchar,c9 CIDR,c11 DATE,c12 FLOAT,c13 DOUBLE PRECISION, - c14 INET,c15 INTEGER,c16 INTERVAL,c17 JSON,c18 JSONB,c21 MACADDR,c22 MONEY, - c23 NUMERIC,c24 OID,c28 REAL,c29 SMALLINT,c30 SMALLSERIAL,c31 SERIAL,c32 TEXT, - c33 TIMESTAMP,c34 TIMESTAMPTZ,c35 TIME, c36 TIMETZ,c37 TSQUERY,c38 TSVECTOR, - c39 TXID_SNAPSHOT,c40 UUID,c41 XML); - CREATE OR REPLACE FUNCTION random_bytea(bytea_length integer) - RETURNS bytea AS $body$ - SELECT decode(string_agg(lpad(to_hex(width_bucket(random(), 0, 1, 256)-1),2,'0') ,''), 'hex') - FROM generate_series(1, $1); - $body$ - LANGUAGE 'sql' - VOLATILE - SET search_path = 'pg_catalog'; - `) - s.NoError(err) - - tableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_types_sf_avro_cdc") - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_types_sf", - TableNameMapping: map[string]string{"e2e_test.test_types_sf_avro_cdc": tableName}, - PostgresPort: postgresPort, - Destination: s.sfHelper.Peer, - CDCSyncMode: protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 1, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and execute a transaction touching toast columns - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - /* test inserting various types*/ - _, err = s.pool.Exec(context.Background(), ` - INSERT INTO e2e_test.test_types_sf_avro_cdc SELECT 2,2,b'1',b'101', - true,random_bytea(32),'s','test','1.1.10.2'::cidr, - CURRENT_DATE,1.23,1.234,'192.168.1.5'::inet,1, - '5 years 2 months 29 days 1 minute 2 seconds 200 milliseconds 20000 microseconds'::interval, - '{"sai":1}'::json,'{"sai":1}'::jsonb,'08:00:2b:01:02:03'::macaddr, - 1.2,1.23,4::oid,1.23,1,1,1,'test',now(),now(),now()::time,now()::timetz, - 'fat & rat'::tsquery,'a fat cat sat on a mat and ate a fat rat'::tsvector, - txid_current_snapshot(), - '66073c38-b8df-4bdb-bbca-1c97596b8940'::uuid,xmlcomment('hello'); - `) - s.NoError(err) - fmt.Println("Executed an insert with all types") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - noNulls, err := s.sfHelper.CheckNull("test_types_sf_avro_cdc", []string{"c41", "c1", "c2", "c3", "c4", - "c6", "c39", "c40", "id", "c9", "c11", "c12", "c13", "c14", "c15", "c16", "c17", "c18", - "c21", "c22", "c23", "c24", "c28", "c29", "c30", "c31", "c33", "c34", "c35", "c36", - "c37", "c38", "c7", "c8", "c32"}) - if err != nil { - fmt.Println("error %w", err) - } - // Make sure that there are no nulls - s.Equal(noNulls, true) - - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Multi_Table_SF() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - CREATE TABLE e2e_test.test1_sf(id serial primary key, c1 int, c2 text); - CREATE TABLE e2e_test.test2_sf(id serial primary key, c1 int, c2 text); - `) - s.NoError(err) - - table1 := fmt.Sprintf(s.sfHelper.testSchemaName + ".test1_sf") - table2 := fmt.Sprintf(s.sfHelper.testSchemaName + ".test2_sf") - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_multi_table_sf", - TableNameMapping: map[string]string{"e2e_test.test1_sf": table1, "e2e_test.test2_sf": table2}, - PostgresPort: postgresPort, - Destination: s.sfHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 1, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and execute a transaction touching toast columns - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - /* inserting across multiple tables*/ - _, err = s.pool.Exec(context.Background(), ` - INSERT INTO e2e_test.test1_sf(c1,c2) VALUES (1,'dummy_1'); - INSERT INTO e2e_test.test2_sf(c1,c2) VALUES (-1,'dummy_-1'); - `) - s.NoError(err) - fmt.Println("Executed an insert with all types") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - count1, err := s.sfHelper.CountRows("test1_sf") - s.NoError(err) - count2, err := s.sfHelper.CountRows("test2_sf") - s.NoError(err) - - s.Equal(1, count1) - s.Equal(1, count2) - - env.AssertExpectations(s.T()) -} diff --git a/flow/e2e/postgres/qrep_flow_pg_test.go b/flow/e2e/postgres/qrep_flow_pg_test.go new file mode 100644 index 0000000000..241659c229 --- /dev/null +++ b/flow/e2e/postgres/qrep_flow_pg_test.go @@ -0,0 +1,147 @@ +package e2e_postgres + +import ( + "context" + "fmt" + "testing" + + "github.com/PeerDB-io/peer-flow/e2e" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/jackc/pgx/v5/pgxpool" + "github.com/joho/godotenv" + log "github.com/sirupsen/logrus" + "github.com/stretchr/testify/suite" + "go.temporal.io/sdk/testsuite" +) + +const postgresSuffix = "postgres" + +type PeerFlowE2ETestSuitePG struct { + suite.Suite + testsuite.WorkflowTestSuite + + pool *pgxpool.Pool +} + +func TestPeerFlowE2ETestSuitePG(t *testing.T) { + suite.Run(t, new(PeerFlowE2ETestSuitePG)) +} + +// Implement SetupAllSuite interface to setup the test suite +func (s *PeerFlowE2ETestSuitePG) SetupSuite() { + err := godotenv.Load() + if err != nil { + // it's okay if the .env file is not present + // we will use the default values + log.Infof("Unable to load .env file, using default values from env") + } + + log.SetReportCaller(true) + + pool, err := e2e.SetupPostgres(postgresSuffix) + if err != nil { + s.Fail("failed to setup postgres", err) + } + s.pool = pool +} + +// Implement TearDownAllSuite interface to tear down the test suite +func (s *PeerFlowE2ETestSuitePG) TearDownSuite() { + err := e2e.TearDownPostgres(s.pool, postgresSuffix) + if err != nil { + s.Fail("failed to drop Postgres schema", err) + } +} + +func (s *PeerFlowE2ETestSuitePG) setupSourceTable(tableName string, rowCount int) { + err := e2e.CreateSourceTableQRep(s.pool, postgresSuffix, tableName) + s.NoError(err) + err = e2e.PopulateSourceTable(s.pool, postgresSuffix, tableName, rowCount) + s.NoError(err) +} + +func (s *PeerFlowE2ETestSuitePG) comparePGTables(srcSchemaQualified, dstSchemaQualified string) error { + // Execute the two EXCEPT queries + err := s.compareQuery(srcSchemaQualified, dstSchemaQualified) + if err != nil { + return err + } + + err = s.compareQuery(dstSchemaQualified, srcSchemaQualified) + if err != nil { + return err + } + + // If no error is returned, then the contents of the two tables are the same + return nil +} + +func (s *PeerFlowE2ETestSuitePG) compareQuery(schema1, schema2 string) error { + query := fmt.Sprintf("SELECT * FROM %s EXCEPT SELECT * FROM %s", schema1, schema2) + rows, _ := s.pool.Query(context.Background(), query) + + defer rows.Close() + for rows.Next() { + values, err := rows.Values() + if err != nil { + return err + } + + columns := rows.FieldDescriptions() + + for i, value := range values { + fmt.Printf("%s: %v\n", columns[i].Name, value) + } + fmt.Println("---") + } + + return rows.Err() +} + +func (s *PeerFlowE2ETestSuitePG) Test_Complete_QRep_Flow_Multi_Insert_PG() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + numRows := 10 + + srcTable := "test_qrep_flow_avro_pg_1" + s.setupSourceTable(srcTable, numRows) + + dstTable := "test_qrep_flow_avro_pg_2" + e2e.CreateSourceTableQRep(s.pool, postgresSuffix, dstTable) // the name is misleading, but this is the destination table + + srcSchemaQualified := fmt.Sprintf("%s_%s.%s", "e2e_test", postgresSuffix, srcTable) + dstSchemaQualified := fmt.Sprintf("%s_%s.%s", "e2e_test", postgresSuffix, dstTable) + + query := fmt.Sprintf("SELECT * FROM e2e_test_%s.%s WHERE updated_at BETWEEN {{.start}} AND {{.end}}", + postgresSuffix, srcTable) + + postgresPeer := e2e.GeneratePostgresPeer(e2e.PostgresPort) + + qrepConfig, err := e2e.CreateQRepWorkflowConfig( + "test_qrep_flow_avro_pg", + srcSchemaQualified, + dstSchemaQualified, + query, + protos.QRepSyncMode_QREP_SYNC_MODE_MULTI_INSERT, + postgresPeer, + "", + ) + s.NoError(err) + + e2e.RunQrepFlowWorkflow(env, qrepConfig) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + + // assert that error contains "invalid connection configs" + err = env.GetWorkflowError() + s.NoError(err) + + err = s.comparePGTables(srcSchemaQualified, dstSchemaQualified) + if err != nil { + s.FailNow(err.Error()) + } + + env.AssertExpectations(s.T()) +} diff --git a/flow/e2e/qrep_flow_test.go b/flow/e2e/qrep_flow_test.go deleted file mode 100644 index c79157bcfd..0000000000 --- a/flow/e2e/qrep_flow_test.go +++ /dev/null @@ -1,625 +0,0 @@ -package e2e - -import ( - "context" - "encoding/json" - "fmt" - "strings" - - connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/model" - "github.com/PeerDB-io/peer-flow/model/qvalue" - peerflow "github.com/PeerDB-io/peer-flow/workflows" - "github.com/google/uuid" - "github.com/stretchr/testify/require" - "go.temporal.io/sdk/testsuite" -) - -func (s *E2EPeerFlowTestSuite) createSourceTable(tableName string) { - tblFields := []string{ - "id UUID NOT NULL PRIMARY KEY", - "card_id UUID", - `"from" TIMESTAMP NOT NULL`, - "price NUMERIC", - "created_at TIMESTAMP NOT NULL", - "updated_at TIMESTAMP NOT NULL", - "transaction_hash BYTEA", - "ownerable_type VARCHAR", - "ownerable_id UUID", - "user_nonce INTEGER", - "transfer_type INTEGER DEFAULT 0 NOT NULL", - "blockchain INTEGER NOT NULL", - "deal_type VARCHAR", - "deal_id UUID", - "ethereum_transaction_id UUID", - "ignore_price BOOLEAN DEFAULT false", - "card_eth_value DOUBLE PRECISION", - "paid_eth_price DOUBLE PRECISION", - "card_bought_notified BOOLEAN DEFAULT false NOT NULL", - "address NUMERIC", - "account_id UUID", - "asset_id NUMERIC NOT NULL", - "status INTEGER", - "transaction_id UUID", - "settled_at TIMESTAMP", - "reference_id VARCHAR", - "settle_at TIMESTAMP", - "settlement_delay_reason INTEGER", - "f1 text[]", - "f2 bigint[]", - "f3 int[]", - "f4 varchar[]", - "f5 jsonb", - "f6 jsonb", - "f7 jsonb", - "f8 smallint", - } - - tblFieldStr := strings.Join(tblFields, ",") - - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` - CREATE TABLE e2e_test.%s ( - %s - );`, tableName, tblFieldStr)) - s.NoError(err) - - fmt.Printf("created table on postgres: e2e_test.%s\n", tableName) -} - -func (s *E2EPeerFlowTestSuite) populateSourceTable(tableName string, rowCount int) { - var ids []string - var rows []string - for i := 0; i < rowCount-1; i++ { - id := uuid.New().String() - ids = append(ids, id) - row := fmt.Sprintf(` - ( - '%s', '%s', CURRENT_TIMESTAMP, 3.86487206688919, CURRENT_TIMESTAMP, - CURRENT_TIMESTAMP, E'\\\\xDEADBEEF', 'type1', '%s', - 1, 0, 1, 'dealType1', - '%s', '%s', false, 1.2345, - 1.2345, false, 12345, '%s', - 12345, 1, '%s', CURRENT_TIMESTAMP, 'refID', - CURRENT_TIMESTAMP, 1, ARRAY['text1', 'text2'], ARRAY[123, 456], ARRAY[789, 012], - ARRAY['varchar1', 'varchar2'], '{"key": 8.5}', - '[{"key1": "value1", "key2": "value2", "key3": "value3"}]', - '{"key": "value"}', 15 - )`, - id, uuid.New().String(), uuid.New().String(), - uuid.New().String(), uuid.New().String(), uuid.New().String(), uuid.New().String()) - rows = append(rows, row) - } - - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` - INSERT INTO e2e_test.%s ( - id, card_id, "from", price, created_at, - updated_at, transaction_hash, ownerable_type, ownerable_id, - user_nonce, transfer_type, blockchain, deal_type, - deal_id, ethereum_transaction_id, ignore_price, card_eth_value, - paid_eth_price, card_bought_notified, address, account_id, - asset_id, status, transaction_id, settled_at, reference_id, - settle_at, settlement_delay_reason, f1, f2, f3, f4, f5, f6, f7, f8 - ) VALUES %s; - `, tableName, strings.Join(rows, ","))) - require.NoError(s.T(), err) - - // add a row where all the nullable fields are null - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` - INSERT INTO e2e_test.%s ( - id, "from", created_at, updated_at, - transfer_type, blockchain, card_bought_notified, asset_id - ) VALUES ( - '%s', CURRENT_TIMESTAMP, CURRENT_TIMESTAMP, CURRENT_TIMESTAMP, - 0, 1, false, 12345 - ); - `, tableName, uuid.New().String())) - require.NoError(s.T(), err) - - // generate a 20 MB json and update id[0]'s col f5 to it - v := s.generate20MBJson() - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` - UPDATE e2e_test.%s SET f5 = '%s' WHERE id = '%s'; - `, tableName, v, ids[0])) - require.NoError(s.T(), err) -} - -func (s *E2EPeerFlowTestSuite) generate20MBJson() []byte { - xn := make(map[string]interface{}) - for i := 0; i < 215000; i++ { - xn[uuid.New().String()] = uuid.New().String() - } - - v, err := json.Marshal(xn) - require.NoError(s.T(), err) - - return v -} - -func (s *E2EPeerFlowTestSuite) setupSourceTable(tableName string, rowCount int) { - s.createSourceTable(tableName) - s.populateSourceTable(tableName, rowCount) -} - -func getOwnersSchema() *model.QRecordSchema { - return &model.QRecordSchema{ - Fields: []*model.QField{ - {Name: "id", Type: qvalue.QValueKindString, Nullable: true}, - {Name: "card_id", Type: qvalue.QValueKindString, Nullable: true}, - {Name: "from", Type: qvalue.QValueKindTimestamp, Nullable: true}, - {Name: "price", Type: qvalue.QValueKindNumeric, Nullable: true}, - {Name: "created_at", Type: qvalue.QValueKindTimestamp, Nullable: true}, - {Name: "updated_at", Type: qvalue.QValueKindTimestamp, Nullable: true}, - {Name: "transaction_hash", Type: qvalue.QValueKindBytes, Nullable: true}, - {Name: "ownerable_type", Type: qvalue.QValueKindString, Nullable: true}, - {Name: "ownerable_id", Type: qvalue.QValueKindString, Nullable: true}, - {Name: "user_nonce", Type: qvalue.QValueKindInt64, Nullable: true}, - {Name: "transfer_type", Type: qvalue.QValueKindInt64, Nullable: true}, - {Name: "blockchain", Type: qvalue.QValueKindInt64, Nullable: true}, - {Name: "deal_type", Type: qvalue.QValueKindString, Nullable: true}, - {Name: "deal_id", Type: qvalue.QValueKindString, Nullable: true}, - {Name: "ethereum_transaction_id", Type: qvalue.QValueKindString, Nullable: true}, - {Name: "ignore_price", Type: qvalue.QValueKindBoolean, Nullable: true}, - {Name: "card_eth_value", Type: qvalue.QValueKindFloat64, Nullable: true}, - {Name: "paid_eth_price", Type: qvalue.QValueKindFloat64, Nullable: true}, - {Name: "card_bought_notified", Type: qvalue.QValueKindBoolean, Nullable: true}, - {Name: "address", Type: qvalue.QValueKindNumeric, Nullable: true}, - {Name: "account_id", Type: qvalue.QValueKindString, Nullable: true}, - {Name: "asset_id", Type: qvalue.QValueKindNumeric, Nullable: true}, - {Name: "status", Type: qvalue.QValueKindInt64, Nullable: true}, - {Name: "transaction_id", Type: qvalue.QValueKindString, Nullable: true}, - {Name: "settled_at", Type: qvalue.QValueKindTimestamp, Nullable: true}, - {Name: "reference_id", Type: qvalue.QValueKindString, Nullable: true}, - {Name: "settle_at", Type: qvalue.QValueKindTimestamp, Nullable: true}, - {Name: "settlement_delay_reason", Type: qvalue.QValueKindInt64, Nullable: true}, - {Name: "f1", Type: qvalue.QValueKindArrayString, Nullable: true}, - {Name: "f2", Type: qvalue.QValueKindArrayInt64, Nullable: true}, - {Name: "f3", Type: qvalue.QValueKindArrayInt32, Nullable: true}, - {Name: "f4", Type: qvalue.QValueKindArrayString, Nullable: true}, - {Name: "f5", Type: qvalue.QValueKindJSON, Nullable: true}, - {Name: "f6", Type: qvalue.QValueKindJSON, Nullable: true}, - {Name: "f7", Type: qvalue.QValueKindJSON, Nullable: true}, - {Name: "f8", Type: qvalue.QValueKindInt16, Nullable: true}, - }, - } -} - -func getOwnersSelectorString() string { - schema := getOwnersSchema() - var fields []string - for _, field := range schema.Fields { - // append quoted field name - fields = append(fields, fmt.Sprintf(`"%s"`, field.Name)) - } - return strings.Join(fields, ",") -} - -func (s *E2EPeerFlowTestSuite) setupBQDestinationTable(dstTable string) { - schema := getOwnersSchema() - err := s.bqHelper.CreateTable(dstTable, schema) - - // fail if table creation fails - require.NoError(s.T(), err) - - fmt.Printf("created table on bigquery: %s.%s. %v\n", s.bqHelper.Config.DatasetId, dstTable, err) -} - -func (s *E2EPeerFlowTestSuite) setupSFDestinationTable(dstTable string) { - schema := getOwnersSchema() - err := s.sfHelper.CreateTable(dstTable, schema) - - // fail if table creation fails - if err != nil { - s.FailNow("unable to create table on snowflake", err) - } - - fmt.Printf("created table on snowflake: %s.%s. %v\n", s.sfHelper.testSchemaName, dstTable, err) -} - -func (s *E2EPeerFlowTestSuite) createQRepWorkflowConfig( - flowJobName string, - sourceTable string, - dstTable string, - query string, - syncMode protos.QRepSyncMode, - dest *protos.Peer, - stagingPath string, -) *protos.QRepConfig { - connectionGen := QRepFlowConnectionGenerationConfig{ - FlowJobName: flowJobName, - WatermarkTable: sourceTable, - DestinationTableIdentifier: dstTable, - PostgresPort: postgresPort, - Destination: dest, - StagingPath: stagingPath, - } - - watermark := "updated_at" - - qrepConfig, err := connectionGen.GenerateQRepConfig(query, watermark, syncMode) - s.NoError(err) - - qrepConfig.InitialCopyOnly = true - - return qrepConfig -} - -func (s *E2EPeerFlowTestSuite) compareTableContentsBQ(tableName string, colsString string) { - // read rows from source table - pgQueryExecutor := connpostgres.NewQRepQueryExecutor(s.pool, context.Background(), "testflow", "testpart") - pgQueryExecutor.SetTestEnv(true) - - pgRows, err := pgQueryExecutor.ExecuteAndProcessQuery( - fmt.Sprintf("SELECT %s FROM e2e_test.%s ORDER BY id", colsString, tableName), - ) - s.NoError(err) - - // read rows from destination table - qualifiedTableName := fmt.Sprintf("`%s.%s`", s.bqHelper.Config.DatasetId, tableName) - bqRows, err := s.bqHelper.ExecuteAndProcessQuery( - fmt.Sprintf("SELECT %s FROM %s ORDER BY id", colsString, qualifiedTableName), - ) - s.NoError(err) - - s.True(pgRows.Equals(bqRows), "rows from source and destination tables are not equal") -} - -func (s *E2EPeerFlowTestSuite) compareTableContentsSF(tableName string, selector string, caseSensitive bool) { - // read rows from source table - pgQueryExecutor := connpostgres.NewQRepQueryExecutor(s.pool, context.Background(), "testflow", "testpart") - pgQueryExecutor.SetTestEnv(true) - pgRows, err := pgQueryExecutor.ExecuteAndProcessQuery( - fmt.Sprintf("SELECT %s FROM e2e_test.%s ORDER BY id", selector, tableName), - ) - require.NoError(s.T(), err) - - // read rows from destination table - qualifiedTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, tableName) - var sfSelQuery string - if caseSensitive { - sfSelQuery = fmt.Sprintf(`SELECT %s FROM %s ORDER BY "id"`, selector, qualifiedTableName) - } else { - sfSelQuery = fmt.Sprintf(`SELECT %s FROM %s ORDER BY id`, selector, qualifiedTableName) - } - fmt.Printf("running query on snowflake: %s\n", sfSelQuery) - - // sleep for 1 min for debugging - // time.Sleep(1 * time.Minute) - - sfRows, err := s.sfHelper.ExecuteAndProcessQuery(sfSelQuery) - require.NoError(s.T(), err) - - s.True(pgRows.Equals(sfRows), "rows from source and destination tables are not equal") -} - -func (s *E2EPeerFlowTestSuite) comparePGTables(srcSchemaQualified, dstSchemaQualified string) error { - // Execute the two EXCEPT queries - err := s.compareQuery(srcSchemaQualified, dstSchemaQualified) - if err != nil { - return err - } - - err = s.compareQuery(dstSchemaQualified, srcSchemaQualified) - if err != nil { - return err - } - - // If no error is returned, then the contents of the two tables are the same - return nil -} - -func (s *E2EPeerFlowTestSuite) compareQuery(schema1, schema2 string) error { - query := fmt.Sprintf("SELECT * FROM %s EXCEPT SELECT * FROM %s", schema1, schema2) - rows, _ := s.pool.Query(context.Background(), query) - - defer rows.Close() - for rows.Next() { - values, err := rows.Values() - if err != nil { - return err - } - - columns := rows.FieldDescriptions() - - for i, value := range values { - fmt.Printf("%s: %v\n", columns[i].Name, value) - } - fmt.Println("---") - } - - return rows.Err() -} - -// NOTE: Disabled due to large JSON tests being added: https://github.com/PeerDB-io/peerdb/issues/309 - -// Test_Complete_QRep_Flow tests a complete flow with data in the source table. -// The test inserts 10 rows into the source table and verifies that the data is -// // correctly synced to the destination table this runs a QRep Flow. -// func (s *E2EPeerFlowTestSuite) Test_Complete_QRep_Flow_Multi_Insert() { -// env := s.NewTestWorkflowEnvironment() -// registerWorkflowsAndActivities(env) - -// numRows := 10 - -// tblName := "test_qrep_flow_multi_insert" -// s.setupSourceTable(tblName, numRows) -// s.setupBQDestinationTable(tblName) - -// query := fmt.Sprintf("SELECT * FROM e2e_test.%s WHERE updated_at BETWEEN {{.start}} AND {{.end}}", tblName) - -// qrepConfig := s.createQRepWorkflowConfig("test_qrep_flow_mi", -// "e2e_test."+tblName, -// tblName, -// query, -// protos.QRepSyncMode_QREP_SYNC_MODE_MULTI_INSERT, -// s.bqHelper.Peer) -// runQrepFlowWorkflow(env, qrepConfig) - -// // Verify workflow completes without error -// s.True(env.IsWorkflowCompleted()) - -// // assert that error contains "invalid connection configs" -// err := env.GetWorkflowError() -// s.NoError(err) - -// count, err := s.bqHelper.CountRows(tblName) -// s.NoError(err) - -// s.Equal(numRows, count) - -// env.AssertExpectations(s.T()) -// } - -func (s *E2EPeerFlowTestSuite) Test_Complete_QRep_Flow_Avro() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - numRows := 10 - - tblName := "test_qrep_flow_avro" - s.setupSourceTable(tblName, numRows) - s.setupBQDestinationTable(tblName) - - query := fmt.Sprintf("SELECT * FROM e2e_test.%s WHERE updated_at BETWEEN {{.start}} AND {{.end}}", tblName) - - qrepConfig := s.createQRepWorkflowConfig( - "test_qrep_flow_avro", - "e2e_test."+tblName, - tblName, - query, - protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, - s.bqHelper.Peer, - "peerdb_staging") - runQrepFlowWorkflow(env, qrepConfig) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - - // assert that error contains "invalid connection configs" - err := env.GetWorkflowError() - s.NoError(err) - - s.compareTableContentsBQ(tblName, "*") - - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Complete_QRep_Flow_Avro_SF() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - numRows := 10 - - tblName := "test_qrep_flow_avro_sf" - s.setupSourceTable(tblName, numRows) - s.setupSFDestinationTable(tblName) - - dstSchemaQualified := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, tblName) - - query := fmt.Sprintf("SELECT * FROM e2e_test.%s WHERE updated_at BETWEEN {{.start}} AND {{.end}}", tblName) - - qrepConfig := s.createQRepWorkflowConfig( - "test_qrep_flow_avro_Sf", - "e2e_test."+tblName, - dstSchemaQualified, - query, - protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, - s.sfHelper.Peer, - "", - ) - - runQrepFlowWorkflow(env, qrepConfig) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - - // assert that error contains "invalid connection configs" - err := env.GetWorkflowError() - s.NoError(err) - - sel := getOwnersSelectorString() - s.compareTableContentsSF(tblName, sel, true) - - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Complete_QRep_Flow_Avro_SF_Upsert_Simple() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - numRows := 10 - - tblName := "test_qrep_flow_avro_sf_ups" - s.setupSourceTable(tblName, numRows) - s.setupSFDestinationTable(tblName) - - dstSchemaQualified := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, tblName) - - query := fmt.Sprintf("SELECT * FROM e2e_test.%s WHERE updated_at >= {{.start}} AND updated_at < {{.end}}", tblName) - - qrepConfig := s.createQRepWorkflowConfig( - "test_qrep_flow_avro_Sf", - "e2e_test."+tblName, - dstSchemaQualified, - query, - protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, - s.sfHelper.Peer, - "", - ) - qrepConfig.WriteMode = &protos.QRepWriteMode{ - WriteType: protos.QRepWriteType_QREP_WRITE_MODE_UPSERT, - UpsertKeyColumns: []string{"id"}, - } - - runQrepFlowWorkflow(env, qrepConfig) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - - // assert that error contains "invalid connection configs" - err := env.GetWorkflowError() - s.NoError(err) - - sel := getOwnersSelectorString() - s.compareTableContentsSF(tblName, sel, true) - - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Complete_QRep_Flow_Multi_Insert_PG() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - numRows := 10 - - srcTable := "test_qrep_flow_avro_pg_1" - s.setupSourceTable(srcTable, numRows) - - dstTable := "test_qrep_flow_avro_pg_2" - s.createSourceTable(dstTable) // the name is misleading, but this is the destination table - - srcSchemaQualified := fmt.Sprintf("%s.%s", "e2e_test", srcTable) - dstSchemaQualified := fmt.Sprintf("%s.%s", "e2e_test", dstTable) - - query := fmt.Sprintf("SELECT * FROM e2e_test.%s WHERE updated_at BETWEEN {{.start}} AND {{.end}}", srcTable) - - postgresPeer := GeneratePostgresPeer(postgresPort) - - qrepConfig := s.createQRepWorkflowConfig( - "test_qrep_flow_avro_pg", - srcSchemaQualified, - dstSchemaQualified, - query, - protos.QRepSyncMode_QREP_SYNC_MODE_MULTI_INSERT, - postgresPeer, - "", - ) - - runQrepFlowWorkflow(env, qrepConfig) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - - // assert that error contains "invalid connection configs" - err := env.GetWorkflowError() - s.NoError(err) - - err = s.comparePGTables(srcSchemaQualified, dstSchemaQualified) - if err != nil { - s.FailNow(err.Error()) - } - - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Complete_QRep_Flow_Avro_SF_S3() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - numRows := 10 - - tblName := "test_qrep_flow_avro_sf_s3" - s.setupSourceTable(tblName, numRows) - s.setupSFDestinationTable(tblName) - - dstSchemaQualified := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, tblName) - - query := fmt.Sprintf("SELECT * FROM e2e_test.%s WHERE updated_at >= {{.start}} AND updated_at < {{.end}}", tblName) - - qrepConfig := s.createQRepWorkflowConfig( - "test_qrep_flow_avro_sf", - "e2e_test."+tblName, - dstSchemaQualified, - query, - protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, - s.sfHelper.Peer, - "", - ) - qrepConfig.StagingPath = fmt.Sprintf("s3://peerdb-test-bucket/avro/%s", uuid.New()) - - runQrepFlowWorkflow(env, qrepConfig) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - - // assert that error contains "invalid connection configs" - err := env.GetWorkflowError() - s.NoError(err) - - sel := getOwnersSelectorString() - s.compareTableContentsSF(tblName, sel, true) - - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Complete_QRep_Flow_Avro_SF_S3_Integration() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - numRows := 10 - - tblName := "test_qrep_flow_avro_sf_s3_int" - s.setupSourceTable(tblName, numRows) - s.setupSFDestinationTable(tblName) - - dstSchemaQualified := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, tblName) - - query := fmt.Sprintf("SELECT * FROM e2e_test.%s WHERE updated_at >= {{.start}} AND updated_at < {{.end}}", tblName) - - sfPeer := s.sfHelper.Peer - sfPeer.GetSnowflakeConfig().S3Integration = "peerdb_s3_integration" - - qrepConfig := s.createQRepWorkflowConfig( - "test_qrep_flow_avro_sf_int", - "e2e_test."+tblName, - dstSchemaQualified, - query, - protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, - sfPeer, - "", - ) - qrepConfig.StagingPath = fmt.Sprintf("s3://peerdb-test-bucket/avro/%s", uuid.New()) - - runQrepFlowWorkflow(env, qrepConfig) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - - // assert that error contains "invalid connection configs" - err := env.GetWorkflowError() - s.NoError(err) - - sel := getOwnersSelectorString() - s.compareTableContentsSF(tblName, sel, true) - - env.AssertExpectations(s.T()) -} - -func runQrepFlowWorkflow(env *testsuite.TestWorkflowEnvironment, config *protos.QRepConfig) { - lastPartition := &protos.QRepPartition{ - PartitionId: "not-applicable-partition", - Range: nil, - } - numPartitionsProcessed := 0 - env.ExecuteWorkflow(peerflow.QRepFlowWorkflow, config, lastPartition, numPartitionsProcessed) -} diff --git a/flow/e2e/s3/qrep_flow_s3_test.go b/flow/e2e/s3/qrep_flow_s3_test.go new file mode 100644 index 0000000000..e47d235922 --- /dev/null +++ b/flow/e2e/s3/qrep_flow_s3_test.go @@ -0,0 +1,183 @@ +package e2e_s3 + +import ( + "context" + "fmt" + "testing" + "time" + + "github.com/PeerDB-io/peer-flow/e2e" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/jackc/pgx/v5/pgxpool" + "github.com/joho/godotenv" + log "github.com/sirupsen/logrus" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + "go.temporal.io/sdk/testsuite" +) + +const s3Suffix = "s3" + +type PeerFlowE2ETestSuiteS3 struct { + suite.Suite + testsuite.WorkflowTestSuite + + pool *pgxpool.Pool + s3Helper *S3TestHelper +} + +func TestPeerFlowE2ETestSuiteS3(t *testing.T) { + suite.Run(t, new(PeerFlowE2ETestSuiteS3)) +} + +func (s *PeerFlowE2ETestSuiteS3) setupSourceTable(tableName string, rowCount int) { + err := e2e.CreateSourceTableQRep(s.pool, s3Suffix, tableName) + s.NoError(err) + err = e2e.PopulateSourceTable(s.pool, s3Suffix, tableName, rowCount) + s.NoError(err) +} + +func (s *PeerFlowE2ETestSuiteS3) setupS3() error { + helper, err := NewS3TestHelper() + if err != nil { + return err + } + + s.s3Helper = helper + return nil +} + +func (s *PeerFlowE2ETestSuiteS3) SetupSuite() { + err := godotenv.Load() + if err != nil { + // it's okay if the .env file is not present + // we will use the default values + log.Infof("Unable to load .env file, using default values from env") + } + + log.SetReportCaller(true) + + pool, err := e2e.SetupPostgres(s3Suffix) + if err != nil { + s.Fail("failed to setup postgres", err) + } + s.pool = pool + + err = s.setupS3() + if err != nil { + s.Fail("failed to setup S3", err) + } +} + +// Implement TearDownAllSuite interface to tear down the test suite +func (s *PeerFlowE2ETestSuiteS3) TearDownSuite() { + err := e2e.TearDownPostgres(s.pool, s3Suffix) + if err != nil { + s.Fail("failed to drop Postgres schema", err) + } + + if s.s3Helper != nil { + err = s.s3Helper.CleanUp() + if err != nil { + s.Fail("failed to clean up s3", err) + } + } +} + +func (s *PeerFlowE2ETestSuiteS3) Test_Complete_QRep_Flow_S3() { + if s.s3Helper == nil { + s.T().Skip("Skipping S3 test") + } + + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + jobName := "test_complete_flow_s3" + schemaQualifiedName := fmt.Sprintf("e2e_test_%s.%s", s3Suffix, jobName) + + s.setupSourceTable(jobName, 10) + query := fmt.Sprintf("SELECT * FROM %s WHERE updated_at >= {{.start}} AND updated_at < {{.end}}", + schemaQualifiedName) + qrepConfig, err := e2e.CreateQRepWorkflowConfig( + jobName, + schemaQualifiedName, + "e2e_dest_1", + query, + protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, + s.s3Helper.GetPeer(), + "stage", + ) + s.NoError(err) + qrepConfig.StagingPath = s.s3Helper.s3Config.Url + + e2e.RunQrepFlowWorkflow(env, qrepConfig) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + s.NoError(err) + + // Verify destination has 1 file + // make context with timeout + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + defer cancel() + + files, err := s.s3Helper.ListAllFiles(ctx, jobName) + + require.NoError(s.T(), err) + + require.Equal(s.T(), 1, len(files)) + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteS3) Test_Complete_QRep_Flow_S3_CTID() { + if s.s3Helper == nil { + s.T().Skip("Skipping S3 test") + } + + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + jobName := "test_complete_flow_s3_ctid" + schemaQualifiedName := fmt.Sprintf("e2e_test_%s.%s", s3Suffix, jobName) + + s.setupSourceTable(jobName, 20000) + query := fmt.Sprintf("SELECT * FROM %s WHERE ctid BETWEEN {{.start}} AND {{.end}}", schemaQualifiedName) + qrepConfig, err := e2e.CreateQRepWorkflowConfig( + jobName, + schemaQualifiedName, + "e2e_dest_ctid", + query, + protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, + s.s3Helper.GetPeer(), + "stage", + ) + s.NoError(err) + qrepConfig.StagingPath = s.s3Helper.s3Config.Url + qrepConfig.NumRowsPerPartition = 2000 + qrepConfig.InitialCopyOnly = true + qrepConfig.WatermarkColumn = "ctid" + + e2e.RunQrepFlowWorkflow(env, qrepConfig) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + s.NoError(err) + + // Verify destination has 1 file + // make context with timeout + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + defer cancel() + + files, err := s.s3Helper.ListAllFiles(ctx, jobName) + + require.NoError(s.T(), err) + + require.Equal(s.T(), 10, len(files)) + + env.AssertExpectations(s.T()) +} diff --git a/flow/e2e/s3_helper.go b/flow/e2e/s3/s3_helper.go similarity index 99% rename from flow/e2e/s3_helper.go rename to flow/e2e/s3/s3_helper.go index 49307e6242..9dbe958eef 100644 --- a/flow/e2e/s3_helper.go +++ b/flow/e2e/s3/s3_helper.go @@ -1,4 +1,4 @@ -package e2e +package e2e_s3 import ( "context" diff --git a/flow/e2e/snowflake/peer_flow_sf_test.go b/flow/e2e/snowflake/peer_flow_sf_test.go new file mode 100644 index 0000000000..db55046961 --- /dev/null +++ b/flow/e2e/snowflake/peer_flow_sf_test.go @@ -0,0 +1,800 @@ +package e2e_snowflake + +import ( + "context" + "fmt" + "testing" + + "github.com/PeerDB-io/peer-flow/e2e" + "github.com/PeerDB-io/peer-flow/generated/protos" + peerflow "github.com/PeerDB-io/peer-flow/workflows" + "github.com/jackc/pgx/v5/pgxpool" + "github.com/joho/godotenv" + log "github.com/sirupsen/logrus" + "github.com/stretchr/testify/suite" + "go.temporal.io/sdk/testsuite" +) + +const snowflakeSuffix = "snowflake" + +type PeerFlowE2ETestSuiteSF struct { + suite.Suite + testsuite.WorkflowTestSuite + + pool *pgxpool.Pool + sfHelper *SnowflakeTestHelper +} + +func TestPeerFlowE2ETestSuiteSF(t *testing.T) { + suite.Run(t, new(PeerFlowE2ETestSuiteSF)) +} + +func (s *PeerFlowE2ETestSuiteSF) attachSchemaSuffix(tableName string) string { + return fmt.Sprintf("e2e_test_%s.%s", snowflakeSuffix, tableName) +} + +func (s *PeerFlowE2ETestSuiteSF) attachSuffix(input string) string { + return fmt.Sprintf("%s_%s", input, snowflakeSuffix) +} + +// setupSnowflake sets up the snowflake connection. +func (s *PeerFlowE2ETestSuiteSF) setupSnowflake() error { + sfHelper, err := NewSnowflakeTestHelper() + if err != nil { + return fmt.Errorf("failed to create snowflake helper: %w", err) + } + + s.sfHelper = sfHelper + + return nil +} + +func (s *PeerFlowE2ETestSuiteSF) SetupSuite() { + err := godotenv.Load() + if err != nil { + // it's okay if the .env file is not present + // we will use the default values + log.Infof("Unable to load .env file, using default values from env") + } + + log.SetReportCaller(true) + + pool, err := e2e.SetupPostgres(snowflakeSuffix) + if err != nil { + s.Fail("failed to setup postgres", err) + } + s.pool = pool + + err = s.setupSnowflake() + if err != nil { + s.Fail("failed to setup snowflake", err) + } +} + +// Implement TearDownAllSuite interface to tear down the test suite +func (s *PeerFlowE2ETestSuiteSF) TearDownSuite() { + err := e2e.TearDownPostgres(s.pool, snowflakeSuffix) + if err != nil { + s.Fail("failed to drop Postgres schema", err) + } + + if s.sfHelper != nil { + err = s.sfHelper.Cleanup() + if err != nil { + s.Fail("failed to clean up Snowflake", err) + } + } +} + +func (s *PeerFlowE2ETestSuiteSF) Test_Complete_Simple_Flow_SF() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_simple_flow_sf") + dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_simple_flow_sf") + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE %s ( + id SERIAL PRIMARY KEY, + key TEXT NOT NULL, + value TEXT NOT NULL + ); + `, srcTableName)) + s.NoError(err) + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_simple_flow"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.sfHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 2, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and then insert 10 rows into the source table + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + // insert 10 rows into the source table + for i := 0; i < 10; i++ { + testKey := fmt.Sprintf("test_key_%d", i) + testValue := fmt.Sprintf("test_value_%d", i) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s (key, value) VALUES ($1, $2) + `, srcTableName), testKey, testValue) + s.NoError(err) + } + fmt.Println("Inserted 10 rows into the source table") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + count, err := s.sfHelper.CountRows("test_simple_flow_sf") + s.NoError(err) + s.Equal(10, count) + + // TODO: verify that the data is correctly synced to the destination table + // on the bigquery side + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteSF) Test_Complete_Simple_Flow_SF_Avro_CDC() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_simple_flow_sf_avro_cdc") + dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_simple_flow_sf_avro_cdc") + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE %s ( + id SERIAL PRIMARY KEY, + key TEXT NOT NULL, + value TEXT NOT NULL + ); + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_simple_flow_avro"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.sfHelper.Peer, + CDCSyncMode: protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 2, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and then insert 10 rows into the source table + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + // insert 10 rows into the source table + for i := 0; i < 10; i++ { + testKey := fmt.Sprintf("test_key_%d", i) + testValue := fmt.Sprintf("test_value_%d", i) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s (key, value) VALUES ($1, $2) + `, srcTableName), testKey, testValue) + s.NoError(err) + } + fmt.Println("Inserted 10 rows into the source table") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + count, err := s.sfHelper.CountRows("test_simple_flow_sf_avro_cdc") + s.NoError(err) + s.Equal(10, count) + + // TODO: verify that the data is correctly synced to the destination table + // on the bigquery side + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteSF) Test_Toast_SF() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_toast_sf_1") + dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_1") + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE %s ( + id SERIAL PRIMARY KEY, + t1 text, + t2 text, + k int + );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_toast_sf_1"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.sfHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 1, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and execute a transaction touching toast columns + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + /* + Executing a transaction which + 1. changes both toast column + 2. changes no toast column + 2. changes 1 toast column + */ + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + BEGIN; + INSERT INTO %s (t1,t2,k) SELECT random_string(9000),random_string(9000), + 1 FROM generate_series(1,2); + UPDATE %s SET k=102 WHERE id=1; + UPDATE %s SET t1='dummy' WHERE id=2; + END; + `, srcTableName, srcTableName, srcTableName)) + s.NoError(err) + fmt.Println("Executed a transaction touching toast columns") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + s.compareTableContentsSF("test_toast_sf_1", `id,t1,t2,k`, false) + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteSF) Test_Toast_Nochanges_SF() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_toast_sf_2") + dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_2") + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE %s ( + id SERIAL PRIMARY KEY, + t1 text, + t2 text, + k int + );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_toast_sf_2"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.sfHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 1, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and execute a transaction touching toast columns + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + /* transaction updating no rows */ + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + BEGIN; + UPDATE %s SET k=102 WHERE id=1; + UPDATE %s SET t1='dummy' WHERE id=2; + END; + `, srcTableName, srcTableName)) + s.NoError(err) + fmt.Println("Executed a transaction touching toast columns") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + s.compareTableContentsSF("test_toast_sf_2", `id,t1,t2,k`, false) + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteSF) Test_Toast_Advance_1_SF() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_toast_sf_3") + dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_3") + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE %s ( + id SERIAL PRIMARY KEY, + t1 text, + t2 text, + k int + );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_toast_sf_3"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.sfHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 2, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and execute a transaction touching toast columns + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + //complex transaction with random DMLs on a table with toast columns + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + BEGIN; + INSERT INTO %s (t1,t2,k) SELECT random_string(9000),random_string(9000), + 1 FROM generate_series(1,2); + UPDATE %s SET k=102 WHERE id=1; + UPDATE %s SET t1='dummy' WHERE id=2; + UPDATE %s SET t2='dummy' WHERE id=2; + DELETE FROM %s WHERE id=1; + INSERT INTO %s(t1,t2,k) SELECT random_string(9000),random_string(9000), + 1 FROM generate_series(1,2); + UPDATE %s SET k=1 WHERE id=1; + UPDATE %s SET t1='dummy1',t2='dummy2' WHERE id=1; + UPDATE %s SET t1='dummy3' WHERE id=3; + DELETE FROM %s WHERE id=2; + DELETE FROM %s WHERE id=3; + DELETE FROM %s WHERE id=2; + END; + `, srcTableName, srcTableName, srcTableName, srcTableName, srcTableName, srcTableName, + srcTableName, srcTableName, srcTableName, srcTableName, srcTableName, srcTableName)) + s.NoError(err) + fmt.Println("Executed a transaction touching toast columns") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + s.compareTableContentsSF("test_toast_sf_3", `id,t1,t2,k`, false) + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteSF) Test_Toast_Advance_2_SF() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_toast_sf_4") + dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_4") + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE %s ( + id SERIAL PRIMARY KEY, + t1 text, + k int + );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_toast_sf_4"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.sfHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 1, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and execute a transaction touching toast columns + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + //complex transaction with random DMLs on a table with toast columns + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + BEGIN; + INSERT INTO %s (t1,k) SELECT random_string(9000), + 1 FROM generate_series(1,1); + UPDATE %s SET t1=sub.t1 FROM (SELECT random_string(9000) t1 + FROM generate_series(1,1) ) sub WHERE id=1; + UPDATE %s SET k=2 WHERE id=1; + UPDATE %s SET k=3 WHERE id=1; + UPDATE %s SET t1=sub.t1 FROM (SELECT random_string(9000) t1 + FROM generate_series(1,1)) sub WHERE id=1; + UPDATE %s SET k=4 WHERE id=1; + END; + `, srcTableName, srcTableName, srcTableName, srcTableName, srcTableName, srcTableName)) + s.NoError(err) + fmt.Println("Executed a transaction touching toast columns") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + s.compareTableContentsSF("test_toast_sf_4", `id,t1,k`, false) + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteSF) Test_Toast_Advance_3_SF() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_toast_sf_5") + dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_5") + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE %s ( + id SERIAL PRIMARY KEY, + t1 text, + t2 text, + k int + );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_toast_sf_5"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.sfHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 1, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and execute a transaction touching toast columns + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + /* + transaction updating a single row + multiple times with changed/unchanged toast columns + */ + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + BEGIN; + INSERT INTO %s (t1,t2,k) SELECT random_string(9000),random_string(9000), + 1 FROM generate_series(1,1); + UPDATE %s SET k=102 WHERE id=1; + UPDATE %s SET t1='dummy' WHERE id=1; + UPDATE %s SET t2='dummy' WHERE id=1; + END; + `, srcTableName, srcTableName, srcTableName, srcTableName)) + s.NoError(err) + fmt.Println("Executed a transaction touching toast columns") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + s.compareTableContentsSF("test_toast_sf_5", `id,t1,t2,k`, false) + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteSF) Test_Types_SF() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_types_sf") + dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_types_sf") + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE %s (id serial PRIMARY KEY,c1 BIGINT,c2 BIT,c3 VARBIT,c4 BOOLEAN, + c6 BYTEA,c7 CHARACTER,c8 varchar,c9 CIDR,c11 DATE,c12 FLOAT,c13 DOUBLE PRECISION, + c14 INET,c15 INTEGER,c16 INTERVAL,c17 JSON,c18 JSONB,c21 MACADDR,c22 MONEY, + c23 NUMERIC,c24 OID,c28 REAL,c29 SMALLINT,c30 SMALLSERIAL,c31 SERIAL,c32 TEXT, + c33 TIMESTAMP,c34 TIMESTAMPTZ,c35 TIME, c36 TIMETZ,c37 TSQUERY,c38 TSVECTOR, + c39 TXID_SNAPSHOT,c40 UUID,c41 XML); + CREATE OR REPLACE FUNCTION random_bytea(bytea_length integer) + RETURNS bytea AS $body$ + SELECT decode(string_agg(lpad(to_hex(width_bucket(random(), 0, 1, 256)-1),2,'0') ,''), 'hex') + FROM generate_series(1, $1); + $body$ + LANGUAGE 'sql' + VOLATILE + SET search_path = 'pg_catalog'; + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_types_sf"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.sfHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 1, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and execute a transaction touching toast columns + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + /* test inserting various types*/ + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s SELECT 2,2,b'1',b'101', + true,random_bytea(32),'s','test','1.1.10.2'::cidr, + CURRENT_DATE,1.23,1.234,'192.168.1.5'::inet,1, + '5 years 2 months 29 days 1 minute 2 seconds 200 milliseconds 20000 microseconds'::interval, + '{"sai":1}'::json,'{"sai":1}'::jsonb,'08:00:2b:01:02:03'::macaddr, + 1.2,1.23,4::oid,1.23,1,1,1,'test',now(),now(),now()::time,now()::timetz, + 'fat & rat'::tsquery,'a fat cat sat on a mat and ate a fat rat'::tsvector, + txid_current_snapshot(), + '66073c38-b8df-4bdb-bbca-1c97596b8940'::uuid,xmlcomment('hello'); + `, srcTableName)) + s.NoError(err) + fmt.Println("Executed an insert with all types") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + noNulls, err := s.sfHelper.CheckNull("test_types_sf", []string{"c41", "c1", "c2", "c3", "c4", + "c6", "c39", "c40", "id", "c9", "c11", "c12", "c13", "c14", "c15", "c16", "c17", "c18", + "c21", "c22", "c23", "c24", "c28", "c29", "c30", "c31", "c33", "c34", "c35", "c36", + "c37", "c38", "c7", "c8", "c32"}) + if err != nil { + fmt.Println("error %w", err) + } + // Make sure that there are no nulls + s.Equal(noNulls, true) + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteSF) Test_Types_SF_Avro_CDC() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_types_sf_avro_cdc") + dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_types_sf_avro_cdc") + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE %s (id serial PRIMARY KEY,c1 BIGINT,c2 BIT,c3 VARBIT,c4 BOOLEAN, + c6 BYTEA,c7 CHARACTER,c8 varchar,c9 CIDR,c11 DATE,c12 FLOAT,c13 DOUBLE PRECISION, + c14 INET,c15 INTEGER,c16 INTERVAL,c17 JSON,c18 JSONB,c21 MACADDR,c22 MONEY, + c23 NUMERIC,c24 OID,c28 REAL,c29 SMALLINT,c30 SMALLSERIAL,c31 SERIAL,c32 TEXT, + c33 TIMESTAMP,c34 TIMESTAMPTZ,c35 TIME, c36 TIMETZ,c37 TSQUERY,c38 TSVECTOR, + c39 TXID_SNAPSHOT,c40 UUID,c41 XML); + CREATE OR REPLACE FUNCTION random_bytea(bytea_length integer) + RETURNS bytea AS $body$ + SELECT decode(string_agg(lpad(to_hex(width_bucket(random(), 0, 1, 256)-1),2,'0') ,''), 'hex') + FROM generate_series(1, $1); + $body$ + LANGUAGE 'sql' + VOLATILE + SET search_path = 'pg_catalog'; + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_types_sf"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.sfHelper.Peer, + CDCSyncMode: protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 1, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and execute a transaction touching toast columns + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + /* test inserting various types*/ + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s SELECT 2,2,b'1',b'101', + true,random_bytea(32),'s','test','1.1.10.2'::cidr, + CURRENT_DATE,1.23,1.234,'192.168.1.5'::inet,1, + '5 years 2 months 29 days 1 minute 2 seconds 200 milliseconds 20000 microseconds'::interval, + '{"sai":1}'::json,'{"sai":1}'::jsonb,'08:00:2b:01:02:03'::macaddr, + 1.2,1.23,4::oid,1.23,1,1,1,'test',now(),now(),now()::time,now()::timetz, + 'fat & rat'::tsquery,'a fat cat sat on a mat and ate a fat rat'::tsvector, + txid_current_snapshot(), + '66073c38-b8df-4bdb-bbca-1c97596b8940'::uuid,xmlcomment('hello'); + `, srcTableName)) + s.NoError(err) + fmt.Println("Executed an insert with all types") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + noNulls, err := s.sfHelper.CheckNull("test_types_sf_avro_cdc", []string{"c41", "c1", "c2", "c3", "c4", + "c6", "c39", "c40", "id", "c9", "c11", "c12", "c13", "c14", "c15", "c16", "c17", "c18", + "c21", "c22", "c23", "c24", "c28", "c29", "c30", "c31", "c33", "c34", "c35", "c36", + "c37", "c38", "c7", "c8", "c32"}) + if err != nil { + fmt.Println("error %w", err) + } + // Make sure that there are no nulls + s.Equal(noNulls, true) + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteSF) Test_Multi_Table_SF() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTable1Name := s.attachSchemaSuffix("test1_sf") + srcTable2Name := s.attachSchemaSuffix("test2_sf") + dstTable1Name := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test1_sf") + dstTable2Name := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test2_sf") + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE %s (id serial primary key, c1 int, c2 text); + CREATE TABLE %s (id serial primary key, c1 int, c2 text); + `, srcTable1Name, srcTable2Name)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_multi_table"), + TableNameMapping: map[string]string{srcTable1Name: dstTable1Name, srcTable2Name: dstTable2Name}, + PostgresPort: e2e.PostgresPort, + Destination: s.sfHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 1, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and execute a transaction touching toast columns + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + /* inserting across multiple tables*/ + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s (c1,c2) VALUES (1,'dummy_1'); + INSERT INTO %s (c1,c2) VALUES (-1,'dummy_-1'); + `, srcTable1Name, srcTable2Name)) + s.NoError(err) + fmt.Println("Executed an insert with all types") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + count1, err := s.sfHelper.CountRows("test1_sf") + s.NoError(err) + count2, err := s.sfHelper.CountRows("test2_sf") + s.NoError(err) + + s.Equal(1, count1) + s.Equal(1, count2) + + env.AssertExpectations(s.T()) +} diff --git a/flow/e2e/snowflake/qrep_flow_sf_test.go b/flow/e2e/snowflake/qrep_flow_sf_test.go new file mode 100644 index 0000000000..4d42adbcc3 --- /dev/null +++ b/flow/e2e/snowflake/qrep_flow_sf_test.go @@ -0,0 +1,232 @@ +package e2e_snowflake + +import ( + "context" + "fmt" + + connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" + "github.com/PeerDB-io/peer-flow/e2e" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/google/uuid" + "github.com/stretchr/testify/require" +) + +func (s *PeerFlowE2ETestSuiteSF) setupSourceTable(tableName string, rowCount int) { + err := e2e.CreateSourceTableQRep(s.pool, snowflakeSuffix, tableName) + s.NoError(err) + err = e2e.PopulateSourceTable(s.pool, snowflakeSuffix, tableName, rowCount) + s.NoError(err) +} + +func (s *PeerFlowE2ETestSuiteSF) setupSFDestinationTable(dstTable string) { + schema := e2e.GetOwnersSchema() + err := s.sfHelper.CreateTable(dstTable, schema) + + // fail if table creation fails + if err != nil { + s.FailNow("unable to create table on snowflake", err) + } + + fmt.Printf("created table on snowflake: %s.%s. %v\n", s.sfHelper.testSchemaName, dstTable, err) +} + +func (s *PeerFlowE2ETestSuiteSF) compareTableContentsSF(tableName string, selector string, caseSensitive bool) { + // read rows from source table + pgQueryExecutor := connpostgres.NewQRepQueryExecutor(s.pool, context.Background(), "testflow", "testpart") + pgQueryExecutor.SetTestEnv(true) + pgRows, err := pgQueryExecutor.ExecuteAndProcessQuery( + fmt.Sprintf("SELECT %s FROM e2e_test_%s.%s ORDER BY id", selector, snowflakeSuffix, tableName), + ) + require.NoError(s.T(), err) + + // read rows from destination table + qualifiedTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, tableName) + var sfSelQuery string + if caseSensitive { + sfSelQuery = fmt.Sprintf(`SELECT %s FROM %s ORDER BY "id"`, selector, qualifiedTableName) + } else { + sfSelQuery = fmt.Sprintf(`SELECT %s FROM %s ORDER BY id`, selector, qualifiedTableName) + } + fmt.Printf("running query on snowflake: %s\n", sfSelQuery) + + // sleep for 1 min for debugging + // time.Sleep(1 * time.Minute) + + sfRows, err := s.sfHelper.ExecuteAndProcessQuery(sfSelQuery) + require.NoError(s.T(), err) + + s.True(pgRows.Equals(sfRows), "rows from source and destination tables are not equal") +} + +func (s *PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + numRows := 10 + + tblName := "test_qrep_flow_avro_sf" + s.setupSourceTable(tblName, numRows) + s.setupSFDestinationTable(tblName) + + dstSchemaQualified := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, tblName) + + query := fmt.Sprintf("SELECT * FROM e2e_test_%s.%s WHERE updated_at BETWEEN {{.start}} AND {{.end}}", + snowflakeSuffix, tblName) + + qrepConfig, err := e2e.CreateQRepWorkflowConfig( + "test_qrep_flow_avro_sf", + fmt.Sprintf("e2e_test_%s.%s", snowflakeSuffix, tblName), + dstSchemaQualified, + query, + protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, + s.sfHelper.Peer, + "", + ) + s.NoError(err) + + e2e.RunQrepFlowWorkflow(env, qrepConfig) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + + // assert that error contains "invalid connection configs" + err = env.GetWorkflowError() + s.NoError(err) + + sel := e2e.GetOwnersSelectorString() + s.compareTableContentsSF(tblName, sel, true) + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF_Upsert_Simple() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + numRows := 10 + + tblName := "test_qrep_flow_avro_sf_ups" + s.setupSourceTable(tblName, numRows) + s.setupSFDestinationTable(tblName) + + dstSchemaQualified := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, tblName) + + query := fmt.Sprintf("SELECT * FROM e2e_test_%s.%s WHERE updated_at >= {{.start}} AND updated_at < {{.end}}", + snowflakeSuffix, tblName) + + qrepConfig, err := e2e.CreateQRepWorkflowConfig( + "test_qrep_flow_avro_sf", + fmt.Sprintf("e2e_test_%s.%s", snowflakeSuffix, tblName), + dstSchemaQualified, + query, + protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, + s.sfHelper.Peer, + "", + ) + qrepConfig.WriteMode = &protos.QRepWriteMode{ + WriteType: protos.QRepWriteType_QREP_WRITE_MODE_UPSERT, + UpsertKeyColumns: []string{"id"}, + } + s.NoError(err) + + e2e.RunQrepFlowWorkflow(env, qrepConfig) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + + // assert that error contains "invalid connection configs" + err = env.GetWorkflowError() + s.NoError(err) + + sel := e2e.GetOwnersSelectorString() + s.compareTableContentsSF(tblName, sel, true) + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF_S3() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + numRows := 10 + + tblName := "test_qrep_flow_avro_sf_s3" + s.setupSourceTable(tblName, numRows) + s.setupSFDestinationTable(tblName) + + dstSchemaQualified := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, tblName) + + query := fmt.Sprintf("SELECT * FROM e2e_test_%s.%s WHERE updated_at >= {{.start}} AND updated_at < {{.end}}", + snowflakeSuffix, tblName) + + qrepConfig, err := e2e.CreateQRepWorkflowConfig( + "test_qrep_flow_avro_sf", + s.attachSchemaSuffix(tblName), + dstSchemaQualified, + query, + protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, + s.sfHelper.Peer, + "", + ) + s.NoError(err) + qrepConfig.StagingPath = fmt.Sprintf("s3://peerdb-test-bucket/avro/%s", uuid.New()) + + e2e.RunQrepFlowWorkflow(env, qrepConfig) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + + // assert that error contains "invalid connection configs" + err = env.GetWorkflowError() + s.NoError(err) + + sel := e2e.GetOwnersSelectorString() + s.compareTableContentsSF(tblName, sel, true) + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF_S3_Integration() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + numRows := 10 + + tblName := "test_qrep_flow_avro_sf_s3_int" + s.setupSourceTable(tblName, numRows) + s.setupSFDestinationTable(tblName) + + dstSchemaQualified := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, tblName) + + query := fmt.Sprintf("SELECT * FROM e2e_test_%s.%s WHERE updated_at >= {{.start}} AND updated_at < {{.end}}", + snowflakeSuffix, tblName) + + sfPeer := s.sfHelper.Peer + sfPeer.GetSnowflakeConfig().S3Integration = "peerdb_s3_integration" + + qrepConfig, err := e2e.CreateQRepWorkflowConfig( + "test_qrep_flow_avro_sf_int", + s.attachSchemaSuffix(tblName), + dstSchemaQualified, + query, + protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, + sfPeer, + "", + ) + s.NoError(err) + qrepConfig.StagingPath = fmt.Sprintf("s3://peerdb-test-bucket/avro/%s", uuid.New()) + + e2e.RunQrepFlowWorkflow(env, qrepConfig) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + + // assert that error contains "invalid connection configs" + err = env.GetWorkflowError() + s.NoError(err) + + sel := e2e.GetOwnersSelectorString() + s.compareTableContentsSF(tblName, sel, true) + + env.AssertExpectations(s.T()) +} diff --git a/flow/e2e/snowflake/snowflake_helper.go b/flow/e2e/snowflake/snowflake_helper.go new file mode 100644 index 0000000000..2a39daf93e --- /dev/null +++ b/flow/e2e/snowflake/snowflake_helper.go @@ -0,0 +1,131 @@ +package e2e_snowflake + +import ( + "context" + "encoding/json" + "fmt" + "os" + + connsnowflake "github.com/PeerDB-io/peer-flow/connectors/snowflake" + "github.com/PeerDB-io/peer-flow/e2e" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/model" + util "github.com/PeerDB-io/peer-flow/utils" +) + +type SnowflakeTestHelper struct { + // config is the Snowflake config. + Config *protos.SnowflakeConfig + // peer struct holder Snowflake + Peer *protos.Peer + // connection to another database, to manage the test database + adminClient *connsnowflake.SnowflakeClient + // connection to the test database + testClient *connsnowflake.SnowflakeClient + // testSchemaName is the schema to use for testing. + testSchemaName string + // dbName is the database used for testing. + testDatabaseName string +} + +func NewSnowflakeTestHelper() (*SnowflakeTestHelper, error) { + jsonPath := os.Getenv("TEST_SF_CREDS") + if jsonPath == "" { + return nil, fmt.Errorf("TEST_SF_CREDS env var not set") + } + + content, err := e2e.ReadFileToBytes(jsonPath) + if err != nil { + return nil, fmt.Errorf("failed to read file: %w", err) + } + + var config protos.SnowflakeConfig + err = json.Unmarshal(content, &config) + if err != nil { + return nil, fmt.Errorf("failed to unmarshal json: %w", err) + } + + peer := generateSFPeer(&config) + runID, err := util.RandomUInt64() + if err != nil { + return nil, fmt.Errorf("failed to generate random uint64: %w", err) + } + + testDatabaseName := fmt.Sprintf("e2e_test_%d", runID) + + adminClient, err := connsnowflake.NewSnowflakeClient(context.Background(), &config) + if err != nil { + return nil, fmt.Errorf("failed to create Snowflake client: %w", err) + } + err = adminClient.ExecuteQuery(fmt.Sprintf("CREATE DATABASE %s", testDatabaseName)) + if err != nil { + return nil, fmt.Errorf("failed to create Snowflake test database: %w", err) + } + + config.Database = testDatabaseName + testClient, err := connsnowflake.NewSnowflakeClient(context.Background(), &config) + if err != nil { + return nil, fmt.Errorf("failed to create Snowflake client: %w", err) + } + + return &SnowflakeTestHelper{ + Config: &config, + Peer: peer, + adminClient: adminClient, + testClient: testClient, + testSchemaName: "PUBLIC", + testDatabaseName: testDatabaseName, + }, nil +} + +func generateSFPeer(snowflakeConfig *protos.SnowflakeConfig) *protos.Peer { + ret := &protos.Peer{} + ret.Name = "test_sf_peer" + ret.Type = protos.DBType_SNOWFLAKE + + ret.Config = &protos.Peer_SnowflakeConfig{ + SnowflakeConfig: snowflakeConfig, + } + + return ret +} + +// Cleanup drops the database. +func (s *SnowflakeTestHelper) Cleanup() error { + err := s.testClient.Close() + if err != nil { + return err + } + err = s.adminClient.ExecuteQuery(fmt.Sprintf("DROP DATABASE %s", s.testDatabaseName)) + if err != nil { + return err + } + return s.adminClient.Close() +} + +// RunCommand runs the given command. +func (s *SnowflakeTestHelper) RunCommand(command string) error { + return s.testClient.ExecuteQuery(command) +} + +// CountRows(tableName) returns the number of rows in the given table. +func (s *SnowflakeTestHelper) CountRows(tableName string) (int, error) { + res, err := s.testClient.CountRows(s.testSchemaName, tableName) + if err != nil { + return 0, err + } + + return int(res), nil +} + +func (s *SnowflakeTestHelper) CheckNull(tableName string, colNames []string) (bool, error) { + return s.testClient.CheckNull(s.testSchemaName, tableName, colNames) +} + +func (s *SnowflakeTestHelper) ExecuteAndProcessQuery(query string) (*model.QRecordBatch, error) { + return s.testClient.ExecuteAndProcessQuery(query) +} + +func (s *SnowflakeTestHelper) CreateTable(tableName string, schema *model.QRecordSchema) error { + return s.testClient.CreateTable(schema, s.testSchemaName, tableName) +} diff --git a/flow/e2e/snowflake_helper.go b/flow/e2e/snowflake_helper.go deleted file mode 100644 index 090ddeccb7..0000000000 --- a/flow/e2e/snowflake_helper.go +++ /dev/null @@ -1,104 +0,0 @@ -package e2e - -import ( - "context" - "encoding/json" - "fmt" - "os" - - connsnowflake "github.com/PeerDB-io/peer-flow/connectors/snowflake" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/model" -) - -type SnowflakeTestHelper struct { - // config is the Snowflake config. - Config *protos.SnowflakeConfig - // peer struct holder Snowflake - Peer *protos.Peer - // connection to Snowflake - client *connsnowflake.SnowflakeClient - // testSchemaName is the schema to use for testing. - testSchemaName string -} - -func NewSnowflakeTestHelper(testSchemaName string) (*SnowflakeTestHelper, error) { - jsonPath := os.Getenv("TEST_SF_CREDS") - if jsonPath == "" { - return nil, fmt.Errorf("TEST_SF_CREDS env var not set") - } - - content, err := readFileToBytes(jsonPath) - if err != nil { - return nil, fmt.Errorf("failed to read file: %w", err) - } - - var config protos.SnowflakeConfig - err = json.Unmarshal(content, &config) - if err != nil { - return nil, fmt.Errorf("failed to unmarshal json: %w", err) - } - - peer := generateSFPeer(&config) - - client, err := connsnowflake.NewSnowflakeClient(context.Background(), &config) - if err != nil { - return nil, fmt.Errorf("failed to create Snowflake client: %w", err) - } - - return &SnowflakeTestHelper{ - Config: &config, - Peer: peer, - client: client, - testSchemaName: testSchemaName, - }, nil -} - -func generateSFPeer(snowflakeConfig *protos.SnowflakeConfig) *protos.Peer { - ret := &protos.Peer{} - ret.Name = "test_sf_peer" - ret.Type = protos.DBType_SNOWFLAKE - - ret.Config = &protos.Peer_SnowflakeConfig{ - SnowflakeConfig: snowflakeConfig, - } - - return ret -} - -// RecreateSchema recreates the schema, i.e., drops it if exists and creates it again. -func (s *SnowflakeTestHelper) RecreateSchema() error { - return s.client.RecreateSchema(s.testSchemaName) -} - -// DropSchema drops the schema. -func (s *SnowflakeTestHelper) DropSchema() error { - return s.client.DropSchema(s.testSchemaName) -} - -// RunCommand runs the given command. -func (s *SnowflakeTestHelper) RunCommand(command string) error { - return s.client.ExecuteQuery(command) -} - -// CountRows(tableName) returns the number of rows in the given table. -func (s *SnowflakeTestHelper) CountRows(tableName string) (int, error) { - res, err := s.client.CountRows(s.testSchemaName, tableName) - if err != nil { - return 0, err - } - - return int(res), nil -} - -func (s *SnowflakeTestHelper) CheckNull(tableName string, colNames []string) (bool, error) { - return s.client.CheckNull(s.testSchemaName, tableName, colNames) -} - -func (s *SnowflakeTestHelper) ExecuteAndProcessQuery(query string) (*model.QRecordBatch, error) { - return s.client.ExecuteAndProcessQuery(query) -} - -func (s *SnowflakeTestHelper) CreateTable(tableName string, schema *model.QRecordSchema) error { - return s.client.CreateTable(schema, s.testSchemaName, tableName) -} diff --git a/flow/e2e/qrep_flow_sqlserver_test.go b/flow/e2e/sqlserver/qrep_flow_sqlserver_test.go similarity index 53% rename from flow/e2e/qrep_flow_sqlserver_test.go rename to flow/e2e/sqlserver/qrep_flow_sqlserver_test.go index 80fe7e1e65..4a072f2011 100644 --- a/flow/e2e/qrep_flow_sqlserver_test.go +++ b/flow/e2e/sqlserver/qrep_flow_sqlserver_test.go @@ -1,23 +1,92 @@ -package e2e +package e2e_sqlserver import ( "context" "fmt" + "os" + "testing" "time" + "github.com/PeerDB-io/peer-flow/e2e" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" + "github.com/jackc/pgx/v5/pgxpool" + "github.com/joho/godotenv" + log "github.com/sirupsen/logrus" "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + "go.temporal.io/sdk/testsuite" ) -func (s *E2EPeerFlowTestSuite) setupSQLServerTable(tableName string) { +const sqlserverSuffix = "s3" + +type PeerFlowE2ETestSuiteSQLServer struct { + suite.Suite + testsuite.WorkflowTestSuite + + pool *pgxpool.Pool + sqlsHelper *SQLServerHelper +} + +func TestPeerFlowE2ETestSuiteSQLServer(t *testing.T) { + suite.Run(t, new(PeerFlowE2ETestSuiteSQLServer)) +} + +// setup sql server connection +func (s *PeerFlowE2ETestSuiteSQLServer) setupSQLServer() { + env := os.Getenv("ENABLE_SQLSERVER_TESTS") + if env != "true" { + s.sqlsHelper = nil + return + } + + sqlsHelper, err := NewSQLServerHelper("test_sqlserver_peer") + require.NoError(s.T(), err) + s.sqlsHelper = sqlsHelper +} + +func (s *PeerFlowE2ETestSuiteSQLServer) SetupSuite() { + err := godotenv.Load() + if err != nil { + // it's okay if the .env file is not present + // we will use the default values + log.Infof("Unable to load .env file, using default values from env") + } + + log.SetReportCaller(true) + + pool, err := e2e.SetupPostgres(sqlserverSuffix) + if err != nil { + s.Fail("failed to setup postgres", err) + } + s.pool = pool + + s.setupSQLServer() +} + +// Implement TearDownAllSuite interface to tear down the test suite +func (s *PeerFlowE2ETestSuiteSQLServer) TearDownSuite() { + err := e2e.TearDownPostgres(s.pool, sqlserverSuffix) + if err != nil { + s.Fail("failed to drop Postgres schema", err) + } + + if s.sqlsHelper != nil { + err = s.sqlsHelper.CleanUp() + if err != nil { + s.Fail("failed to clean up sqlserver", err) + } + } +} + +func (s *PeerFlowE2ETestSuiteSQLServer) setupSQLServerTable(tableName string) { schema := getSimpleTableSchema() err := s.sqlsHelper.CreateTable(schema, tableName) require.NoError(s.T(), err) } -func (s *E2EPeerFlowTestSuite) insertRowsIntoSQLServerTable(tableName string, numRows int) { +func (s *PeerFlowE2ETestSuiteSQLServer) insertRowsIntoSQLServerTable(tableName string, numRows int) { schemaQualified := fmt.Sprintf("%s.%s", s.sqlsHelper.SchemaName, tableName) for i := 0; i < numRows; i++ { params := make(map[string]interface{}) @@ -37,16 +106,15 @@ func (s *E2EPeerFlowTestSuite) insertRowsIntoSQLServerTable(tableName string, nu } } -func (s *E2EPeerFlowTestSuite) setupPGDestinationTable(schemaName, tableName string) { +func (s *PeerFlowE2ETestSuiteSQLServer) setupPGDestinationTable(tableName string) { ctx := context.Background() - _, err := s.pool.Exec(ctx, fmt.Sprintf("CREATE SCHEMA IF NOT EXISTS %s", schemaName)) - require.NoError(s.T(), err) - _, err = s.pool.Exec(ctx, fmt.Sprintf("DROP TABLE IF EXISTS %s.%s", schemaName, tableName)) + _, err := s.pool.Exec(ctx, fmt.Sprintf("DROP TABLE IF EXISTS e2e_test_%s.%s", sqlserverSuffix, tableName)) require.NoError(s.T(), err) - //nolint:lll - _, err = s.pool.Exec(ctx, fmt.Sprintf("CREATE TABLE %s.%s (id TEXT, card_id TEXT, v_from TIMESTAMP, price NUMERIC, status INT)", schemaName, tableName)) + _, err = s.pool.Exec(ctx, + fmt.Sprintf("CREATE TABLE e2e_test_%s.%s (id TEXT, card_id TEXT, v_from TIMESTAMP, price NUMERIC, status INT)", + sqlserverSuffix, tableName)) require.NoError(s.T(), err) } @@ -62,13 +130,13 @@ func getSimpleTableSchema() *model.QRecordSchema { } } -func (s *E2EPeerFlowTestSuite) Test_Complete_QRep_Flow_SqlServer_Append() { +func (s *PeerFlowE2ETestSuiteSQLServer) Test_Complete_QRep_Flow_SqlServer_Append() { if s.sqlsHelper == nil { s.T().Skip("Skipping SQL Server test") } env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env) numRows := 10 tblName := "test_qrep_flow_avro_ss_append" @@ -77,13 +145,13 @@ func (s *E2EPeerFlowTestSuite) Test_Complete_QRep_Flow_SqlServer_Append() { s.setupSQLServerTable(tblName) s.insertRowsIntoSQLServerTable(tblName, numRows) - s.setupPGDestinationTable(s.sqlsHelper.SchemaName, tblName) - dstTableName := fmt.Sprintf("%s.%s", s.sqlsHelper.SchemaName, tblName) + s.setupPGDestinationTable(tblName) + dstTableName := fmt.Sprintf("e2e_test_%s.%s", sqlserverSuffix, tblName) - //nolint:lll - query := fmt.Sprintf("SELECT * FROM %s.%s WHERE v_from BETWEEN {{.start}} AND {{.end}}", s.sqlsHelper.SchemaName, tblName) + query := fmt.Sprintf("SELECT * FROM %s.%s WHERE v_from BETWEEN {{.start}} AND {{.end}}", + s.sqlsHelper.SchemaName, tblName) - postgresPeer := GeneratePostgresPeer(postgresPort) + postgresPeer := e2e.GeneratePostgresPeer(e2e.PostgresPort) qrepConfig := &protos.QRepConfig{ FlowJobName: tblName, @@ -100,7 +168,7 @@ func (s *E2EPeerFlowTestSuite) Test_Complete_QRep_Flow_SqlServer_Append() { WaitBetweenBatchesSeconds: 5, } - runQrepFlowWorkflow(env, qrepConfig) + e2e.RunQrepFlowWorkflow(env, qrepConfig) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) diff --git a/flow/e2e/sqlserver_helper.go b/flow/e2e/sqlserver/sqlserver_helper.go similarity index 99% rename from flow/e2e/sqlserver_helper.go rename to flow/e2e/sqlserver/sqlserver_helper.go index 38933b35e1..1be94fc83e 100644 --- a/flow/e2e/sqlserver_helper.go +++ b/flow/e2e/sqlserver/sqlserver_helper.go @@ -1,4 +1,4 @@ -package e2e +package e2e_sqlserver import ( "context" diff --git a/flow/e2e/test_utils.go b/flow/e2e/test_utils.go index 293af9095f..761e0cbf51 100644 --- a/flow/e2e/test_utils.go +++ b/flow/e2e/test_utils.go @@ -1,13 +1,27 @@ package e2e import ( + "context" + "encoding/json" "fmt" "io" "os" + "strings" + "time" + + "github.com/PeerDB-io/peer-flow/activities" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/model" + "github.com/PeerDB-io/peer-flow/model/qvalue" + peerflow "github.com/PeerDB-io/peer-flow/workflows" + "github.com/google/uuid" + "github.com/jackc/pgx/v5/pgxpool" + log "github.com/sirupsen/logrus" + "go.temporal.io/sdk/testsuite" ) // readFileToBytes reads a file to a byte array. -func readFileToBytes(path string) ([]byte, error) { +func ReadFileToBytes(path string) ([]byte, error) { var ret []byte f, err := os.Open(path) @@ -24,3 +38,277 @@ func readFileToBytes(path string) ([]byte, error) { return ret, nil } + +func RegisterWorkflowsAndActivities(env *testsuite.TestWorkflowEnvironment) { + // set a 300 second timeout for the workflow to execute a few runs. + env.SetTestTimeout(300 * time.Second) + + env.RegisterWorkflow(peerflow.PeerFlowWorkflow) + env.RegisterWorkflow(peerflow.PeerFlowWorkflowWithConfig) + env.RegisterWorkflow(peerflow.SyncFlowWorkflow) + env.RegisterWorkflow(peerflow.SetupFlowWorkflow) + env.RegisterWorkflow(peerflow.SnapshotFlowWorkflow) + env.RegisterWorkflow(peerflow.NormalizeFlowWorkflow) + env.RegisterWorkflow(peerflow.QRepFlowWorkflow) + env.RegisterWorkflow(peerflow.QRepPartitionWorkflow) + env.RegisterActivity(&activities.FetchConfigActivity{}) + env.RegisterActivity(&activities.FlowableActivity{}) + env.RegisterActivity(&activities.SnapshotActivity{}) +} + +func SetupPeerFlowStatusQuery(env *testsuite.TestWorkflowEnvironment, + connectionGen FlowConnectionGenerationConfig) { + // wait for PeerFlowStatusQuery to finish setup + // sleep for 5 second to allow the workflow to start + time.Sleep(5 * time.Second) + for { + response, err := env.QueryWorkflow( + peerflow.PeerFlowStatusQuery, + connectionGen.FlowJobName, + ) + if err == nil { + var state peerflow.PeerFlowState + err = response.Get(&state) + if err != nil { + log.Errorln(err) + } + + if state.SetupComplete { + fmt.Println("query indicates setup is complete") + break + } + } else { + // log the error for informational purposes + log.Errorln(err) + } + time.Sleep(1 * time.Second) + } +} + +func CreateSourceTableQRep(pool *pgxpool.Pool, suffix string, tableName string) error { + tblFields := []string{ + "id UUID NOT NULL PRIMARY KEY", + "card_id UUID", + `"from" TIMESTAMP NOT NULL`, + "price NUMERIC", + "created_at TIMESTAMP NOT NULL", + "updated_at TIMESTAMP NOT NULL", + "transaction_hash BYTEA", + "ownerable_type VARCHAR", + "ownerable_id UUID", + "user_nonce INTEGER", + "transfer_type INTEGER DEFAULT 0 NOT NULL", + "blockchain INTEGER NOT NULL", + "deal_type VARCHAR", + "deal_id UUID", + "ethereum_transaction_id UUID", + "ignore_price BOOLEAN DEFAULT false", + "card_eth_value DOUBLE PRECISION", + "paid_eth_price DOUBLE PRECISION", + "card_bought_notified BOOLEAN DEFAULT false NOT NULL", + "address NUMERIC", + "account_id UUID", + "asset_id NUMERIC NOT NULL", + "status INTEGER", + "transaction_id UUID", + "settled_at TIMESTAMP", + "reference_id VARCHAR", + "settle_at TIMESTAMP", + "settlement_delay_reason INTEGER", + "f1 text[]", + "f2 bigint[]", + "f3 int[]", + "f4 varchar[]", + "f5 jsonb", + "f6 jsonb", + "f7 jsonb", + "f8 smallint", + } + + tblFieldStr := strings.Join(tblFields, ",") + + _, err := pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE e2e_test_%s.%s ( + %s + );`, suffix, tableName, tblFieldStr)) + if err != nil { + return err + } + + fmt.Printf("created table on postgres: e2e_test_%s.%s\n", suffix, tableName) + return nil +} + +func generate20MBJson() ([]byte, error) { + xn := make(map[string]interface{}) + for i := 0; i < 215000; i++ { + xn[uuid.New().String()] = uuid.New().String() + } + + v, err := json.Marshal(xn) + if err != nil { + return nil, err + } + + return v, nil +} + +func PopulateSourceTable(pool *pgxpool.Pool, suffix string, tableName string, rowCount int) error { + var ids []string + var rows []string + for i := 0; i < rowCount-1; i++ { + id := uuid.New().String() + ids = append(ids, id) + row := fmt.Sprintf(` + ( + '%s', '%s', CURRENT_TIMESTAMP, 3.86487206688919, CURRENT_TIMESTAMP, + CURRENT_TIMESTAMP, E'\\\\xDEADBEEF', 'type1', '%s', + 1, 0, 1, 'dealType1', + '%s', '%s', false, 1.2345, + 1.2345, false, 12345, '%s', + 12345, 1, '%s', CURRENT_TIMESTAMP, 'refID', + CURRENT_TIMESTAMP, 1, ARRAY['text1', 'text2'], ARRAY[123, 456], ARRAY[789, 012], + ARRAY['varchar1', 'varchar2'], '{"key": 8.5}', + '[{"key1": "value1", "key2": "value2", "key3": "value3"}]', + '{"key": "value"}', 15 + )`, + id, uuid.New().String(), uuid.New().String(), + uuid.New().String(), uuid.New().String(), uuid.New().String(), uuid.New().String()) + rows = append(rows, row) + } + + _, err := pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO e2e_test_%s.%s ( + id, card_id, "from", price, created_at, + updated_at, transaction_hash, ownerable_type, ownerable_id, + user_nonce, transfer_type, blockchain, deal_type, + deal_id, ethereum_transaction_id, ignore_price, card_eth_value, + paid_eth_price, card_bought_notified, address, account_id, + asset_id, status, transaction_id, settled_at, reference_id, + settle_at, settlement_delay_reason, f1, f2, f3, f4, f5, f6, f7, f8 + ) VALUES %s; + `, suffix, tableName, strings.Join(rows, ","))) + if err != nil { + return err + } + + // add a row where all the nullable fields are null + _, err = pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO e2e_test_%s.%s ( + id, "from", created_at, updated_at, + transfer_type, blockchain, card_bought_notified, asset_id + ) VALUES ( + '%s', CURRENT_TIMESTAMP, CURRENT_TIMESTAMP, CURRENT_TIMESTAMP, + 0, 1, false, 12345 + ); + `, suffix, tableName, uuid.New().String())) + if err != nil { + return err + } + + // generate a 20 MB json and update id[0]'s col f5 to it + v, err := generate20MBJson() + if err != nil { + return err + } + _, err = pool.Exec(context.Background(), fmt.Sprintf(` + UPDATE e2e_test_%s.%s SET f5 = $1 WHERE id = $2; + `, suffix, tableName), v, ids[0]) + if err != nil { + return err + } + + return nil +} + +func CreateQRepWorkflowConfig( + flowJobName string, + sourceTable string, + dstTable string, + query string, + syncMode protos.QRepSyncMode, + dest *protos.Peer, + stagingPath string, +) (*protos.QRepConfig, error) { + connectionGen := QRepFlowConnectionGenerationConfig{ + FlowJobName: flowJobName, + WatermarkTable: sourceTable, + DestinationTableIdentifier: dstTable, + PostgresPort: PostgresPort, + Destination: dest, + StagingPath: stagingPath, + } + + watermark := "updated_at" + + qrepConfig, err := connectionGen.GenerateQRepConfig(query, watermark, syncMode) + if err != nil { + return nil, err + } + + qrepConfig.InitialCopyOnly = true + + return qrepConfig, nil +} + +func RunQrepFlowWorkflow(env *testsuite.TestWorkflowEnvironment, config *protos.QRepConfig) { + lastPartition := &protos.QRepPartition{ + PartitionId: "not-applicable-partition", + Range: nil, + } + numPartitionsProcessed := 0 + env.ExecuteWorkflow(peerflow.QRepFlowWorkflow, config, lastPartition, numPartitionsProcessed) +} + +func GetOwnersSchema() *model.QRecordSchema { + return &model.QRecordSchema{ + Fields: []*model.QField{ + {Name: "id", Type: qvalue.QValueKindString, Nullable: true}, + {Name: "card_id", Type: qvalue.QValueKindString, Nullable: true}, + {Name: "from", Type: qvalue.QValueKindTimestamp, Nullable: true}, + {Name: "price", Type: qvalue.QValueKindNumeric, Nullable: true}, + {Name: "created_at", Type: qvalue.QValueKindTimestamp, Nullable: true}, + {Name: "updated_at", Type: qvalue.QValueKindTimestamp, Nullable: true}, + {Name: "transaction_hash", Type: qvalue.QValueKindBytes, Nullable: true}, + {Name: "ownerable_type", Type: qvalue.QValueKindString, Nullable: true}, + {Name: "ownerable_id", Type: qvalue.QValueKindString, Nullable: true}, + {Name: "user_nonce", Type: qvalue.QValueKindInt64, Nullable: true}, + {Name: "transfer_type", Type: qvalue.QValueKindInt64, Nullable: true}, + {Name: "blockchain", Type: qvalue.QValueKindInt64, Nullable: true}, + {Name: "deal_type", Type: qvalue.QValueKindString, Nullable: true}, + {Name: "deal_id", Type: qvalue.QValueKindString, Nullable: true}, + {Name: "ethereum_transaction_id", Type: qvalue.QValueKindString, Nullable: true}, + {Name: "ignore_price", Type: qvalue.QValueKindBoolean, Nullable: true}, + {Name: "card_eth_value", Type: qvalue.QValueKindFloat64, Nullable: true}, + {Name: "paid_eth_price", Type: qvalue.QValueKindFloat64, Nullable: true}, + {Name: "card_bought_notified", Type: qvalue.QValueKindBoolean, Nullable: true}, + {Name: "address", Type: qvalue.QValueKindNumeric, Nullable: true}, + {Name: "account_id", Type: qvalue.QValueKindString, Nullable: true}, + {Name: "asset_id", Type: qvalue.QValueKindNumeric, Nullable: true}, + {Name: "status", Type: qvalue.QValueKindInt64, Nullable: true}, + {Name: "transaction_id", Type: qvalue.QValueKindString, Nullable: true}, + {Name: "settled_at", Type: qvalue.QValueKindTimestamp, Nullable: true}, + {Name: "reference_id", Type: qvalue.QValueKindString, Nullable: true}, + {Name: "settle_at", Type: qvalue.QValueKindTimestamp, Nullable: true}, + {Name: "settlement_delay_reason", Type: qvalue.QValueKindInt64, Nullable: true}, + {Name: "f1", Type: qvalue.QValueKindArrayString, Nullable: true}, + {Name: "f2", Type: qvalue.QValueKindArrayInt64, Nullable: true}, + {Name: "f3", Type: qvalue.QValueKindArrayInt32, Nullable: true}, + {Name: "f4", Type: qvalue.QValueKindArrayString, Nullable: true}, + {Name: "f5", Type: qvalue.QValueKindJSON, Nullable: true}, + {Name: "f6", Type: qvalue.QValueKindJSON, Nullable: true}, + {Name: "f7", Type: qvalue.QValueKindJSON, Nullable: true}, + {Name: "f8", Type: qvalue.QValueKindInt16, Nullable: true}, + }, + } +} + +func GetOwnersSelectorString() string { + schema := GetOwnersSchema() + var fields []string + for _, field := range schema.Fields { + // append quoted field name + fields = append(fields, fmt.Sprintf(`"%s"`, field.Name)) + } + return strings.Join(fields, ",") +} From 86083fedff48df32bb1440bdbfcdd58b49eae6c8 Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Thu, 31 Aug 2023 07:39:14 -0700 Subject: [PATCH 25/37] improve avro writer heart beats (#366) --- flow/connectors/utils/avro/avro_writer.go | 37 ++++++++++++----------- 1 file changed, 19 insertions(+), 18 deletions(-) diff --git a/flow/connectors/utils/avro/avro_writer.go b/flow/connectors/utils/avro/avro_writer.go index 38c80d510f..6fd0d22bd3 100644 --- a/flow/connectors/utils/avro/avro_writer.go +++ b/flow/connectors/utils/avro/avro_writer.go @@ -5,16 +5,17 @@ import ( "fmt" "io" "os" + "time" "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" - "go.temporal.io/sdk/activity" "github.com/aws/aws-sdk-go/aws" "github.com/aws/aws-sdk-go/service/s3/s3manager" "github.com/linkedin/goavro/v2" log "github.com/sirupsen/logrus" + uber_atomic "go.uber.org/atomic" ) type PeerDBOCFWriter struct { @@ -54,8 +55,21 @@ func (p *PeerDBOCFWriter) writeRecordsToOCFWriter(ocfWriter *goavro.OCFWriter) ( } colNames := schema.GetColumnNames() - numRows := 0 - const heartBeatNumRows = 10000 + + var numRows uber_atomic.Uint32 + numRows.Store(0) + + if p.ctx != nil { + shutdown := utils.HeartbeatRoutine(p.ctx, 30*time.Second, func() string { + written := numRows.Load() + return fmt.Sprintf("[avro] written %d rows to OCF", written) + }) + + defer func() { + shutdown <- true + }() + } + for qRecordOrErr := range p.stream.Records { if qRecordOrErr.Err != nil { log.Errorf("[avro] failed to get record from stream: %v", qRecordOrErr.Err) @@ -82,23 +96,10 @@ func (p *PeerDBOCFWriter) writeRecordsToOCFWriter(ocfWriter *goavro.OCFWriter) ( return 0, fmt.Errorf("failed to write record to OCF: %w", err) } - if numRows%heartBeatNumRows == 0 { - log.Infof("written %d rows to OCF", numRows) - msg := fmt.Sprintf("written %d rows to OCF", numRows) - if p.ctx != nil { - activity.RecordHeartbeat(p.ctx, msg) - } - } - - numRows++ + numRows.Inc() } - if p.ctx != nil { - msg := fmt.Sprintf("written all: %d rows to OCF", numRows) - activity.RecordHeartbeat(p.ctx, msg) - } - - return numRows, nil + return int(numRows.Load()), nil } func (p *PeerDBOCFWriter) WriteOCF(w io.Writer) (int, error) { From 23ec4b2acbcea02f475f871d0449aa3339de9eb9 Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Fri, 1 Sep 2023 05:17:14 -0700 Subject: [PATCH 26/37] Send last processed client log position (#367) --- flow/connectors/postgres/cdc.go | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/flow/connectors/postgres/cdc.go b/flow/connectors/postgres/cdc.go index 0ea09f923d..f89ce4419f 100644 --- a/flow/connectors/postgres/cdc.go +++ b/flow/connectors/postgres/cdc.go @@ -131,8 +131,16 @@ func (p *PostgresCDCSource) consumeStream( for { if time.Now().After(nextStandbyMessageDeadline) { + // update the WALWritePosition to be clientXLogPos - 1 + // as the clientXLogPos is the last checkpoint id + 1 + // and we want to send the last checkpoint id as the last + // checkpoint id that we have processed. + lastProcessedXLogPos := clientXLogPos + if clientXLogPos > 0 { + lastProcessedXLogPos = clientXLogPos - 1 + } err := pglogrepl.SendStandbyStatusUpdate(p.ctx, conn, - pglogrepl.StandbyStatusUpdate{WALWritePosition: clientXLogPos}) + pglogrepl.StandbyStatusUpdate{WALWritePosition: lastProcessedXLogPos}) if err != nil { return nil, fmt.Errorf("SendStandbyStatusUpdate failed: %w", err) } @@ -239,8 +247,8 @@ func (p *PostgresCDCSource) consumeStream( } } - clientXLogPos = xld.WALStart + pglogrepl.LSN(len(xld.WALData)) - result.LastCheckPointID = int64(clientXLogPos) + currentPos := xld.WALStart + pglogrepl.LSN(len(xld.WALData)) + result.LastCheckPointID = int64(currentPos) if result.Records != nil && len(result.Records) == int(req.MaxBatchSize) { return result, nil From 09f26f3fb749c7fc2f80937e35a64083de44d7ab Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Fri, 1 Sep 2023 19:34:00 +0530 Subject: [PATCH 27/37] Ignore error for catalog monitor for EH (#369) CDC to Eventhub errors in SyncFlow with 'GetLastSyncBatchID not supported for Eventhub', because in `flowable.go`'s `StartFlow` this method is being called for catalog metrics. Eventhub does not have this method implemented so we can just ignore the error in this section and have sync ID in the metrics start from 0: ```Go if a.CatalogMirrorMonitor.IsActive() && len(records.Records) > 0 { syncBatchID, err := dest.GetLastSyncBatchID(input.FlowConnectionConfigs.FlowJobName) if err != nil && conn.Destination.Type != protos.DBType_EVENTHUB { return nil, err } .... } ``` --- flow/activities/flowable.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index 49e2f6a27a..6219069d91 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -205,7 +205,7 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, input *protos.StartFlo } if a.CatalogMirrorMonitor.IsActive() && len(records.Records) > 0 { syncBatchID, err := dest.GetLastSyncBatchID(input.FlowConnectionConfigs.FlowJobName) - if err != nil { + if err != nil && conn.Destination.Type != protos.DBType_EVENTHUB { return nil, err } From 3f48101348a8570e15a3d2985631c3fad65abf99 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Fri, 1 Sep 2023 21:19:29 +0530 Subject: [PATCH 28/37] Eventhub Normalize Fix (#370) `res` (result of normalize flow) in the below call is nil in case of Eventhub so we make it a non-nil: ```Go err = a.CatalogMirrorMonitor.UpdateEndTimeForCDCBatch(ctx, input.FlowConnectionConfigs.FlowJobName, res.EndBatchID) ``` --- flow/connectors/eventhub/eventhub.go | 6 +++++- flow/connectors/postgres/cdc.go | 2 +- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/flow/connectors/eventhub/eventhub.go b/flow/connectors/eventhub/eventhub.go index f94b5b0d3d..e674da2f67 100644 --- a/flow/connectors/eventhub/eventhub.go +++ b/flow/connectors/eventhub/eventhub.go @@ -310,7 +310,11 @@ func (c *EventHubConnector) SetupNormalizedTables( func (c *EventHubConnector) NormalizeRecords(req *model.NormalizeRecordsRequest) (*model.NormalizeResponse, error) { log.Infof("normalization for event hub is a no-op") - return nil, nil + return &model.NormalizeResponse{ + EndBatchID: 0, + StartBatchID: 0, + Done: true, + }, nil } // cleanup diff --git a/flow/connectors/postgres/cdc.go b/flow/connectors/postgres/cdc.go index f89ce4419f..a3e45b6ed4 100644 --- a/flow/connectors/postgres/cdc.go +++ b/flow/connectors/postgres/cdc.go @@ -144,7 +144,7 @@ func (p *PostgresCDCSource) consumeStream( if err != nil { return nil, fmt.Errorf("SendStandbyStatusUpdate failed: %w", err) } - log.Debugf("Sent Standby status message") + log.Infof("Sent Standby status message") nextStandbyMessageDeadline = time.Now().Add(standbyMessageTimeout) } From 5e20aa2660944be3a84422ffb8abe65ab71788a5 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 5 Sep 2023 18:45:54 +0530 Subject: [PATCH 29/37] Bump rustls-webpki from 0.100.1 to 0.100.2 in /nexus (#341) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Bumps [rustls-webpki](https://github.com/rustls/webpki) from 0.100.1 to 0.100.2.
Release notes

Sourced from rustls-webpki's releases.

v/0.100.2

Release notes

  • certificate path building and verification is now capped at 100 signature validation operations to avoid the risk of CPU usage denial-of-service attack when validating crafted certificate chains producing quadratic runtime. This risk affected both clients, as well as servers that verified client certificates.

What's Changed

Full Changelog: https://github.com/rustls/webpki/compare/v/0.100.1...v/0.100.2

Commits
  • c8b8214 Bump MSRV to 1.60
  • 8557522 Avoid testing MSRV of dev-dependencies
  • 73a7f0c Cargo: version 0.100.1 -> 0.100.2
  • 4ea0523 verify_cert: enforce maximum number of signatures.
  • See full diff in compare view

[![Dependabot compatibility score](https://dependabot-badges.githubapp.com/badges/compatibility_score?dependency-name=rustls-webpki&package-manager=cargo&previous-version=0.100.1&new-version=0.100.2)](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores) Dependabot will resolve any conflicts with this PR as long as you don't alter it yourself. You can also trigger a rebase manually by commenting `@dependabot rebase`. [//]: # (dependabot-automerge-start) [//]: # (dependabot-automerge-end) ---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR: - `@dependabot rebase` will rebase this PR - `@dependabot recreate` will recreate this PR, overwriting any edits that have been made to it - `@dependabot merge` will merge this PR after your CI passes on it - `@dependabot squash and merge` will squash and merge this PR after your CI passes on it - `@dependabot cancel merge` will cancel a previously requested merge and block automerging - `@dependabot reopen` will reopen this PR if it is closed - `@dependabot close` will close this PR and stop Dependabot recreating it. You can achieve the same result by closing it manually - `@dependabot show ignore conditions` will show all of the ignore conditions of the specified dependency - `@dependabot ignore this major version` will close this PR and stop Dependabot creating any more for this major version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this minor version` will close this PR and stop Dependabot creating any more for this minor version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this dependency` will close this PR and stop Dependabot creating any more for this dependency (unless you reopen the PR or upgrade to it yourself) You can disable automated security fix PRs for this repo from the [Security Alerts page](https://github.com/PeerDB-io/peerdb/network/alerts).
Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Kevin K Biju <52661649+heavycrystal@users.noreply.github.com> --- nexus/Cargo.lock | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/nexus/Cargo.lock b/nexus/Cargo.lock index 2e9ae610a3..d7ef91cbaa 100644 --- a/nexus/Cargo.lock +++ b/nexus/Cargo.lock @@ -2874,9 +2874,9 @@ dependencies = [ [[package]] name = "rustls-webpki" -version = "0.100.1" +version = "0.100.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d6207cd5ed3d8dca7816f8f3725513a34609c0c765bf652b8c3cb4cfd87db46b" +checksum = "e98ff011474fa39949b7e5c0428f9b4937eda7da7848bbb947786b7be0b27dab" dependencies = [ "ring", "untrusted", From cba15f0c1e4687ce13cd764343cdea53736dc3c4 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Tue, 5 Sep 2023 20:28:53 +0530 Subject: [PATCH 30/37] Parameters for EH CDC (#375) --- flow/activities/flowable.go | 10 +- flow/connectors/eventhub/eventhub.go | 17 +- flow/generated/protos/flow.pb.go | 836 ++++++++++++++------------- flow/model/model.go | 4 + nexus/analyzer/src/lib.rs | 19 +- nexus/flow-rs/src/grpc.rs | 2 + nexus/pt/src/flow_model.rs | 4 +- nexus/pt/src/peerdb_flow.rs | 5 + nexus/pt/src/peerdb_flow.serde.rs | 40 ++ protos/flow.proto | 4 + 10 files changed, 523 insertions(+), 418 deletions(-) diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index 6219069d91..a71ac05c36 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -237,10 +237,12 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, input *protos.StartFlo } res, err := dest.SyncRecords(&model.SyncRecordsRequest{ - Records: records, - FlowJobName: input.FlowConnectionConfigs.FlowJobName, - SyncMode: input.FlowConnectionConfigs.CdcSyncMode, - StagingPath: input.FlowConnectionConfigs.CdcStagingPath, + Records: records, + FlowJobName: input.FlowConnectionConfigs.FlowJobName, + SyncMode: input.FlowConnectionConfigs.CdcSyncMode, + StagingPath: input.FlowConnectionConfigs.CdcStagingPath, + PushBatchSize: input.FlowConnectionConfigs.PushBatchSize, + PushParallelism: input.FlowConnectionConfigs.PushParallelism, }) if err != nil { log.Warnf("failed to push records: %v", err) diff --git a/flow/connectors/eventhub/eventhub.go b/flow/connectors/eventhub/eventhub.go index e674da2f67..c4fba1e2b0 100644 --- a/flow/connectors/eventhub/eventhub.go +++ b/flow/connectors/eventhub/eventhub.go @@ -107,7 +107,8 @@ func (c *EventHubConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S batch := req.Records eventsPerHeartBeat := 1000 - eventsPerBatch := 100000 + eventsPerBatch := int(req.PushBatchSize) + maxParallelism := req.PushParallelism batchPerTopic := make(map[string][]*eventhub.Event) for i, record := range batch.Records { @@ -131,7 +132,7 @@ func (c *EventHubConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S } if (i+1)%eventsPerBatch == 0 { - err := c.sendEventBatch(batchPerTopic) + err := c.sendEventBatch(batchPerTopic, maxParallelism) if err != nil { return nil, err } @@ -142,7 +143,7 @@ func (c *EventHubConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S // send the remaining events. if len(batchPerTopic) > 0 { - err := c.sendEventBatch(batchPerTopic) + err := c.sendEventBatch(batchPerTopic, maxParallelism) if err != nil { return nil, err } @@ -163,7 +164,7 @@ func (c *EventHubConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S }, nil } -func (c *EventHubConnector) sendEventBatch(events map[string][]*eventhub.Event) error { +func (c *EventHubConnector) sendEventBatch(events map[string][]*eventhub.Event, maxParallelism int64) error { if len(events) == 0 { log.Info("no events to send") return nil @@ -176,11 +177,17 @@ func (c *EventHubConnector) sendEventBatch(events map[string][]*eventhub.Event) var wg sync.WaitGroup var once sync.Once var firstErr error + // Limiting concurrent sends + guard := make(chan struct{}, maxParallelism) for tblName, eventBatch := range events { + guard <- struct{}{} wg.Add(1) go func(tblName string, eventBatch []*eventhub.Event) { - defer wg.Done() + defer func() { + <-guard + wg.Done() + }() hub, err := c.getOrCreateHubConnection(tblName) if err != nil { diff --git a/flow/generated/protos/flow.pb.go b/flow/generated/protos/flow.pb.go index 00cb9fa2f6..13fca6b7b5 100644 --- a/flow/generated/protos/flow.pb.go +++ b/flow/generated/protos/flow.pb.go @@ -198,6 +198,9 @@ type FlowConnectionConfigs struct { // currently only works for snowflake SoftDelete bool `protobuf:"varint,19,opt,name=soft_delete,json=softDelete,proto3" json:"soft_delete,omitempty"` ReplicationSlotName string `protobuf:"bytes,20,opt,name=replication_slot_name,json=replicationSlotName,proto3" json:"replication_slot_name,omitempty"` + // the below two are for eventhub only + PushBatchSize int64 `protobuf:"varint,21,opt,name=push_batch_size,json=pushBatchSize,proto3" json:"push_batch_size,omitempty"` + PushParallelism int64 `protobuf:"varint,22,opt,name=push_parallelism,json=pushParallelism,proto3" json:"push_parallelism,omitempty"` } func (x *FlowConnectionConfigs) Reset() { @@ -372,6 +375,20 @@ func (x *FlowConnectionConfigs) GetReplicationSlotName() string { return "" } +func (x *FlowConnectionConfigs) GetPushBatchSize() int64 { + if x != nil { + return x.PushBatchSize + } + return 0 +} + +func (x *FlowConnectionConfigs) GetPushParallelism() int64 { + if x != nil { + return x.PushParallelism + } + return 0 +} + type SyncFlowOptions struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -2452,7 +2469,7 @@ var file_flow_proto_rawDesc = []byte{ 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x34, 0x0a, 0x16, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x14, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xe6, 0x0b, + 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xb9, 0x0c, 0x0a, 0x15, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x12, 0x2a, 0x0a, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, @@ -2532,426 +2549,431 @@ var file_flow_proto_rawDesc = []byte{ 0x0a, 0x15, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x14, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x6c, 0x6f, 0x74, 0x4e, 0x61, - 0x6d, 0x65, 0x1a, 0x43, 0x0a, 0x15, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, - 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, - 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x48, 0x0a, 0x1a, 0x53, 0x72, 0x63, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x49, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, - 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x0d, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, - 0x01, 0x1a, 0x63, 0x0a, 0x1b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, - 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, - 0x65, 0x79, 0x12, 0x2e, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, - 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x05, 0x76, 0x61, 0x6c, - 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x30, 0x0a, 0x0f, 0x53, 0x79, 0x6e, 0x63, 0x46, 0x6c, - 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x62, 0x61, 0x74, - 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x62, - 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x22, 0x35, 0x0a, 0x14, 0x4e, 0x6f, 0x72, 0x6d, - 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, - 0x12, 0x1d, 0x0a, 0x0a, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x62, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x22, - 0x71, 0x0a, 0x0d, 0x4c, 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x53, 0x74, 0x61, 0x74, 0x65, - 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, - 0x12, 0x40, 0x0a, 0x0e, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x73, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x5f, - 0x61, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, - 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, - 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0c, 0x6c, 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x65, 0x64, - 0x41, 0x74, 0x22, 0xfa, 0x01, 0x0a, 0x0e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x46, 0x6c, 0x6f, 0x77, - 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x42, 0x0a, 0x0f, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x73, 0x79, - 0x6e, 0x63, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, - 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x4c, 0x61, 0x73, - 0x74, 0x53, 0x79, 0x6e, 0x63, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x0d, 0x6c, 0x61, 0x73, 0x74, - 0x53, 0x79, 0x6e, 0x63, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x5a, 0x0a, 0x17, 0x66, 0x6c, 0x6f, + 0x6d, 0x65, 0x12, 0x26, 0x0a, 0x0f, 0x70, 0x75, 0x73, 0x68, 0x5f, 0x62, 0x61, 0x74, 0x63, 0x68, + 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x15, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x70, 0x75, 0x73, + 0x68, 0x42, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x70, 0x75, + 0x73, 0x68, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x69, 0x73, 0x6d, 0x18, 0x16, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x0f, 0x70, 0x75, 0x73, 0x68, 0x50, 0x61, 0x72, 0x61, 0x6c, 0x6c, + 0x65, 0x6c, 0x69, 0x73, 0x6d, 0x1a, 0x43, 0x0a, 0x15, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, + 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, + 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, + 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x48, 0x0a, 0x1a, 0x53, 0x72, + 0x63, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, + 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x63, 0x0a, 0x1b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, + 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2e, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, + 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x30, 0x0a, 0x0f, 0x53, 0x79, 0x6e, + 0x63, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1d, 0x0a, 0x0a, + 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, + 0x52, 0x09, 0x62, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x22, 0x35, 0x0a, 0x14, 0x4e, + 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x62, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, + 0x7a, 0x65, 0x22, 0x71, 0x0a, 0x0d, 0x4c, 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x53, 0x74, + 0x61, 0x74, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, + 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, + 0x69, 0x6e, 0x74, 0x12, 0x40, 0x0a, 0x0e, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x73, 0x79, 0x6e, 0x63, + 0x65, 0x64, 0x5f, 0x61, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, + 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, + 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0c, 0x6c, 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, + 0x63, 0x65, 0x64, 0x41, 0x74, 0x22, 0xfa, 0x01, 0x0a, 0x0e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x46, + 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x42, 0x0a, 0x0f, 0x6c, 0x61, 0x73, 0x74, + 0x5f, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, + 0x4c, 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x0d, 0x6c, + 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x5a, 0x0a, 0x17, + 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, + 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x46, 0x6c, 0x6f, 0x77, + 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x73, 0x52, 0x15, 0x66, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x12, 0x48, 0x0a, 0x11, 0x73, 0x79, 0x6e, 0x63, + 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, + 0x77, 0x2e, 0x53, 0x79, 0x6e, 0x63, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x52, 0x0f, 0x73, 0x79, 0x6e, 0x63, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x22, 0x71, 0x0a, 0x13, 0x53, 0x74, 0x61, 0x72, 0x74, 0x4e, 0x6f, 0x72, 0x6d, 0x61, + 0x6c, 0x69, 0x7a, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x5a, 0x0a, 0x17, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, - 0x66, 0x69, 0x67, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x65, 0x65, + 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, 0x48, 0x0a, 0x11, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x66, 0x6c, - 0x6f, 0x77, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, - 0x79, 0x6e, 0x63, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x0f, - 0x73, 0x79, 0x6e, 0x63, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, - 0x71, 0x0a, 0x13, 0x53, 0x74, 0x61, 0x72, 0x74, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, - 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x5a, 0x0a, 0x17, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x63, - 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, - 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, - 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x52, 0x15, 0x66, 0x6c, 0x6f, - 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x73, 0x22, 0x84, 0x01, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x53, 0x79, - 0x6e, 0x63, 0x65, 0x64, 0x49, 0x44, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, - 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, - 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, - 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, - 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, - 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xbe, 0x01, 0x0a, 0x16, 0x45, 0x6e, - 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x49, - 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, - 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, - 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, - 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, - 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, - 0x6d, 0x65, 0x12, 0x36, 0x0a, 0x17, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x74, 0x61, 0x62, - 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x15, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x22, 0xc5, 0x01, 0x0a, 0x1b, 0x45, - 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, - 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, - 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, - 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, - 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, - 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, - 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x38, 0x0a, 0x18, 0x73, 0x6f, 0x75, 0x72, - 0x63, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, - 0x69, 0x65, 0x72, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x16, 0x73, 0x6f, 0x75, 0x72, - 0x63, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, - 0x72, 0x73, 0x22, 0x30, 0x0a, 0x17, 0x50, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x15, 0x0a, - 0x06, 0x72, 0x65, 0x6c, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x72, - 0x65, 0x6c, 0x49, 0x64, 0x22, 0x89, 0x01, 0x0a, 0x0f, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, - 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x62, 0x0a, 0x19, 0x70, 0x6f, 0x73, 0x74, - 0x67, 0x72, 0x65, 0x73, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, - 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x70, 0x65, - 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x50, 0x6f, 0x73, 0x74, 0x67, 0x72, - 0x65, 0x73, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, - 0x72, 0x48, 0x00, 0x52, 0x17, 0x70, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x42, 0x12, 0x0a, 0x10, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, - 0x22, 0x62, 0x0a, 0x17, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, - 0x69, 0x6c, 0x69, 0x74, 0x79, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x47, 0x0a, 0x10, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, - 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, - 0x69, 0x65, 0x72, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, - 0x66, 0x69, 0x65, 0x72, 0x22, 0x88, 0x02, 0x0a, 0x1c, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, - 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, - 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x7f, 0x0a, 0x18, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, - 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, - 0x67, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x45, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, - 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, - 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, - 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, - 0x65, 0x72, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x16, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x4d, - 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x67, 0x0a, 0x1b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, - 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, - 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x32, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, - 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, - 0x66, 0x69, 0x65, 0x72, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, - 0x9a, 0x04, 0x0a, 0x15, 0x53, 0x65, 0x74, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, - 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, - 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, - 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, - 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, - 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, - 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, - 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x66, 0x0a, 0x12, 0x74, 0x61, 0x62, 0x6c, 0x65, - 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x03, 0x20, - 0x03, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, - 0x77, 0x2e, 0x53, 0x65, 0x74, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, - 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x10, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x12, - 0x3d, 0x0a, 0x10, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, - 0x65, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, - 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0f, 0x64, - 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x65, 0x65, 0x72, 0x12, 0x26, - 0x0a, 0x0f, 0x64, 0x6f, 0x5f, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x63, 0x6f, 0x70, - 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x64, 0x6f, 0x49, 0x6e, 0x69, 0x74, 0x69, - 0x61, 0x6c, 0x43, 0x6f, 0x70, 0x79, 0x12, 0x3a, 0x0a, 0x19, 0x65, 0x78, 0x69, 0x73, 0x74, 0x69, - 0x6e, 0x67, 0x5f, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, - 0x61, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x17, 0x65, 0x78, 0x69, 0x73, 0x74, - 0x69, 0x6e, 0x67, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, - 0x6d, 0x65, 0x12, 0x43, 0x0a, 0x1e, 0x65, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x72, - 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x5f, - 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1b, 0x65, 0x78, 0x69, 0x73, - 0x74, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, - 0x6c, 0x6f, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0x43, 0x0a, 0x15, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, - 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, - 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x5a, 0x0a, 0x16, - 0x53, 0x65, 0x74, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x6c, 0x6f, 0x74, 0x5f, 0x6e, - 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x73, 0x6c, 0x6f, 0x74, 0x4e, - 0x61, 0x6d, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x5f, - 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x6e, 0x61, 0x70, - 0x73, 0x68, 0x6f, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xed, 0x02, 0x0a, 0x13, 0x43, 0x72, 0x65, - 0x61, 0x74, 0x65, 0x52, 0x61, 0x77, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, - 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, - 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, - 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x64, - 0x0a, 0x12, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x6d, 0x61, 0x70, - 0x70, 0x69, 0x6e, 0x67, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x70, 0x65, 0x65, - 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, - 0x61, 0x77, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, - 0x72, 0x79, 0x52, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, - 0x70, 0x69, 0x6e, 0x67, 0x12, 0x3d, 0x0a, 0x0d, 0x63, 0x64, 0x63, 0x5f, 0x73, 0x79, 0x6e, 0x63, - 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x65, - 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x53, 0x79, - 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x0b, 0x63, 0x64, 0x63, 0x53, 0x79, 0x6e, 0x63, 0x4d, - 0x6f, 0x64, 0x65, 0x1a, 0x43, 0x0a, 0x15, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, - 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, - 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, - 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, - 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x41, 0x0a, 0x14, 0x43, 0x72, 0x65, 0x61, - 0x74, 0x65, 0x52, 0x61, 0x77, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, - 0x12, 0x29, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, - 0x66, 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x22, 0xe3, 0x01, 0x0a, 0x0b, - 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x29, 0x0a, 0x10, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, - 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x3f, 0x0a, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, - 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, - 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x07, - 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x2c, 0x0a, 0x12, 0x70, 0x72, 0x69, 0x6d, 0x61, - 0x72, 0x79, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x10, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x4b, 0x65, 0x79, 0x43, - 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x1a, 0x3a, 0x0a, 0x0c, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, - 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, - 0x01, 0x22, 0x91, 0x01, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, + 0x6e, 0x66, 0x69, 0x67, 0x73, 0x22, 0x84, 0x01, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x4c, 0x61, 0x73, + 0x74, 0x53, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x49, 0x44, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 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, 0x65, 0x72, 0x5f, + 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, + 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xbe, 0x01, 0x0a, + 0x16, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, + 0x74, 0x79, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x12, 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, + 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, + 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x36, 0x0a, 0x17, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, + 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x15, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x22, 0xc5, 0x01, + 0x0a, 0x1b, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, 0x6c, + 0x69, 0x74, 0x79, 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, + 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, + 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, + 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, + 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, + 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x38, 0x0a, 0x18, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, + 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x16, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, + 0x66, 0x69, 0x65, 0x72, 0x73, 0x22, 0x30, 0x0a, 0x17, 0x50, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, + 0x73, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, + 0x12, 0x15, 0x0a, 0x06, 0x72, 0x65, 0x6c, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, + 0x52, 0x05, 0x72, 0x65, 0x6c, 0x49, 0x64, 0x22, 0x89, 0x01, 0x0a, 0x0f, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x62, 0x0a, 0x19, 0x70, + 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, + 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, + 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x50, 0x6f, 0x73, + 0x74, 0x67, 0x72, 0x65, 0x73, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, + 0x66, 0x69, 0x65, 0x72, 0x48, 0x00, 0x52, 0x17, 0x70, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x42, + 0x12, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, + 0x69, 0x65, 0x72, 0x22, 0x62, 0x0a, 0x17, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, + 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x47, + 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, + 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, + 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, + 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, + 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x22, 0x88, 0x02, 0x0a, 0x1c, 0x45, 0x6e, 0x73, 0x75, + 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x42, 0x61, 0x74, + 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x7f, 0x0a, 0x18, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x5f, 0x6d, 0x61, 0x70, + 0x70, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x45, 0x2e, 0x70, 0x65, 0x65, + 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, + 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, + 0x75, 0x74, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, + 0x69, 0x66, 0x69, 0x65, 0x72, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x52, 0x16, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, + 0x65, 0x72, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x67, 0x0a, 0x1b, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x4d, 0x61, 0x70, 0x70, + 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x32, 0x0a, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, + 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, + 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, + 0x38, 0x01, 0x22, 0x9a, 0x04, 0x0a, 0x15, 0x53, 0x65, 0x74, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6c, + 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, + 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, + 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, + 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, + 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x66, 0x0a, 0x12, 0x74, 0x61, + 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, + 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, + 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x65, 0x74, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x52, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, + 0x6e, 0x67, 0x12, 0x3d, 0x0a, 0x10, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, + 0x52, 0x0f, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x65, 0x65, + 0x72, 0x12, 0x26, 0x0a, 0x0f, 0x64, 0x6f, 0x5f, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, + 0x63, 0x6f, 0x70, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x64, 0x6f, 0x49, 0x6e, + 0x69, 0x74, 0x69, 0x61, 0x6c, 0x43, 0x6f, 0x70, 0x79, 0x12, 0x3a, 0x0a, 0x19, 0x65, 0x78, 0x69, + 0x73, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x17, 0x65, 0x78, + 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x43, 0x0a, 0x1e, 0x65, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, + 0x67, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x6c, + 0x6f, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1b, 0x65, + 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x53, 0x6c, 0x6f, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0x43, 0x0a, 0x15, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, + 0x5a, 0x0a, 0x16, 0x53, 0x65, 0x74, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x6c, 0x6f, + 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x73, 0x6c, + 0x6f, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, + 0x6f, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x73, + 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xed, 0x02, 0x0a, 0x13, + 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x61, 0x77, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, + 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, + 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, + 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, + 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, + 0x65, 0x12, 0x64, 0x0a, 0x12, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, + 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, + 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x43, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x52, 0x61, 0x77, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, + 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x12, 0x3d, 0x0a, 0x0d, 0x63, 0x64, 0x63, 0x5f, 0x73, + 0x79, 0x6e, 0x63, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, + 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, + 0x70, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x0b, 0x63, 0x64, 0x63, 0x53, 0x79, + 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x1a, 0x43, 0x0a, 0x15, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, + 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, + 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, + 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x41, 0x0a, 0x14, 0x43, + 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x61, 0x77, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x75, 0x74, + 0x70, 0x75, 0x74, 0x12, 0x29, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, + 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x22, 0xe3, + 0x01, 0x0a, 0x0b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x29, + 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, + 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, + 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x3f, 0x0a, 0x07, 0x63, 0x6f, 0x6c, + 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x70, 0x65, 0x65, + 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x52, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x2c, 0x0a, 0x12, 0x70, 0x72, + 0x69, 0x6d, 0x61, 0x72, 0x79, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x4b, + 0x65, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x1a, 0x3a, 0x0a, 0x0c, 0x43, 0x6f, 0x6c, 0x75, + 0x6d, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x3a, 0x02, 0x38, 0x01, 0x22, 0x91, 0x01, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, + 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, + 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x2b, 0x0a, 0x11, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, + 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, + 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, 0x22, 0xff, 0x01, 0x0a, 0x19, 0x47, 0x65, 0x74, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x61, 0x74, 0x63, 0x68, + 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x7d, 0x0a, 0x19, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, + 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x6d, 0x61, 0x70, 0x70, + 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x42, 0x2e, 0x70, 0x65, 0x65, 0x72, + 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, + 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, + 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x16, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, + 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x63, 0x0a, 0x1b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, + 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2e, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, + 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xda, 0x01, 0x0a, 0x19, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, - 0x62, 0x6c, 0x65, 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, + 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, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, - 0x74, 0x72, 0x79, 0x52, 0x12, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, - 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x45, 0x0a, 0x17, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, - 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x3b, - 0x0a, 0x11, 0x49, 0x6e, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, - 0x6e, 0x67, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x03, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x65, 0x6e, 0x64, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0x79, 0x0a, 0x17, 0x54, - 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x30, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, - 0x70, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x2c, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, - 0x70, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0x4d, 0x0a, 0x03, 0x54, 0x49, 0x44, 0x12, 0x21, 0x0a, - 0x0c, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0d, 0x52, 0x0b, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, - 0x12, 0x23, 0x0a, 0x0d, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, - 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0c, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x4e, - 0x75, 0x6d, 0x62, 0x65, 0x72, 0x22, 0x5f, 0x0a, 0x11, 0x54, 0x49, 0x44, 0x50, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x26, 0x0a, 0x05, 0x73, 0x74, - 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x70, 0x65, 0x65, 0x72, - 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x49, 0x44, 0x52, 0x05, 0x73, 0x74, 0x61, - 0x72, 0x74, 0x12, 0x22, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x10, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x49, - 0x44, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0xe8, 0x01, 0x0a, 0x0e, 0x50, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x3d, 0x0a, 0x09, 0x69, 0x6e, 0x74, - 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, - 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x49, 0x6e, 0x74, 0x50, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x48, 0x00, 0x52, 0x08, - 0x69, 0x6e, 0x74, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x4f, 0x0a, 0x0f, 0x74, 0x69, 0x6d, 0x65, - 0x73, 0x74, 0x61, 0x6d, 0x70, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x24, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, - 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x48, 0x00, 0x52, 0x0e, 0x74, 0x69, 0x6d, 0x65, 0x73, - 0x74, 0x61, 0x6d, 0x70, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x3d, 0x0a, 0x09, 0x74, 0x69, 0x64, - 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, - 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x49, 0x44, 0x50, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x48, 0x00, 0x52, 0x08, - 0x74, 0x69, 0x64, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x42, 0x07, 0x0a, 0x05, 0x72, 0x61, 0x6e, 0x67, - 0x65, 0x22, 0x78, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4d, 0x6f, - 0x64, 0x65, 0x12, 0x39, 0x0a, 0x0a, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, - 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, - 0x70, 0x65, 0x52, 0x09, 0x77, 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x2c, 0x0a, - 0x12, 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, - 0x6d, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x75, 0x70, 0x73, 0x65, 0x72, - 0x74, 0x4b, 0x65, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x22, 0x96, 0x06, 0x0a, 0x0a, - 0x51, 0x52, 0x65, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, - 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x33, - 0x0a, 0x0b, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, - 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0a, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, - 0x65, 0x65, 0x72, 0x12, 0x3d, 0x0a, 0x10, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, - 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, - 0x72, 0x52, 0x0f, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x65, - 0x65, 0x72, 0x12, 0x40, 0x0a, 0x1c, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, - 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1a, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, - 0x66, 0x69, 0x65, 0x72, 0x12, 0x14, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x05, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x12, 0x27, 0x0a, 0x0f, 0x77, 0x61, - 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x06, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x0e, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, - 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x77, - 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x2a, - 0x0a, 0x11, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x63, 0x6f, 0x70, 0x79, 0x5f, 0x6f, - 0x6e, 0x6c, 0x79, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x69, 0x6e, 0x69, 0x74, 0x69, - 0x61, 0x6c, 0x43, 0x6f, 0x70, 0x79, 0x4f, 0x6e, 0x6c, 0x79, 0x12, 0x36, 0x0a, 0x09, 0x73, 0x79, - 0x6e, 0x63, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, - 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, - 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x08, 0x73, 0x79, 0x6e, 0x63, 0x4d, 0x6f, - 0x64, 0x65, 0x12, 0x24, 0x0a, 0x0e, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, - 0x5f, 0x69, 0x6e, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0c, 0x62, 0x61, 0x74, 0x63, - 0x68, 0x53, 0x69, 0x7a, 0x65, 0x49, 0x6e, 0x74, 0x12, 0x34, 0x0a, 0x16, 0x62, 0x61, 0x74, 0x63, - 0x68, 0x5f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, - 0x64, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x14, 0x62, 0x61, 0x74, 0x63, 0x68, 0x44, - 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x30, - 0x0a, 0x14, 0x6d, 0x61, 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x5f, 0x77, - 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x12, 0x6d, 0x61, - 0x78, 0x50, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73, - 0x12, 0x3f, 0x0a, 0x1c, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x62, 0x65, 0x74, 0x77, 0x65, 0x65, 0x6e, - 0x5f, 0x62, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, - 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x19, 0x77, 0x61, 0x69, 0x74, 0x42, 0x65, 0x74, 0x77, - 0x65, 0x65, 0x6e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, - 0x73, 0x12, 0x39, 0x0a, 0x0a, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, - 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, - 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4d, 0x6f, 0x64, - 0x65, 0x52, 0x09, 0x77, 0x72, 0x69, 0x74, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x21, 0x0a, 0x0c, - 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x0f, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x0b, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x50, 0x61, 0x74, 0x68, 0x12, - 0x33, 0x0a, 0x16, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x70, 0x65, 0x72, 0x5f, - 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0d, 0x52, - 0x13, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x50, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x97, 0x01, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x70, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x31, 0x0a, 0x05, 0x72, 0x61, 0x6e, - 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x30, 0x0a, 0x14, - 0x66, 0x75, 0x6c, 0x6c, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x66, 0x75, 0x6c, 0x6c, - 0x54, 0x61, 0x62, 0x6c, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x6b, - 0x0a, 0x12, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x42, - 0x61, 0x74, 0x63, 0x68, 0x12, 0x19, 0x0a, 0x08, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x69, 0x64, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x62, 0x61, 0x74, 0x63, 0x68, 0x49, 0x64, 0x12, - 0x3a, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, - 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, - 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, - 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x50, 0x0a, 0x12, 0x51, - 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c, - 0x74, 0x12, 0x3a, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, - 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, - 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x2c, 0x0a, - 0x0d, 0x44, 0x72, 0x6f, 0x70, 0x46, 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x1b, - 0x0a, 0x09, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x08, 0x66, 0x6c, 0x6f, 0x77, 0x4e, 0x61, 0x6d, 0x65, 0x2a, 0x50, 0x0a, 0x0c, 0x51, - 0x52, 0x65, 0x70, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x1f, 0x0a, 0x1b, 0x51, - 0x52, 0x45, 0x50, 0x5f, 0x53, 0x59, 0x4e, 0x43, 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x4d, 0x55, - 0x4c, 0x54, 0x49, 0x5f, 0x49, 0x4e, 0x53, 0x45, 0x52, 0x54, 0x10, 0x00, 0x12, 0x1f, 0x0a, 0x1b, - 0x51, 0x52, 0x45, 0x50, 0x5f, 0x53, 0x59, 0x4e, 0x43, 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x53, - 0x54, 0x4f, 0x52, 0x41, 0x47, 0x45, 0x5f, 0x41, 0x56, 0x52, 0x4f, 0x10, 0x01, 0x2a, 0x47, 0x0a, - 0x0d, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1a, - 0x0a, 0x16, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x5f, 0x4d, 0x4f, 0x44, - 0x45, 0x5f, 0x41, 0x50, 0x50, 0x45, 0x4e, 0x44, 0x10, 0x00, 0x12, 0x1a, 0x0a, 0x16, 0x51, 0x52, - 0x45, 0x50, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x55, 0x50, - 0x53, 0x45, 0x52, 0x54, 0x10, 0x01, 0x42, 0x76, 0x0a, 0x0f, 0x63, 0x6f, 0x6d, 0x2e, 0x70, 0x65, - 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x42, 0x09, 0x46, 0x6c, 0x6f, 0x77, 0x50, - 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x10, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, - 0x64, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x73, 0xa2, 0x02, 0x03, 0x50, 0x58, 0x58, 0xaa, 0x02, - 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0xca, 0x02, 0x0a, 0x50, 0x65, - 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0xe2, 0x02, 0x16, 0x50, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x46, 0x6c, 0x6f, 0x77, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, - 0x61, 0xea, 0x02, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0x62, 0x06, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, + 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, + 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, + 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x12, 0x82, 0x01, 0x0a, 0x19, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, + 0x6e, 0x67, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x47, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, + 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, + 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, + 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x52, 0x16, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x63, 0x0a, 0x1b, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, + 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2e, 0x0a, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x65, 0x65, 0x72, + 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x6e, + 0x0a, 0x1a, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, + 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x29, 0x0a, 0x10, + 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, + 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x25, 0x0a, 0x0e, 0x61, 0x6c, 0x72, 0x65, 0x61, + 0x64, 0x79, 0x5f, 0x65, 0x78, 0x69, 0x73, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x0d, 0x61, 0x6c, 0x72, 0x65, 0x61, 0x64, 0x79, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x22, 0xe0, + 0x01, 0x0a, 0x1f, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, + 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, + 0x75, 0x74, 0x12, 0x76, 0x0a, 0x14, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x65, 0x78, 0x69, 0x73, + 0x74, 0x73, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x44, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, + 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x2e, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, + 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x12, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, + 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x45, 0x0a, 0x17, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, + 0x01, 0x22, 0x3b, 0x0a, 0x11, 0x49, 0x6e, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x10, 0x0a, 0x03, + 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0x79, + 0x0a, 0x17, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x30, 0x0a, 0x05, 0x73, 0x74, 0x61, + 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, + 0x74, 0x61, 0x6d, 0x70, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x2c, 0x0a, 0x03, 0x65, + 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, + 0x74, 0x61, 0x6d, 0x70, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0x4d, 0x0a, 0x03, 0x54, 0x49, 0x44, + 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0b, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x4e, 0x75, 0x6d, + 0x62, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x5f, 0x6e, 0x75, + 0x6d, 0x62, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0c, 0x6f, 0x66, 0x66, 0x73, + 0x65, 0x74, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x22, 0x5f, 0x0a, 0x11, 0x54, 0x49, 0x44, 0x50, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x26, 0x0a, + 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x49, 0x44, 0x52, 0x05, + 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x22, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, + 0x2e, 0x54, 0x49, 0x44, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0xe8, 0x01, 0x0a, 0x0e, 0x50, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x3d, 0x0a, 0x09, + 0x69, 0x6e, 0x74, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x49, 0x6e, + 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x48, + 0x00, 0x52, 0x08, 0x69, 0x6e, 0x74, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x4f, 0x0a, 0x0f, 0x74, + 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, + 0x6f, 0x77, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x50, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x48, 0x00, 0x52, 0x0e, 0x74, 0x69, + 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x3d, 0x0a, 0x09, + 0x74, 0x69, 0x64, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x49, + 0x44, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x48, + 0x00, 0x52, 0x08, 0x74, 0x69, 0x64, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x42, 0x07, 0x0a, 0x05, 0x72, + 0x61, 0x6e, 0x67, 0x65, 0x22, 0x78, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, + 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x39, 0x0a, 0x0a, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x74, + 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, + 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, + 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x77, 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, + 0x12, 0x2c, 0x0a, 0x12, 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x63, + 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x75, 0x70, + 0x73, 0x65, 0x72, 0x74, 0x4b, 0x65, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x22, 0x96, + 0x06, 0x0a, 0x0a, 0x51, 0x52, 0x65, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, + 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, + 0x65, 0x12, 0x33, 0x0a, 0x0b, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x70, 0x65, 0x65, 0x72, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, + 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0a, 0x73, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x50, 0x65, 0x65, 0x72, 0x12, 0x3d, 0x0a, 0x10, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, + 0x50, 0x65, 0x65, 0x72, 0x52, 0x0f, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x50, 0x65, 0x65, 0x72, 0x12, 0x40, 0x0a, 0x1c, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, + 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1a, 0x64, 0x65, 0x73, + 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, + 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x14, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x12, 0x27, 0x0a, + 0x0f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, + 0x6b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, + 0x61, 0x72, 0x6b, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x43, 0x6f, 0x6c, 0x75, 0x6d, + 0x6e, 0x12, 0x2a, 0x0a, 0x11, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x63, 0x6f, 0x70, + 0x79, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x69, 0x6e, + 0x69, 0x74, 0x69, 0x61, 0x6c, 0x43, 0x6f, 0x70, 0x79, 0x4f, 0x6e, 0x6c, 0x79, 0x12, 0x36, 0x0a, + 0x09, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x19, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, + 0x52, 0x65, 0x70, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x08, 0x73, 0x79, 0x6e, + 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x24, 0x0a, 0x0e, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, + 0x69, 0x7a, 0x65, 0x5f, 0x69, 0x6e, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0c, 0x62, + 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x49, 0x6e, 0x74, 0x12, 0x34, 0x0a, 0x16, 0x62, + 0x61, 0x74, 0x63, 0x68, 0x5f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x65, + 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x14, 0x62, 0x61, 0x74, + 0x63, 0x68, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, + 0x73, 0x12, 0x30, 0x0a, 0x14, 0x6d, 0x61, 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, + 0x6c, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0d, 0x52, + 0x12, 0x6d, 0x61, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x57, 0x6f, 0x72, 0x6b, + 0x65, 0x72, 0x73, 0x12, 0x3f, 0x0a, 0x1c, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x62, 0x65, 0x74, 0x77, + 0x65, 0x65, 0x6e, 0x5f, 0x62, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x5f, 0x73, 0x65, 0x63, 0x6f, + 0x6e, 0x64, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x19, 0x77, 0x61, 0x69, 0x74, 0x42, + 0x65, 0x74, 0x77, 0x65, 0x65, 0x6e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x53, 0x65, 0x63, + 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x39, 0x0a, 0x0a, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x6d, 0x6f, + 0x64, 0x65, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, + 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, + 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x09, 0x77, 0x72, 0x69, 0x74, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x12, + 0x21, 0x0a, 0x0c, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, + 0x0f, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x50, 0x61, + 0x74, 0x68, 0x12, 0x33, 0x0a, 0x16, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x70, + 0x65, 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x10, 0x20, 0x01, + 0x28, 0x0d, 0x52, 0x13, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x50, 0x65, 0x72, 0x50, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x97, 0x01, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, + 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x31, 0x0a, 0x05, + 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x12, + 0x30, 0x0a, 0x14, 0x66, 0x75, 0x6c, 0x6c, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x70, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x66, + 0x75, 0x6c, 0x6c, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x22, 0x6b, 0x0a, 0x12, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x12, 0x19, 0x0a, 0x08, 0x62, 0x61, 0x74, 0x63, 0x68, + 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x62, 0x61, 0x74, 0x63, 0x68, + 0x49, 0x64, 0x12, 0x3a, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, + 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 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, 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, 0x47, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, + 0x65, 0x12, 0x1a, 0x0a, 0x16, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x5f, + 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x41, 0x50, 0x50, 0x45, 0x4e, 0x44, 0x10, 0x00, 0x12, 0x1a, 0x0a, + 0x16, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x5f, 0x4d, 0x4f, 0x44, 0x45, + 0x5f, 0x55, 0x50, 0x53, 0x45, 0x52, 0x54, 0x10, 0x01, 0x42, 0x76, 0x0a, 0x0f, 0x63, 0x6f, 0x6d, + 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x42, 0x09, 0x46, 0x6c, + 0x6f, 0x77, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x10, 0x67, 0x65, 0x6e, 0x65, 0x72, + 0x61, 0x74, 0x65, 0x64, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x73, 0xa2, 0x02, 0x03, 0x50, 0x58, + 0x58, 0xaa, 0x02, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0xca, 0x02, + 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0xe2, 0x02, 0x16, 0x50, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, + 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, + 0x77, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( diff --git a/flow/model/model.go b/flow/model/model.go index d87314ceef..ad2f22ef7b 100644 --- a/flow/model/model.go +++ b/flow/model/model.go @@ -179,6 +179,10 @@ type SyncRecordsRequest struct { SyncMode protos.QRepSyncMode // Staging path for AVRO files in CDC StagingPath string + // PushBatchSize is the number of records to push in a batch for EventHub. + PushBatchSize int64 + // PushParallelism is the number of batches in Event Hub to push in parallel. + PushParallelism int64 } type NormalizeRecordsRequest struct { diff --git a/nexus/analyzer/src/lib.rs b/nexus/analyzer/src/lib.rs index 70dae94f9d..6893b46457 100644 --- a/nexus/analyzer/src/lib.rs +++ b/nexus/analyzer/src/lib.rs @@ -230,6 +230,21 @@ impl StatementAnalyzer for PeerDDLAnalyzer { _ => false, }; + let push_parallelism: Option = match raw_options + .remove("push_parallelism") + { + Some(sqlparser::ast::Value::Number(n, _)) => Some(n.parse::()?), + _ => None, + }; + + let push_batch_size: Option = match raw_options + .remove("push_batch_size") + { + Some(sqlparser::ast::Value::Number(n, _)) => Some(n.parse::()?), + _ => None, + }; + + let flow_job = FlowJob { name: cdc.mirror_name.to_string().to_lowercase(), source_peer: cdc.source_peer.to_string().to_lowercase(), @@ -246,7 +261,9 @@ impl StatementAnalyzer for PeerDDLAnalyzer { cdc_sync_mode, cdc_staging_path, soft_delete, - replication_slot_name + replication_slot_name, + push_batch_size, + push_parallelism, }; // Error reporting diff --git a/nexus/flow-rs/src/grpc.rs b/nexus/flow-rs/src/grpc.rs index a749b22cb7..c06d7aa71e 100644 --- a/nexus/flow-rs/src/grpc.rs +++ b/nexus/flow-rs/src/grpc.rs @@ -162,6 +162,8 @@ impl FlowGrpcClient { cdc_staging_path: job.cdc_staging_path.clone().unwrap_or_default(), soft_delete: job.soft_delete, replication_slot_name: replication_slot_name.unwrap_or_default(), + push_batch_size: job.push_batch_size.unwrap_or_default(), + push_parallelism: job.push_parallelism.unwrap_or_default(), ..Default::default() }; diff --git a/nexus/pt/src/flow_model.rs b/nexus/pt/src/flow_model.rs index 74c2234824..e407bc287b 100644 --- a/nexus/pt/src/flow_model.rs +++ b/nexus/pt/src/flow_model.rs @@ -72,7 +72,9 @@ pub struct FlowJob { pub cdc_sync_mode: Option, pub cdc_staging_path: Option, pub soft_delete: bool, - pub replication_slot_name: Option + pub replication_slot_name: Option, + pub push_parallelism: Option, + pub push_batch_size: Option, } #[derive(Debug, PartialEq, Eq, Serialize, Deserialize, Clone)] diff --git a/nexus/pt/src/peerdb_flow.rs b/nexus/pt/src/peerdb_flow.rs index 6e996b372f..8612d7632c 100644 --- a/nexus/pt/src/peerdb_flow.rs +++ b/nexus/pt/src/peerdb_flow.rs @@ -54,6 +54,11 @@ pub struct FlowConnectionConfigs { pub soft_delete: bool, #[prost(string, tag="20")] pub replication_slot_name: ::prost::alloc::string::String, + /// the below two are for eventhub only + #[prost(int64, tag="21")] + pub push_batch_size: i64, + #[prost(int64, tag="22")] + pub push_parallelism: i64, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] diff --git a/nexus/pt/src/peerdb_flow.serde.rs b/nexus/pt/src/peerdb_flow.serde.rs index 2d2dc32e9f..3470c31dd9 100644 --- a/nexus/pt/src/peerdb_flow.serde.rs +++ b/nexus/pt/src/peerdb_flow.serde.rs @@ -871,6 +871,12 @@ impl serde::Serialize for FlowConnectionConfigs { if !self.replication_slot_name.is_empty() { len += 1; } + if self.push_batch_size != 0 { + len += 1; + } + if self.push_parallelism != 0 { + len += 1; + } let mut struct_ser = serializer.serialize_struct("peerdb_flow.FlowConnectionConfigs", len)?; if let Some(v) = self.source.as_ref() { struct_ser.serialize_field("source", v)?; @@ -936,6 +942,12 @@ impl serde::Serialize for FlowConnectionConfigs { if !self.replication_slot_name.is_empty() { struct_ser.serialize_field("replicationSlotName", &self.replication_slot_name)?; } + if self.push_batch_size != 0 { + struct_ser.serialize_field("pushBatchSize", ToString::to_string(&self.push_batch_size).as_str())?; + } + if self.push_parallelism != 0 { + struct_ser.serialize_field("pushParallelism", ToString::to_string(&self.push_parallelism).as_str())?; + } struct_ser.end() } } @@ -984,6 +996,10 @@ impl<'de> serde::Deserialize<'de> for FlowConnectionConfigs { "softDelete", "replication_slot_name", "replicationSlotName", + "push_batch_size", + "pushBatchSize", + "push_parallelism", + "pushParallelism", ]; #[allow(clippy::enum_variant_names)] @@ -1008,6 +1024,8 @@ impl<'de> serde::Deserialize<'de> for FlowConnectionConfigs { CdcStagingPath, SoftDelete, ReplicationSlotName, + PushBatchSize, + PushParallelism, __SkipField__, } impl<'de> serde::Deserialize<'de> for GeneratedField { @@ -1050,6 +1068,8 @@ impl<'de> serde::Deserialize<'de> for FlowConnectionConfigs { "cdcStagingPath" | "cdc_staging_path" => Ok(GeneratedField::CdcStagingPath), "softDelete" | "soft_delete" => Ok(GeneratedField::SoftDelete), "replicationSlotName" | "replication_slot_name" => Ok(GeneratedField::ReplicationSlotName), + "pushBatchSize" | "push_batch_size" => Ok(GeneratedField::PushBatchSize), + "pushParallelism" | "push_parallelism" => Ok(GeneratedField::PushParallelism), _ => Ok(GeneratedField::__SkipField__), } } @@ -1089,6 +1109,8 @@ impl<'de> serde::Deserialize<'de> for FlowConnectionConfigs { let mut cdc_staging_path__ = None; let mut soft_delete__ = None; let mut replication_slot_name__ = None; + let mut push_batch_size__ = None; + let mut push_parallelism__ = None; while let Some(k) = map.next_key()? { match k { GeneratedField::Source => { @@ -1226,6 +1248,22 @@ impl<'de> serde::Deserialize<'de> for FlowConnectionConfigs { } replication_slot_name__ = Some(map.next_value()?); } + GeneratedField::PushBatchSize => { + if push_batch_size__.is_some() { + return Err(serde::de::Error::duplicate_field("pushBatchSize")); + } + push_batch_size__ = + Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::PushParallelism => { + if push_parallelism__.is_some() { + return Err(serde::de::Error::duplicate_field("pushParallelism")); + } + push_parallelism__ = + Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } GeneratedField::__SkipField__ => { let _ = map.next_value::()?; } @@ -1252,6 +1290,8 @@ impl<'de> serde::Deserialize<'de> for FlowConnectionConfigs { cdc_staging_path: cdc_staging_path__.unwrap_or_default(), soft_delete: soft_delete__.unwrap_or_default(), replication_slot_name: replication_slot_name__.unwrap_or_default(), + push_batch_size: push_batch_size__.unwrap_or_default(), + push_parallelism: push_parallelism__.unwrap_or_default(), }) } } diff --git a/protos/flow.proto b/protos/flow.proto index a279a91914..7ddd873975 100644 --- a/protos/flow.proto +++ b/protos/flow.proto @@ -40,6 +40,10 @@ message FlowConnectionConfigs { bool soft_delete = 19; string replication_slot_name = 20; + + // the below two are for eventhub only + int64 push_batch_size = 21; + int64 push_parallelism = 22; } message SyncFlowOptions { int32 batch_size = 1; } From b49c51ecb3d5e8e6af5bbd3da5cf8654a7ee4831 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Tue, 5 Sep 2023 22:57:00 +0530 Subject: [PATCH 31/37] Event Hub CDC Logs (#374) --- flow/connectors/eventhub/eventhub.go | 49 +++++++++++++++++++++------- flow/connectors/eventhub/metadata.go | 8 +++-- 2 files changed, 44 insertions(+), 13 deletions(-) diff --git a/flow/connectors/eventhub/eventhub.go b/flow/connectors/eventhub/eventhub.go index c4fba1e2b0..e6e1e050ff 100644 --- a/flow/connectors/eventhub/eventhub.go +++ b/flow/connectors/eventhub/eventhub.go @@ -104,8 +104,16 @@ func (c *EventHubConnector) PullRecords(req *model.PullRecordsRequest) (*model.R } func (c *EventHubConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.SyncResponse, error) { - batch := req.Records + shutdown := utils.HeartbeatRoutine(c.ctx, 1*time.Minute, func() string { + return fmt.Sprintf("syncing records to eventhub with"+ + " push parallelism %d and push batch size %d", + req.PushParallelism, req.PushBatchSize) + }) + defer func() { + shutdown <- true + }() + batch := req.Records eventsPerHeartBeat := 1000 eventsPerBatch := int(req.PushBatchSize) maxParallelism := req.PushParallelism @@ -114,7 +122,9 @@ func (c *EventHubConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S for i, record := range batch.Records { json, err := record.GetItems().ToJSON() if err != nil { - log.Errorf("failed to convert record to json: %v", err) + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + }).Infof("failed to convert record to json: %v", err) return nil, err } @@ -132,7 +142,8 @@ func (c *EventHubConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S } if (i+1)%eventsPerBatch == 0 { - err := c.sendEventBatch(batchPerTopic, maxParallelism) + err := c.sendEventBatch(batchPerTopic, maxParallelism, + req.FlowJobName) if err != nil { return nil, err } @@ -143,13 +154,16 @@ func (c *EventHubConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S // send the remaining events. if len(batchPerTopic) > 0 { - err := c.sendEventBatch(batchPerTopic, maxParallelism) + err := c.sendEventBatch(batchPerTopic, maxParallelism, + req.FlowJobName) if err != nil { return nil, err } } - log.Infof("[total] successfully sent %d records to event hub", len(batch.Records)) + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + }).Infof("[total] successfully sent %d records to event hub", len(batch.Records)) err := c.UpdateLastOffset(req.FlowJobName, batch.LastCheckPointID) if err != nil { @@ -164,9 +178,13 @@ func (c *EventHubConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S }, nil } -func (c *EventHubConnector) sendEventBatch(events map[string][]*eventhub.Event, maxParallelism int64) error { +func (c *EventHubConnector) sendEventBatch(events map[string][]*eventhub.Event, + maxParallelism int64, + flowName string) error { if len(events) == 0 { - log.Info("no events to send") + log.WithFields(log.Fields{ + "flowName": flowName, + }).Infof("no events to send") return nil } @@ -194,7 +212,10 @@ func (c *EventHubConnector) sendEventBatch(events map[string][]*eventhub.Event, once.Do(func() { firstErr = err }) return } - + log.WithFields(log.Fields{ + "flowName": flowName, + }).Infof("obtained hub connection and now sending %d events to event hub: %s", + len(eventBatch), tblName) err = hub.SendBatch(subCtx, eventhub.NewEventBatchIterator(eventBatch...)) if err != nil { once.Do(func() { firstErr = err }) @@ -202,6 +223,10 @@ func (c *EventHubConnector) sendEventBatch(events map[string][]*eventhub.Event, } atomic.AddInt32(&numEventsPushed, int32(len(eventBatch))) + log.WithFields(log.Fields{ + "flowName": flowName, + }).Infof("pushed %d events to event hub: %s", + numEventsPushed, tblName) }(tblName, eventBatch) } @@ -237,7 +262,7 @@ func (c *EventHubConnector) CreateRawTable(req *protos.CreateRawTableInput) (*pr tableMap := req.GetTableNameMapping() for _, table := range tableMap { - err := c.ensureEventHub(c.ctx, table) + err := c.ensureEventHub(c.ctx, table, req.FlowJobName) if err != nil { log.WithFields(log.Fields{ "flowName": req.FlowJobName, @@ -255,7 +280,7 @@ func (c *EventHubConnector) GetTableSchema( panic("get table schema not implemented for event hub") } -func (c *EventHubConnector) ensureEventHub(ctx context.Context, name string) error { +func (c *EventHubConnector) ensureEventHub(ctx context.Context, name string, flowName string) error { hubClient, err := c.getEventHubMgmtClient() if err != nil { return err @@ -282,7 +307,9 @@ func (c *EventHubConnector) ensureEventHub(ctx context.Context, name string) err return err } - log.Infof("event hub %s created", name) + log.WithFields(log.Fields{ + "flowName": flowName, + }).Infof("event hub %s created", name) } else { log.Infof("event hub %s already exists", name) } diff --git a/flow/connectors/eventhub/metadata.go b/flow/connectors/eventhub/metadata.go index 84175b0d27..07bf30b339 100644 --- a/flow/connectors/eventhub/metadata.go +++ b/flow/connectors/eventhub/metadata.go @@ -125,7 +125,9 @@ func (c *EventHubConnector) GetLastOffset(jobName string) (*protos.LastSyncState }, nil } - log.Errorf("failed to get last offset: %v", err) + log.WithFields(log.Fields{ + "flowName": jobName, + }).Errorf("failed to get last offset: %v", err) return nil, err } @@ -155,7 +157,9 @@ func (c *EventHubConnector) UpdateLastOffset(jobName string, offset int64) error } // update the last offset - log.Infof("updating last offset for job `%s` to `%d`", jobName, offset) + log.WithFields(log.Fields{ + "flowName": jobName, + }).Infof("updating last offset for job `%s` to `%d`", jobName, offset) _, err = tx.Exec(c.ctx, ` INSERT INTO `+metadataSchema+`.`+lastSyncStateTableName+` (job_name, last_offset) VALUES ($1, $2) From 40a454a68e86fbac1ed0e304c9b8a7856bbfefa1 Mon Sep 17 00:00:00 2001 From: Kevin K Biju <52661649+heavycrystal@users.noreply.github.com> Date: Wed, 6 Sep 2023 21:12:35 +0530 Subject: [PATCH 32/37] Basic ADD COLUMN and DROP COLUMN replay support for PG, BQ and SF (#368) --- flow/activities/flowable.go | 42 +- flow/connectors/bigquery/bigquery.go | 40 +- flow/connectors/core.go | 5 +- flow/connectors/eventhub/eventhub.go | 7 +- flow/connectors/postgres/cdc.go | 176 +- flow/connectors/postgres/postgres.go | 81 +- flow/connectors/postgres/postgres_cdc_test.go | 80 +- .../connectors/postgres/postgres_repl_test.go | 5 +- flow/connectors/s3/s3.go | 7 +- flow/connectors/snowflake/snowflake.go | 59 +- flow/connectors/sqlserver/sqlserver.go | 7 +- flow/e2e/bigquery/bigquery_helper.go | 4 +- flow/e2e/bigquery/peer_flow_bq_test.go | 112 +- flow/e2e/bigquery/qrep_flow_bq_test.go | 28 + flow/e2e/postgres/peer_flow_pg_test.go | 184 ++ flow/e2e/postgres/postgres_helper.go | 15 + flow/e2e/postgres/qrep_flow_pg_test.go | 39 +- flow/e2e/snowflake/peer_flow_sf_test.go | 104 + flow/e2e/snowflake/qrep_flow_sf_test.go | 32 +- flow/e2e/test_utils.go | 30 + flow/generated/protos/flow.pb.go | 1944 ++++++++++------- flow/model/model.go | 34 + flow/workflows/peer_flow.go | 57 +- flow/workflows/sync_flow.go | 25 +- nexus/pt/src/peerdb_flow.rs | 52 + nexus/pt/src/peerdb_flow.serde.rs | 687 ++++++ protos/flow.proto | 35 +- stacks/flow-api.Dockerfile | 2 +- stacks/flow-snapshot-worker.Dockerfile | 2 +- stacks/flow-worker.Dockerfile | 2 +- 30 files changed, 3027 insertions(+), 870 deletions(-) create mode 100644 flow/e2e/postgres/peer_flow_pg_test.go create mode 100644 flow/e2e/postgres/postgres_helper.go diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index a71ac05c36..4837eb9d2e 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -159,7 +159,8 @@ func (a *FlowableActivity) CreateNormalizedTable( } // StartFlow implements StartFlow. -func (a *FlowableActivity) StartFlow(ctx context.Context, input *protos.StartFlowInput) (*model.SyncResponse, error) { +func (a *FlowableActivity) StartFlow(ctx context.Context, + input *protos.StartFlowInput) (*model.SyncResponse, error) { conn := input.FlowConnectionConfigs ctx = context.WithValue(ctx, shared.EnableMetricsKey, a.EnableMetrics) @@ -189,7 +190,7 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, input *protos.StartFlo }).Info("pulling records...") startTime := time.Now() - records, err := src.PullRecords(&model.PullRecordsRequest{ + recordsWithTableSchemaDelta, err := src.PullRecords(&model.PullRecordsRequest{ FlowJobName: input.FlowConnectionConfigs.FlowJobName, SrcTableIDNameMapping: input.FlowConnectionConfigs.SrcTableIdNameMapping, TableNameMapping: input.FlowConnectionConfigs.TableNameMapping, @@ -199,11 +200,13 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, input *protos.StartFlo TableNameSchemaMapping: input.FlowConnectionConfigs.TableNameSchemaMapping, OverridePublicationName: input.FlowConnectionConfigs.PublicationName, OverrideReplicationSlotName: input.FlowConnectionConfigs.ReplicationSlotName, + RelationMessageMapping: input.RelationMessageMapping, }) if err != nil { return nil, fmt.Errorf("failed to pull records: %w", err) } - if a.CatalogMirrorMonitor.IsActive() && len(records.Records) > 0 { + recordBatch := recordsWithTableSchemaDelta.RecordBatch + if a.CatalogMirrorMonitor.IsActive() && len(recordBatch.Records) > 0 { syncBatchID, err := dest.GetLastSyncBatchID(input.FlowConnectionConfigs.FlowJobName) if err != nil && conn.Destination.Type != protos.DBType_EVENTHUB { return nil, err @@ -212,9 +215,9 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, input *protos.StartFlo err = a.CatalogMirrorMonitor.AddCDCBatchForFlow(ctx, input.FlowConnectionConfigs.FlowJobName, monitoring.CDCBatchInfo{ BatchID: syncBatchID + 1, - RowsInBatch: uint32(len(records.Records)), - BatchStartLSN: pglogrepl.LSN(records.FirstCheckPointID), - BatchEndlSN: pglogrepl.LSN(records.LastCheckPointID), + RowsInBatch: uint32(len(recordBatch.Records)), + BatchStartLSN: pglogrepl.LSN(recordBatch.FirstCheckPointID), + BatchEndlSN: pglogrepl.LSN(recordBatch.LastCheckPointID), StartTime: startTime, }) if err != nil { @@ -223,7 +226,7 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, input *protos.StartFlo } // log the number of records - numRecords := len(records.Records) + numRecords := len(recordBatch.Records) log.WithFields(log.Fields{ "flowName": input.FlowConnectionConfigs.FlowJobName, }).Printf("pulled %d records", numRecords) @@ -233,11 +236,14 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, input *protos.StartFlo log.WithFields(log.Fields{ "flowName": input.FlowConnectionConfigs.FlowJobName, }).Info("no records to push") - return nil, nil + return &model.SyncResponse{ + RelationMessageMapping: recordsWithTableSchemaDelta.RelationMessageMapping, + TableSchemaDelta: recordsWithTableSchemaDelta.TableSchemaDelta, + }, nil } res, err := dest.SyncRecords(&model.SyncRecordsRequest{ - Records: records, + Records: recordBatch, FlowJobName: input.FlowConnectionConfigs.FlowJobName, SyncMode: input.FlowConnectionConfigs.CdcSyncMode, StagingPath: input.FlowConnectionConfigs.CdcStagingPath, @@ -254,7 +260,7 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, input *protos.StartFlo err = a.CatalogMirrorMonitor. UpdateLatestLSNAtTargetForCDCFlow(ctx, input.FlowConnectionConfigs.FlowJobName, - pglogrepl.LSN(records.LastCheckPointID)) + pglogrepl.LSN(recordBatch.LastCheckPointID)) if err != nil { return nil, err } @@ -265,7 +271,8 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, input *protos.StartFlo return nil, err } } - + res.TableSchemaDelta = recordsWithTableSchemaDelta.TableSchemaDelta + res.RelationMessageMapping = recordsWithTableSchemaDelta.RelationMessageMapping activity.RecordHeartbeat(ctx, "pushed records") return res, nil @@ -325,6 +332,19 @@ func (a *FlowableActivity) StartNormalize( return res, nil } +func (a *FlowableActivity) ReplayTableSchemaDelta( + ctx context.Context, + input *protos.ReplayTableSchemaDeltaInput, +) error { + dest, err := connectors.GetConnector(ctx, input.FlowConnectionConfigs.Destination) + defer connectors.CloseConnector(dest) + if err != nil { + return fmt.Errorf("failed to get destination connector: %w", err) + } + + return dest.ReplayTableSchemaDelta(input.FlowConnectionConfigs.FlowJobName, input.TableSchemaDelta) +} + // SetupQRepMetadataTables sets up the metadata tables for QReplication. func (a *FlowableActivity) SetupQRepMetadataTables(ctx context.Context, config *protos.QRepConfig) error { conn, err := connectors.GetConnector(ctx, config.DestinationPeer) diff --git a/flow/connectors/bigquery/bigquery.go b/flow/connectors/bigquery/bigquery.go index f3e0d84863..2074f2b62e 100644 --- a/flow/connectors/bigquery/bigquery.go +++ b/flow/connectors/bigquery/bigquery.go @@ -205,6 +205,43 @@ func (c *BigQueryConnector) InitializeTableSchema(req map[string]*protos.TableSc return nil } +// ReplayTableSchemaDelta changes a destination table to match the schema at source +// This could involve adding or dropping multiple columns. +func (c *BigQueryConnector) ReplayTableSchemaDelta(flowJobName string, + schemaDelta *protos.TableSchemaDelta) error { + if (schemaDelta == nil) || (len(schemaDelta.AddedColumns) == 0 && len(schemaDelta.DroppedColumns) == 0) { + return nil + } + + for _, droppedColumn := range schemaDelta.DroppedColumns { + _, err := c.client.Query(fmt.Sprintf("ALTER TABLE %s.%s DROP COLUMN %s", c.datasetID, + schemaDelta.DstTableName, droppedColumn)).Read(c.ctx) + if err != nil { + return fmt.Errorf("failed to drop column %s for table %s: %w", droppedColumn, + schemaDelta.SrcTableName, err) + } + log.WithFields(log.Fields{ + "flowName": flowJobName, + "tableName": schemaDelta.SrcTableName, + }).Infof("[schema delta replay] dropped column %s", droppedColumn) + } + for _, addedColumn := range schemaDelta.AddedColumns { + _, err := c.client.Query(fmt.Sprintf("ALTER TABLE %s.%s ADD COLUMN %s %s", c.datasetID, + schemaDelta.DstTableName, addedColumn.ColumnName, addedColumn.ColumnType)).Read(c.ctx) + if err != nil { + return fmt.Errorf("failed to add column %s for table %s: %w", addedColumn.ColumnName, + schemaDelta.SrcTableName, err) + } + log.WithFields(log.Fields{ + "flowName": flowJobName, + "tableName": schemaDelta.SrcTableName, + }).Infof("[schema delta replay] added column %s with data type %s", addedColumn.ColumnName, + addedColumn.ColumnType) + } + + return nil +} + // SetupMetadataTables sets up the metadata tables. func (c *BigQueryConnector) SetupMetadataTables() error { // check if the dataset exists @@ -383,7 +420,8 @@ func (c *BigQueryConnector) getTableNametoUnchangedCols(flowJobName string, sync } // PullRecords pulls records from the source. -func (c *BigQueryConnector) PullRecords(req *model.PullRecordsRequest) (*model.RecordBatch, error) { +func (c *BigQueryConnector) PullRecords(req *model.PullRecordsRequest) ( + *model.RecordsWithTableSchemaDelta, error) { panic("not implemented") } diff --git a/flow/connectors/core.go b/flow/connectors/core.go index 18ef1ada1c..37d6a0aec2 100644 --- a/flow/connectors/core.go +++ b/flow/connectors/core.go @@ -35,12 +35,15 @@ type Connector interface { // InitializeTableSchema initializes the table schema of all the destination tables for the connector. InitializeTableSchema(req map[string]*protos.TableSchema) error + // ReplayTableSchemaDelta changes a destination table to match the schema at source + // This could involve adding or dropping multiple columns. + ReplayTableSchemaDelta(flowJobName string, schemaDelta *protos.TableSchemaDelta) error // Methods related to retrieving and pusing records for this connector as a source and destination. // PullRecords pulls records from the source, and returns a RecordBatch. // This method should be idempotent, and should be able to be called multiple times with the same request. - PullRecords(req *model.PullRecordsRequest) (*model.RecordBatch, error) + PullRecords(req *model.PullRecordsRequest) (*model.RecordsWithTableSchemaDelta, error) // SyncRecords pushes records to the destination peer and stores it in PeerDB specific tables. // This method should be idempotent, and should be able to be called multiple times with the same request. diff --git a/flow/connectors/eventhub/eventhub.go b/flow/connectors/eventhub/eventhub.go index e6e1e050ff..70c1adafa8 100644 --- a/flow/connectors/eventhub/eventhub.go +++ b/flow/connectors/eventhub/eventhub.go @@ -99,7 +99,12 @@ func (c *EventHubConnector) InitializeTableSchema(req map[string]*protos.TableSc return nil } -func (c *EventHubConnector) PullRecords(req *model.PullRecordsRequest) (*model.RecordBatch, error) { +func (c *EventHubConnector) ReplayTableSchemaDelta(flowJobName string, schemaDelta *protos.TableSchemaDelta) error { + log.Warnf("ReplayTableSchemaDelta is a no-op for EventHub flow connector") + return nil +} + +func (c *EventHubConnector) PullRecords(req *model.PullRecordsRequest) (*model.RecordsWithTableSchemaDelta, error) { panic("pull records not implemented for event hub") } diff --git a/flow/connectors/postgres/cdc.go b/flow/connectors/postgres/cdc.go index a3e45b6ed4..34d91dea45 100644 --- a/flow/connectors/postgres/cdc.go +++ b/flow/connectors/postgres/cdc.go @@ -6,6 +6,7 @@ import ( "reflect" "time" + "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/jackc/pglogrepl" @@ -18,42 +19,44 @@ import ( ) type PostgresCDCSource struct { - ctx context.Context - replPool *pgxpool.Pool - SrcTableIDNameMapping map[uint32]string - TableNameMapping map[string]string - slot string - publication string - relations map[uint32]*pglogrepl.RelationMessage - typeMap *pgtype.Map - startLSN pglogrepl.LSN + ctx context.Context + replPool *pgxpool.Pool + SrcTableIDNameMapping map[uint32]string + TableNameMapping map[string]string + slot string + publication string + relationMessageMapping model.RelationMessageMapping + typeMap *pgtype.Map + startLSN pglogrepl.LSN } type PostgresCDCConfig struct { - AppContext context.Context - Connection *pgxpool.Pool - Slot string - Publication string - SrcTableIDNameMapping map[uint32]string - TableNameMapping map[string]string + AppContext context.Context + Connection *pgxpool.Pool + Slot string + Publication string + SrcTableIDNameMapping map[uint32]string + TableNameMapping map[string]string + RelationMessageMapping model.RelationMessageMapping } // Create a new PostgresCDCSource func NewPostgresCDCSource(cdcConfig *PostgresCDCConfig) (*PostgresCDCSource, error) { return &PostgresCDCSource{ - ctx: cdcConfig.AppContext, - replPool: cdcConfig.Connection, - SrcTableIDNameMapping: cdcConfig.SrcTableIDNameMapping, - TableNameMapping: cdcConfig.TableNameMapping, - slot: cdcConfig.Slot, - publication: cdcConfig.Publication, - relations: make(map[uint32]*pglogrepl.RelationMessage), - typeMap: pgtype.NewMap(), + ctx: cdcConfig.AppContext, + replPool: cdcConfig.Connection, + SrcTableIDNameMapping: cdcConfig.SrcTableIDNameMapping, + TableNameMapping: cdcConfig.TableNameMapping, + slot: cdcConfig.Slot, + publication: cdcConfig.Publication, + relationMessageMapping: cdcConfig.RelationMessageMapping, + typeMap: pgtype.NewMap(), }, nil } // PullRecords pulls records from the cdc stream -func (p *PostgresCDCSource) PullRecords(req *model.PullRecordsRequest) (*model.RecordBatch, error) { +func (p *PostgresCDCSource) PullRecords(req *model.PullRecordsRequest) ( + *model.RecordsWithTableSchemaDelta, error) { // setup options pluginArguments := []string{ "proto_version '1'", @@ -110,16 +113,23 @@ func (p *PostgresCDCSource) consumeStream( conn *pgconn.PgConn, req *model.PullRecordsRequest, clientXLogPos pglogrepl.LSN, -) (*model.RecordBatch, error) { +) (*model.RecordsWithTableSchemaDelta, error) { // TODO (kaushik): take into consideration the MaxBatchSize // parameters in the original request. - result := &model.RecordBatch{ + records := &model.RecordBatch{ Records: make([]model.Record, 0), TablePKeyLastSeen: make(map[model.TableWithPkey]int), } + result := &model.RecordsWithTableSchemaDelta{ + RecordBatch: records, + TableSchemaDelta: nil, + RelationMessageMapping: p.relationMessageMapping, + } standbyMessageTimeout := req.IdleTimeout nextStandbyMessageDeadline := time.Now().Add(standbyMessageTimeout) + earlyReturn := false + defer func() { err := conn.Close(p.ctx) if err != nil { @@ -193,7 +203,7 @@ func (p *PostgresCDCSource) consumeStream( log.Debugf("XLogData => WALStart %s ServerWALEnd %s ServerTime %s\n", xld.WALStart, xld.ServerWALEnd, xld.ServerTime) - rec, err := p.processMessage(result, xld) + rec, err := p.processMessage(records, xld) if err != nil { return nil, fmt.Errorf("error processing message: %w", err) @@ -201,7 +211,7 @@ func (p *PostgresCDCSource) consumeStream( if !firstProcessed { firstProcessed = true - result.FirstCheckPointID = int64(xld.WALStart) + records.FirstCheckPointID = int64(xld.WALStart) } if rec != nil { tableName := rec.GetTableName() @@ -216,12 +226,12 @@ func (p *PostgresCDCSource) consumeStream( TableName: tableName, PkeyColVal: pkeyColVal, } - _, ok := result.TablePKeyLastSeen[tablePkeyVal] + _, ok := records.TablePKeyLastSeen[tablePkeyVal] if !ok { - result.Records = append(result.Records, rec) - result.TablePKeyLastSeen[tablePkeyVal] = len(result.Records) - 1 + records.Records = append(records.Records, rec) + records.TablePKeyLastSeen[tablePkeyVal] = len(records.Records) - 1 } else { - oldRec := result.Records[result.TablePKeyLastSeen[tablePkeyVal]] + oldRec := records.Records[records.TablePKeyLastSeen[tablePkeyVal]] // iterate through unchanged toast cols and set them for col, val := range oldRec.GetItems() { if _, ok := r.NewItems[col]; !ok { @@ -229,8 +239,8 @@ func (p *PostgresCDCSource) consumeStream( delete(r.UnchangedToastColumns, col) } } - result.Records = append(result.Records, rec) - result.TablePKeyLastSeen[tablePkeyVal] = len(result.Records) - 1 + records.Records = append(records.Records, rec) + records.TablePKeyLastSeen[tablePkeyVal] = len(records.Records) - 1 } case *model.InsertRecord: pkeyCol := req.TableNameSchemaMapping[tableName].PrimaryKeyColumn @@ -239,18 +249,27 @@ func (p *PostgresCDCSource) consumeStream( TableName: tableName, PkeyColVal: pkeyColVal, } - result.Records = append(result.Records, rec) + records.Records = append(records.Records, rec) // all columns will be set in insert record, so add it to the map - result.TablePKeyLastSeen[tablePkeyVal] = len(result.Records) - 1 + records.TablePKeyLastSeen[tablePkeyVal] = len(records.Records) - 1 case *model.DeleteRecord: - result.Records = append(result.Records, rec) + records.Records = append(records.Records, rec) + case *model.RelationRecord: + tableSchemaDelta := rec.(*model.RelationRecord).TableSchemaDelta + if len(tableSchemaDelta.AddedColumns) > 0 || len(tableSchemaDelta.DroppedColumns) > 0 { + result.TableSchemaDelta = tableSchemaDelta + log.Infof("Detected schema change for table %s, returning currently accumulated records", + result.TableSchemaDelta.SrcTableName) + earlyReturn = true + } } } currentPos := xld.WALStart + pglogrepl.LSN(len(xld.WALData)) - result.LastCheckPointID = int64(currentPos) + records.LastCheckPointID = int64(currentPos) - if result.Records != nil && len(result.Records) == int(req.MaxBatchSize) { + if records.Records != nil && + ((len(records.Records) == int(req.MaxBatchSize)) || earlyReturn) { return result, nil } } @@ -281,7 +300,12 @@ func (p *PostgresCDCSource) processMessage(batch *model.RecordBatch, xld pglogre // the state of the relation message somewhere log.Infof("RelationMessage => RelationID: %d, Namespace: %s, RelationName: %s, Columns: %v", msg.RelationID, msg.Namespace, msg.RelationName, msg.Columns) - p.relations[msg.RelationID] = msg + if p.relationMessageMapping[msg.RelationID] == nil { + p.relationMessageMapping[msg.RelationID] = convertRelationMessageToProto(msg) + } else { + return p.processRelationMessage(xld.WALStart, convertRelationMessageToProto(msg)) + } + case *pglogrepl.TruncateMessage: log.Warnf("TruncateMessage not supported") default: @@ -304,7 +328,7 @@ func (p *PostgresCDCSource) processInsertMessage( // log lsn and relation id for debugging log.Debugf("InsertMessage => LSN: %d, RelationID: %d, Relation Name: %s", lsn, msg.RelationID, tableName) - rel, ok := p.relations[msg.RelationID] + rel, ok := p.relationMessageMapping[msg.RelationID] if !ok { return nil, fmt.Errorf("unknown relation id: %d", msg.RelationID) } @@ -337,7 +361,7 @@ func (p *PostgresCDCSource) processUpdateMessage( // log lsn and relation id for debugging log.Debugf("UpdateMessage => LSN: %d, RelationID: %d, Relation Name: %s", lsn, msg.RelationID, tableName) - rel, ok := p.relations[msg.RelationID] + rel, ok := p.relationMessageMapping[msg.RelationID] if !ok { return nil, fmt.Errorf("unknown relation id: %d", msg.RelationID) } @@ -376,7 +400,7 @@ func (p *PostgresCDCSource) processDeleteMessage( // log lsn and relation id for debugging log.Debugf("DeleteMessage => LSN: %d, RelationID: %d, Relation Name: %s", lsn, msg.RelationID, tableName) - rel, ok := p.relations[msg.RelationID] + rel, ok := p.relationMessageMapping[msg.RelationID] if !ok { return nil, fmt.Errorf("unknown relation id: %d", msg.RelationID) } @@ -405,7 +429,7 @@ It takes a tuple and a relation message as input and returns */ func (p *PostgresCDCSource) convertTupleToMap( tuple *pglogrepl.TupleData, - rel *pglogrepl.RelationMessage, + rel *protos.RelationMessage, ) (model.RecordItems, map[string]bool, error) { // if the tuple is nil, return an empty map if tuple == nil { @@ -470,3 +494,67 @@ func (p *PostgresCDCSource) decodeColumnData(data []byte, dataType uint32, forma } return &qvalue.QValue{Kind: qvalue.QValueKindString, Value: string(data)}, nil } + +func convertRelationMessageToProto(msg *pglogrepl.RelationMessage) *protos.RelationMessage { + protoColArray := make([]*protos.RelationMessageColumn, 0) + for _, column := range msg.Columns { + protoColArray = append(protoColArray, &protos.RelationMessageColumn{ + Name: column.Name, + Flags: uint32(column.Flags), + DataType: column.DataType, + }) + } + return &protos.RelationMessage{ + RelationId: msg.RelationID, + RelationName: msg.RelationName, + Columns: protoColArray, + } +} + +// processRelationMessage processes a delete message and returns a TableSchemaDelta +func (p *PostgresCDCSource) processRelationMessage( + lsn pglogrepl.LSN, + currRel *protos.RelationMessage, +) (model.Record, error) { + // retrieve initial RelationMessage for table changed. + prevRel := p.relationMessageMapping[currRel.RelationId] + // creating maps for lookup later + prevRelMap := make(map[string]bool) + currRelMap := make(map[string]bool) + for _, column := range prevRel.Columns { + prevRelMap[column.Name] = true + } + for _, column := range currRel.Columns { + currRelMap[column.Name] = true + } + + schemaDelta := &protos.TableSchemaDelta{ + // set it to the source table for now, so we can update the schema on the source side + // then at the Workflow level we set it t + SrcTableName: p.SrcTableIDNameMapping[currRel.RelationId], + DstTableName: p.TableNameMapping[p.SrcTableIDNameMapping[currRel.RelationId]], + AddedColumns: make([]*protos.DeltaAddedColumn, 0), + DroppedColumns: make([]string, 0), + } + for _, column := range currRel.Columns { + // not present in previous relation message, but in current one, so added. + if !prevRelMap[column.Name] { + schemaDelta.AddedColumns = append(schemaDelta.AddedColumns, &protos.DeltaAddedColumn{ + ColumnName: column.Name, + ColumnType: string(postgresOIDToQValueKind(column.DataType)), + }) + } + } + for _, column := range prevRel.Columns { + // present in previous relation message, but not in current one, so dropped. + if !currRelMap[column.Name] { + schemaDelta.DroppedColumns = append(schemaDelta.DroppedColumns, column.Name) + } + } + + p.relationMessageMapping[currRel.RelationId] = currRel + return &model.RelationRecord{ + TableSchemaDelta: schemaDelta, + CheckPointID: int64(lsn), + }, nil +} diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index 00b82705b7..078a912ce5 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -173,7 +173,7 @@ func (c *PostgresConnector) GetLastOffset(jobName string) (*protos.LastSyncState } // PullRecords pulls records from the source. -func (c *PostgresConnector) PullRecords(req *model.PullRecordsRequest) (*model.RecordBatch, error) { +func (c *PostgresConnector) PullRecords(req *model.PullRecordsRequest) (*model.RecordsWithTableSchemaDelta, error) { // Slotname would be the job name prefixed with "peerflow_slot_" slotName := fmt.Sprintf("peerflow_slot_%s", req.FlowJobName) if req.OverrideReplicationSlotName != "" { @@ -211,27 +211,28 @@ func (c *PostgresConnector) PullRecords(req *model.PullRecordsRequest) (*model.R }).Infof("PullRecords: performed checks for slot and publication") cdc, err := NewPostgresCDCSource(&PostgresCDCConfig{ - AppContext: c.ctx, - Connection: c.replPool, - SrcTableIDNameMapping: req.SrcTableIDNameMapping, - Slot: slotName, - Publication: publicationName, - TableNameMapping: req.TableNameMapping, + AppContext: c.ctx, + Connection: c.replPool, + SrcTableIDNameMapping: req.SrcTableIDNameMapping, + Slot: slotName, + Publication: publicationName, + TableNameMapping: req.TableNameMapping, + RelationMessageMapping: req.RelationMessageMapping, }) if err != nil { return nil, fmt.Errorf("failed to create cdc source: %w", err) } - recordBatch, err := cdc.PullRecords(req) + recordsWithSchemaDelta, err := cdc.PullRecords(req) if err != nil { return nil, err } - if len(recordBatch.Records) > 0 { + if len(recordsWithSchemaDelta.RecordBatch.Records) > 0 { totalRecordsAtSource, err := c.getApproxTableCounts(maps.Keys(req.TableNameMapping)) if err != nil { return nil, err } - metrics.LogPullMetrics(c.ctx, req.FlowJobName, recordBatch, totalRecordsAtSource) + metrics.LogPullMetrics(c.ctx, req.FlowJobName, recordsWithSchemaDelta.RecordBatch, totalRecordsAtSource) cdcMirrorMonitor, ok := c.ctx.Value(shared.CDCMirrorMonitorKey).(*monitoring.CatalogMirrorMonitor) if ok { latestLSN, err := c.getCurrentLSN() @@ -245,7 +246,7 @@ func (c *PostgresConnector) PullRecords(req *model.PullRecordsRequest) (*model.R } } - return recordBatch, nil + return recordsWithSchemaDelta, nil } // SyncRecords pushes records to the destination. @@ -654,6 +655,64 @@ func (c *PostgresConnector) InitializeTableSchema(req map[string]*protos.TableSc return nil } +// ReplayTableSchemaDelta changes a destination table to match the schema at source +// This could involve adding or dropping multiple columns. +func (c *PostgresConnector) ReplayTableSchemaDelta(flowJobName string, schemaDelta *protos.TableSchemaDelta) error { + if (schemaDelta == nil) || (len(schemaDelta.AddedColumns) == 0 && len(schemaDelta.DroppedColumns) == 0) { + return nil + } + + // Postgres is cool and supports transactional DDL. So we use a transaction. + tableSchemaModifyTx, err := c.pool.Begin(c.ctx) + if err != nil { + return fmt.Errorf("error starting transaction for schema modification for table %s: %w", + schemaDelta.SrcTableName, err) + } + defer func() { + deferErr := tableSchemaModifyTx.Rollback(c.ctx) + if deferErr != pgx.ErrTxClosed && deferErr != nil { + log.WithFields(log.Fields{ + "flowName": flowJobName, + }).Errorf("unexpected error rolling back transaction for table schema modification: %v", err) + } + }() + + for _, droppedColumn := range schemaDelta.DroppedColumns { + _, err = tableSchemaModifyTx.Exec(c.ctx, fmt.Sprintf("ALTER TABLE %s DROP COLUMN %s", schemaDelta.DstTableName, + droppedColumn)) + if err != nil { + return fmt.Errorf("failed to drop column %s for table %s: %w", droppedColumn, + schemaDelta.SrcTableName, err) + } + log.WithFields(log.Fields{ + "flowName": flowJobName, + "tableName": schemaDelta.SrcTableName, + }).Infof("[schema delta replay] dropped column %s", droppedColumn) + } + for _, addedColumn := range schemaDelta.AddedColumns { + _, err = tableSchemaModifyTx.Exec(c.ctx, fmt.Sprintf("ALTER TABLE %s ADD COLUMN %s %s", + schemaDelta.DstTableName, addedColumn.ColumnName, + qValueKindToPostgresType(addedColumn.ColumnType))) + if err != nil { + return fmt.Errorf("failed to add column %s for table %s: %w", addedColumn.ColumnName, + schemaDelta.SrcTableName, err) + } + log.WithFields(log.Fields{ + "flowName": flowJobName, + "tableName": schemaDelta.SrcTableName, + }).Infof("[schema delta replay] added column %s with data type %s", + addedColumn.ColumnName, addedColumn.ColumnType) + } + + err = tableSchemaModifyTx.Commit(c.ctx) + if err != nil { + return fmt.Errorf("failed to commit transaction for table schema modification for table %s: %w", + schemaDelta.SrcTableName, err) + } + + return nil +} + // EnsurePullability ensures that a table is pullable, implementing the Connector interface. func (c *PostgresConnector) EnsurePullability(req *protos.EnsurePullabilityBatchInput, ) (*protos.EnsurePullabilityBatchOutput, error) { diff --git a/flow/connectors/postgres/postgres_cdc_test.go b/flow/connectors/postgres/postgres_cdc_test.go index ecaf46d39c..727ffa133a 100644 --- a/flow/connectors/postgres/postgres_cdc_test.go +++ b/flow/connectors/postgres/postgres_cdc_test.go @@ -344,6 +344,7 @@ func (suite *PostgresCDCTestSuite) TestErrorForTableNotExist() { tableNameMapping := map[string]string{ nonExistentFlowSrcTableName: nonExistentFlowDstTableName, } + relationMessageMapping := make(model.RelationMessageMapping) getTblSchemaInput := &protos.GetTableSchemaBatchInput{ TableIdentifiers: []string{nonExistentFlowSrcTableName}, @@ -389,7 +390,7 @@ func (suite *PostgresCDCTestSuite) TestErrorForTableNotExist() { }) suite.failTestError(err) suite.dropTable(nonExistentFlowSrcTableName) - records, err := suite.connector.PullRecords(&model.PullRecordsRequest{ + recordsWithSchemaDelta, err := suite.connector.PullRecords(&model.PullRecordsRequest{ FlowJobName: nonExistentFlowName, LastSyncState: nil, IdleTimeout: 5 * time.Second, @@ -397,8 +398,10 @@ func (suite *PostgresCDCTestSuite) TestErrorForTableNotExist() { SrcTableIDNameMapping: relIDTableNameMapping, TableNameMapping: tableNameMapping, TableNameSchemaMapping: tableNameSchemaMapping, + RelationMessageMapping: relationMessageMapping, }) - suite.Equal(0, len(records.Records)) + suite.Equal(0, len(recordsWithSchemaDelta.RecordBatch.Records)) + suite.Nil(recordsWithSchemaDelta.TableSchemaDelta) suite.Nil(err) err = suite.connector.PullFlowCleanup(nonExistentFlowName) @@ -429,6 +432,8 @@ func (suite *PostgresCDCTestSuite) TestSimpleHappyFlow() { tableNameMapping := map[string]string{ simpleHappyFlowSrcTableName: simpleHappyFlowDstTableName, } + relationMessageMapping := make(model.RelationMessageMapping) + err = suite.connector.SetupReplication(nil, &protos.SetupReplicationInput{ FlowJobName: simpleHappyFlowName, TableNameMapping: tableNameMapping, @@ -458,8 +463,8 @@ func (suite *PostgresCDCTestSuite) TestSimpleHappyFlow() { tableNameSchemaMapping[simpleHappyFlowDstTableName] = tableNameSchema.TableNameSchemaMapping[simpleHappyFlowSrcTableName] - // pulling with no records. - records, err := suite.connector.PullRecords(&model.PullRecordsRequest{ + // pulling with no recordsWithSchemaDelta. + recordsWithSchemaDelta, err := suite.connector.PullRecords(&model.PullRecordsRequest{ FlowJobName: simpleHappyFlowName, LastSyncState: nil, IdleTimeout: 5 * time.Second, @@ -467,15 +472,18 @@ func (suite *PostgresCDCTestSuite) TestSimpleHappyFlow() { SrcTableIDNameMapping: relIDTableNameMapping, TableNameMapping: tableNameMapping, TableNameSchemaMapping: tableNameSchemaMapping, + RelationMessageMapping: relationMessageMapping, }) suite.failTestError(err) - suite.Equal(0, len(records.Records)) - suite.Equal(int64(0), records.FirstCheckPointID) - suite.Equal(int64(0), records.LastCheckPointID) + suite.Equal(0, len(recordsWithSchemaDelta.RecordBatch.Records)) + suite.Nil(recordsWithSchemaDelta.TableSchemaDelta) + suite.Equal(int64(0), recordsWithSchemaDelta.RecordBatch.FirstCheckPointID) + suite.Equal(int64(0), recordsWithSchemaDelta.RecordBatch.LastCheckPointID) + relationMessageMapping = recordsWithSchemaDelta.RelationMessageMapping // pulling after inserting records. suite.insertSimpleRecords(simpleHappyFlowSrcTableName) - records, err = suite.connector.PullRecords(&model.PullRecordsRequest{ + recordsWithSchemaDelta, err = suite.connector.PullRecords(&model.PullRecordsRequest{ FlowJobName: simpleHappyFlowName, LastSyncState: nil, IdleTimeout: 5 * time.Second, @@ -483,19 +491,24 @@ func (suite *PostgresCDCTestSuite) TestSimpleHappyFlow() { SrcTableIDNameMapping: relIDTableNameMapping, TableNameMapping: tableNameMapping, TableNameSchemaMapping: tableNameSchemaMapping, + RelationMessageMapping: relationMessageMapping, }) suite.failTestError(err) - suite.validateInsertedSimpleRecords(records.Records, simpleHappyFlowSrcTableName, simpleHappyFlowDstTableName) - suite.Greater(records.FirstCheckPointID, int64(0)) - suite.GreaterOrEqual(records.LastCheckPointID, records.FirstCheckPointID) - currentCheckPointID := records.LastCheckPointID + suite.Nil(recordsWithSchemaDelta.TableSchemaDelta) + suite.validateInsertedSimpleRecords(recordsWithSchemaDelta.RecordBatch.Records, + simpleHappyFlowSrcTableName, simpleHappyFlowDstTableName) + suite.Greater(recordsWithSchemaDelta.RecordBatch.FirstCheckPointID, int64(0)) + suite.GreaterOrEqual(recordsWithSchemaDelta.RecordBatch.LastCheckPointID, + recordsWithSchemaDelta.RecordBatch.FirstCheckPointID) + currentCheckPointID := recordsWithSchemaDelta.RecordBatch.LastCheckPointID + relationMessageMapping = recordsWithSchemaDelta.RelationMessageMapping // pulling after mutating records. suite.mutateSimpleRecords(simpleHappyFlowSrcTableName) - records, err = suite.connector.PullRecords(&model.PullRecordsRequest{ + recordsWithSchemaDelta, err = suite.connector.PullRecords(&model.PullRecordsRequest{ FlowJobName: simpleHappyFlowName, LastSyncState: &protos.LastSyncState{ - Checkpoint: records.LastCheckPointID, + Checkpoint: recordsWithSchemaDelta.RecordBatch.LastCheckPointID, LastSyncedAt: nil, }, IdleTimeout: 5 * time.Second, @@ -503,11 +516,15 @@ func (suite *PostgresCDCTestSuite) TestSimpleHappyFlow() { SrcTableIDNameMapping: relIDTableNameMapping, TableNameMapping: tableNameMapping, TableNameSchemaMapping: tableNameSchemaMapping, + RelationMessageMapping: relationMessageMapping, }) suite.failTestError(err) - suite.validateSimpleMutatedRecords(records.Records, simpleHappyFlowSrcTableName, simpleHappyFlowDstTableName) - suite.GreaterOrEqual(records.FirstCheckPointID, currentCheckPointID) - suite.GreaterOrEqual(records.LastCheckPointID, records.FirstCheckPointID) + suite.Nil(recordsWithSchemaDelta.TableSchemaDelta) + suite.validateSimpleMutatedRecords(recordsWithSchemaDelta.RecordBatch.Records, + simpleHappyFlowSrcTableName, simpleHappyFlowDstTableName) + suite.GreaterOrEqual(recordsWithSchemaDelta.RecordBatch.FirstCheckPointID, currentCheckPointID) + suite.GreaterOrEqual(recordsWithSchemaDelta.RecordBatch.LastCheckPointID, + recordsWithSchemaDelta.RecordBatch.FirstCheckPointID) err = suite.connector.PullFlowCleanup(simpleHappyFlowName) suite.failTestError(err) @@ -538,6 +555,7 @@ func (suite *PostgresCDCTestSuite) TestAllTypesHappyFlow() { suite.failTestError(err) tableRelID := ensurePullabilityOutput.TableIdentifierMapping[allTypesHappyFlowSrcTableName]. GetPostgresTableIdentifier().RelId + relationMessageMapping := make(model.RelationMessageMapping) relIDTableNameMapping := map[uint32]string{ tableRelID: allTypesHappyFlowSrcTableName, @@ -627,10 +645,11 @@ func (suite *PostgresCDCTestSuite) TestAllTypesHappyFlow() { SrcTableIDNameMapping: relIDTableNameMapping, TableNameMapping: tableNameMapping, TableNameSchemaMapping: tableNameSchemaMapping, + RelationMessageMapping: relationMessageMapping, }) suite.failTestError(err) - suite.Equal(1, len(records.Records)) - suite.Equal(35, len(records.Records[0].GetItems())) + suite.Equal(1, len(records.RecordBatch.Records)) + suite.Equal(35, len(records.RecordBatch.Records[0].GetItems())) err = suite.connector.PullFlowCleanup(allTypesHappyFlowName) suite.failTestError(err) @@ -663,6 +682,8 @@ func (suite *PostgresCDCTestSuite) TestToastHappyFlow() { tableNameMapping := map[string]string{ toastHappyFlowSrcTableName: toastHappyFlowDstTableName, } + relationMessageMapping := make(model.RelationMessageMapping) + err = suite.connector.SetupReplication(nil, &protos.SetupReplicationInput{ FlowJobName: toastHappyFlowName, TableNameMapping: tableNameMapping, @@ -695,7 +716,7 @@ func (suite *PostgresCDCTestSuite) TestToastHappyFlow() { tableNameSchema.TableNameSchemaMapping[toastHappyFlowSrcTableName] suite.insertToastRecords(toastHappyFlowSrcTableName) - records, err := suite.connector.PullRecords(&model.PullRecordsRequest{ + recordsWithSchemaDelta, err := suite.connector.PullRecords(&model.PullRecordsRequest{ FlowJobName: toastHappyFlowName, LastSyncState: nil, IdleTimeout: 10 * time.Second, @@ -703,17 +724,22 @@ func (suite *PostgresCDCTestSuite) TestToastHappyFlow() { SrcTableIDNameMapping: relIDTableNameMapping, TableNameMapping: tableNameMapping, TableNameSchemaMapping: tableNameSchemaMapping, + RelationMessageMapping: relationMessageMapping, }) suite.failTestError(err) - suite.validateInsertedToastRecords(records.Records, toastHappyFlowSrcTableName, toastHappyFlowDstTableName) - suite.Greater(records.FirstCheckPointID, int64(0)) - suite.GreaterOrEqual(records.LastCheckPointID, records.FirstCheckPointID) + suite.Nil(recordsWithSchemaDelta.TableSchemaDelta) + suite.validateInsertedToastRecords(recordsWithSchemaDelta.RecordBatch.Records, + toastHappyFlowSrcTableName, toastHappyFlowDstTableName) + suite.Greater(recordsWithSchemaDelta.RecordBatch.FirstCheckPointID, int64(0)) + suite.GreaterOrEqual(recordsWithSchemaDelta.RecordBatch.LastCheckPointID, + recordsWithSchemaDelta.RecordBatch.FirstCheckPointID) + relationMessageMapping = recordsWithSchemaDelta.RelationMessageMapping suite.mutateToastRecords(toastHappyFlowSrcTableName) - records, err = suite.connector.PullRecords(&model.PullRecordsRequest{ + recordsWithSchemaDelta, err = suite.connector.PullRecords(&model.PullRecordsRequest{ FlowJobName: toastHappyFlowName, LastSyncState: &protos.LastSyncState{ - Checkpoint: records.LastCheckPointID, + Checkpoint: recordsWithSchemaDelta.RecordBatch.LastCheckPointID, LastSyncedAt: nil, }, IdleTimeout: 10 * time.Second, @@ -721,9 +747,11 @@ func (suite *PostgresCDCTestSuite) TestToastHappyFlow() { SrcTableIDNameMapping: relIDTableNameMapping, TableNameMapping: tableNameMapping, TableNameSchemaMapping: tableNameSchemaMapping, + RelationMessageMapping: relationMessageMapping, }) suite.failTestError(err) - suite.validateMutatedToastRecords(records.Records, toastHappyFlowSrcTableName, toastHappyFlowDstTableName) + suite.validateMutatedToastRecords(recordsWithSchemaDelta.RecordBatch.Records, toastHappyFlowSrcTableName, + toastHappyFlowDstTableName) err = suite.connector.PullFlowCleanup(toastHappyFlowName) suite.failTestError(err) diff --git a/flow/connectors/postgres/postgres_repl_test.go b/flow/connectors/postgres/postgres_repl_test.go index 933273174c..11ef7e2be8 100644 --- a/flow/connectors/postgres/postgres_repl_test.go +++ b/flow/connectors/postgres/postgres_repl_test.go @@ -72,7 +72,7 @@ func (suite *PostgresReplicationSnapshotTestSuite) TearDownSuite() { // Fetch all the publications rows, err := teardownTx.Query(context.Background(), - "SELECT pubname FROM pg_publication WHERE pubname LIKE 'peerflow_pub%'") + "SELECT pubname FROM pg_publication WHERE pubname LIKE $1", fmt.Sprintf("%%%s", "test_simple_slot_creation")) require.NoError(suite.T(), err) // Iterate over the publications and drop them @@ -87,7 +87,8 @@ func (suite *PostgresReplicationSnapshotTestSuite) TearDownSuite() { } _, err = teardownTx.Exec(context.Background(), - "SELECT pg_drop_replication_slot(slot_name) FROM pg_replication_slots") + "SELECT pg_drop_replication_slot(slot_name) FROM pg_replication_slots WHERE slot_name LIKE $1", + fmt.Sprintf("%%%s", "test_simple_slot_creation")) require.NoError(suite.T(), err) err = teardownTx.Commit(context.Background()) diff --git a/flow/connectors/s3/s3.go b/flow/connectors/s3/s3.go index 742bd28981..39bf5297ab 100644 --- a/flow/connectors/s3/s3.go +++ b/flow/connectors/s3/s3.go @@ -82,7 +82,12 @@ func (c *S3Connector) InitializeTableSchema(req map[string]*protos.TableSchema) return fmt.Errorf("cdc based replication is not currently supported for S3 target") } -func (c *S3Connector) PullRecords(req *model.PullRecordsRequest) (*model.RecordBatch, error) { +func (c *S3Connector) ReplayTableSchemaDelta(flowJobName string, schemaDelta *protos.TableSchemaDelta) error { + log.Warnf("ReplayTableSchemaDelta is a no-op for S3 flow connector") + return nil +} + +func (c *S3Connector) PullRecords(req *model.PullRecordsRequest) (*model.RecordsWithTableSchemaDelta, error) { log.Errorf("panicking at call to PullRecords for S3 flow connector") panic("PullRecords is not implemented for the S3 flow connector") } diff --git a/flow/connectors/snowflake/snowflake.go b/flow/connectors/snowflake/snowflake.go index 7852eedc9c..b2aa542e75 100644 --- a/flow/connectors/snowflake/snowflake.go +++ b/flow/connectors/snowflake/snowflake.go @@ -364,7 +364,64 @@ func (c *SnowflakeConnector) InitializeTableSchema(req map[string]*protos.TableS return nil } -func (c *SnowflakeConnector) PullRecords(req *model.PullRecordsRequest) (*model.RecordBatch, error) { +// ReplayTableSchemaDelta changes a destination table to match the schema at source +// This could involve adding or dropping multiple columns. +func (c *SnowflakeConnector) ReplayTableSchemaDelta(flowJobName string, schemaDelta *protos.TableSchemaDelta) error { + if (schemaDelta == nil) || (len(schemaDelta.AddedColumns) == 0 && len(schemaDelta.DroppedColumns) == 0) { + return nil + } + + // Postgres is cool and supports transactional DDL. So we use a transaction. + tableSchemaModifyTx, err := c.database.Begin() + if err != nil { + return fmt.Errorf("error starting transaction for schema modification for table %s: %w", + schemaDelta.SrcTableName, err) + } + defer func() { + deferErr := tableSchemaModifyTx.Rollback() + if deferErr != sql.ErrTxDone && deferErr != nil { + log.WithFields(log.Fields{ + "flowName": flowJobName, + }).Errorf("unexpected error rolling back transaction for table schema modification: %v", err) + } + }() + + for _, droppedColumn := range schemaDelta.DroppedColumns { + _, err = tableSchemaModifyTx.Exec(fmt.Sprintf("ALTER TABLE %s DROP COLUMN %s", schemaDelta.DstTableName, + droppedColumn)) + if err != nil { + return fmt.Errorf("failed to drop column %s for table %s: %w", droppedColumn, + schemaDelta.SrcTableName, err) + } + log.WithFields(log.Fields{ + "flowName": flowJobName, + "tableName": schemaDelta.SrcTableName, + }).Infof("[schema delta replay] dropped column %s", droppedColumn) + } + for _, addedColumn := range schemaDelta.AddedColumns { + _, err = tableSchemaModifyTx.Exec(fmt.Sprintf("ALTER TABLE %s ADD COLUMN %s %s", schemaDelta.DstTableName, + addedColumn.ColumnName, qValueKindToSnowflakeType(qvalue.QValueKind(addedColumn.ColumnType)))) + if err != nil { + return fmt.Errorf("failed to add column %s for table %s: %w", addedColumn.ColumnName, + schemaDelta.SrcTableName, err) + } + log.WithFields(log.Fields{ + "flowName": flowJobName, + "tableName": schemaDelta.SrcTableName, + }).Infof("[schema delta replay] added column %s with data type %s", addedColumn.ColumnName, + addedColumn.ColumnType) + } + + err = tableSchemaModifyTx.Commit() + if err != nil { + return fmt.Errorf("failed to commit transaction for table schema modification for table %s: %w", + schemaDelta.SrcTableName, err) + } + + return nil +} + +func (c *SnowflakeConnector) PullRecords(req *model.PullRecordsRequest) (*model.RecordsWithTableSchemaDelta, error) { log.Errorf("panicking at call to PullRecords for Snowflake flow connector") panic("PullRecords is not implemented for the Snowflake flow connector") } diff --git a/flow/connectors/sqlserver/sqlserver.go b/flow/connectors/sqlserver/sqlserver.go index a6f1d70b52..feb15e8a47 100644 --- a/flow/connectors/sqlserver/sqlserver.go +++ b/flow/connectors/sqlserver/sqlserver.go @@ -105,7 +105,12 @@ func (c *SQLServerConnector) InitializeTableSchema(req map[string]*protos.TableS return fmt.Errorf("cdc based replication is not currently supported for SQLServer target") } -func (c *SQLServerConnector) PullRecords(req *model.PullRecordsRequest) (*model.RecordBatch, error) { +func (c *SQLServerConnector) ReplayTableSchemaDelta(flowJobName string, schemaDelta *protos.TableSchemaDelta) error { + log.Warnf("ReplayTableSchemaDelta is a no-op for SQLServer flow connector") + return nil +} + +func (c *SQLServerConnector) PullRecords(req *model.PullRecordsRequest) (*model.RecordsWithTableSchemaDelta, error) { log.Errorf("panicking at call to PullRecords for SQLServer flow connector") panic("PullRecords is not implemented for the SQLServer flow connector") } diff --git a/flow/e2e/bigquery/bigquery_helper.go b/flow/e2e/bigquery/bigquery_helper.go index 72e8b3edea..d17fe7a668 100644 --- a/flow/e2e/bigquery/bigquery_helper.go +++ b/flow/e2e/bigquery/bigquery_helper.go @@ -169,8 +169,8 @@ func (b *BigQueryTestHelper) RunCommand(command string) error { return nil } -// CountRows(tableName) returns the number of rows in the given table. -func (b *BigQueryTestHelper) CountRows(tableName string) (int, error) { +// countRows(tableName) returns the number of rows in the given table. +func (b *BigQueryTestHelper) countRows(tableName string) (int, error) { command := fmt.Sprintf("SELECT COUNT(*) FROM `%s.%s`", b.Config.DatasetId, tableName) it, err := b.client.Query(command).Read(context.Background()) if err != nil { diff --git a/flow/e2e/bigquery/peer_flow_bq_test.go b/flow/e2e/bigquery/peer_flow_bq_test.go index 655a317854..72acb12230 100644 --- a/flow/e2e/bigquery/peer_flow_bq_test.go +++ b/flow/e2e/bigquery/peer_flow_bq_test.go @@ -261,7 +261,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Complete_Simple_Flow_BQ() { s.Error(err) s.Contains(err.Error(), "continue as new") - count, err := s.bqHelper.CountRows(dstTableName) + count, err := s.bqHelper.countRows(dstTableName) s.NoError(err) s.Equal(10, count) @@ -854,7 +854,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Simple_Flow_BQ_Avro_CDC() { s.Error(err) s.Contains(err.Error(), "continue as new") - count, err := s.bqHelper.CountRows(dstTableName) + count, err := s.bqHelper.countRows(dstTableName) s.NoError(err) s.Equal(10, count) @@ -913,9 +913,9 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Multi_Table_BQ() { require.True(s.T(), env.IsWorkflowCompleted()) err = env.GetWorkflowError() - count1, err := s.bqHelper.CountRows(dstTable1Name) + count1, err := s.bqHelper.countRows(dstTable1Name) s.NoError(err) - count2, err := s.bqHelper.CountRows(dstTable2Name) + count2, err := s.bqHelper.countRows(dstTable2Name) s.NoError(err) s.Equal(1, count1) @@ -923,3 +923,107 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Multi_Table_BQ() { env.AssertExpectations(s.T()) } + +func (s *PeerFlowE2ETestSuiteBQ) Test_Simple_Schema_Changes_BQ() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_simple_schema_changes") + dstTableName := "test_simple_schema_changes" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id BIGINT PRIMARY KEY GENERATED ALWAYS AS IDENTITY, + c1 BIGINT + ); + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_simple_schema_changes"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.bqHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 10, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and then insert and mutate schema repeatedly. + go func() { + // insert first row. + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1) VALUES ($1)`, srcTableName), 1) + s.NoError(err) + fmt.Println("Inserted initial row in the source table") + + // verify we got our first row. + e2e.NormalizeFlowCountQuery(env, connectionGen, 2) + s.compareTableSchemasBQ("test_simple_schema_changes") + s.compareTableContentsBQ("test_simple_schema_changes", "id,c1") + + // alter source table, add column c2 and insert another row. + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + ALTER TABLE %s ADD COLUMN c2 BIGINT`, srcTableName)) + s.NoError(err) + fmt.Println("Altered source table, added column c2") + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1,c2) VALUES ($1,$2)`, srcTableName), 2, 2) + s.NoError(err) + fmt.Println("Inserted row with added c2 in the source table") + + // verify we got our two rows, if schema did not match up it will error. + e2e.NormalizeFlowCountQuery(env, connectionGen, 4) + s.compareTableSchemasBQ("test_simple_schema_changes") + s.compareTableContentsBQ("test_simple_schema_changes", "id,c1,c2") + + // alter source table, add column c3, drop column c2 and insert another row. + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + ALTER TABLE %s DROP COLUMN c2, ADD COLUMN c3 BIGINT`, srcTableName)) + s.NoError(err) + fmt.Println("Altered source table, dropped column c2 and added column c3") + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1,c3) VALUES ($1,$2)`, srcTableName), 3, 3) + s.NoError(err) + fmt.Println("Inserted row with added c3 in the source table") + + // verify we got our two rows, if schema did not match up it will error. + e2e.NormalizeFlowCountQuery(env, connectionGen, 6) + s.compareTableSchemasBQ("test_simple_schema_changes") + s.compareTableContentsBQ("test_simple_schema_changes", "id,c1,c3") + + // alter source table, drop column c3 and insert another row. + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + ALTER TABLE %s DROP COLUMN c3`, srcTableName)) + s.NoError(err) + fmt.Println("Altered source table, dropped column c3") + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1) VALUES ($1)`, srcTableName), 4) + s.NoError(err) + fmt.Println("Inserted row after dropping all columns in the source table") + + // verify we got our two rows, if schema did not match up it will error. + e2e.NormalizeFlowCountQuery(env, connectionGen, 8) + s.compareTableSchemasBQ("test_simple_schema_changes") + s.compareTableContentsBQ("test_simple_schema_changes", "id,c1") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + env.AssertExpectations(s.T()) +} diff --git a/flow/e2e/bigquery/qrep_flow_bq_test.go b/flow/e2e/bigquery/qrep_flow_bq_test.go index 5e6374cc1a..fb0fc3b876 100644 --- a/flow/e2e/bigquery/qrep_flow_bq_test.go +++ b/flow/e2e/bigquery/qrep_flow_bq_test.go @@ -3,6 +3,8 @@ package e2e_bigquery import ( "context" "fmt" + "sort" + "strings" connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" "github.com/PeerDB-io/peer-flow/e2e" @@ -27,6 +29,32 @@ func (s *PeerFlowE2ETestSuiteBQ) setupBQDestinationTable(dstTable string) { fmt.Printf("created table on bigquery: %s.%s. %v\n", s.bqHelper.Config.DatasetId, dstTable, err) } +func (s *PeerFlowE2ETestSuiteBQ) compareTableSchemasBQ(tableName string) { + // read rows from source table + pgQueryExecutor := connpostgres.NewQRepQueryExecutor(s.pool, context.Background(), "testflow", "testpart") + pgQueryExecutor.SetTestEnv(true) + + pgRows, err := pgQueryExecutor.ExecuteAndProcessQuery( + fmt.Sprintf("SELECT * FROM e2e_test_%s.%s ORDER BY id", bigquerySuffix, tableName), + ) + s.NoError(err) + sort.Slice(pgRows.Schema.Fields, func(i int, j int) bool { + return strings.Compare(pgRows.Schema.Fields[i].Name, pgRows.Schema.Fields[j].Name) == -1 + }) + + // read rows from destination table + qualifiedTableName := fmt.Sprintf("`%s.%s`", s.bqHelper.Config.DatasetId, tableName) + bqRows, err := s.bqHelper.ExecuteAndProcessQuery( + fmt.Sprintf("SELECT * FROM %s ORDER BY id", qualifiedTableName), + ) + s.NoError(err) + sort.Slice(bqRows.Schema.Fields, func(i int, j int) bool { + return strings.Compare(bqRows.Schema.Fields[i].Name, bqRows.Schema.Fields[j].Name) == -1 + }) + + s.True(pgRows.Schema.EqualNames(bqRows.Schema), "schemas from source and destination tables are not equal") +} + func (s *PeerFlowE2ETestSuiteBQ) compareTableContentsBQ(tableName string, colsString string) { // read rows from source table pgQueryExecutor := connpostgres.NewQRepQueryExecutor(s.pool, context.Background(), "testflow", "testpart") diff --git a/flow/e2e/postgres/peer_flow_pg_test.go b/flow/e2e/postgres/peer_flow_pg_test.go new file mode 100644 index 0000000000..ec120a1e8c --- /dev/null +++ b/flow/e2e/postgres/peer_flow_pg_test.go @@ -0,0 +1,184 @@ +package e2e_postgres + +import ( + "context" + "fmt" + + "github.com/PeerDB-io/peer-flow/e2e" + peerflow "github.com/PeerDB-io/peer-flow/workflows" +) + +func (s *PeerFlowE2ETestSuitePG) attachSchemaSuffix(tableName string) string { + return fmt.Sprintf("e2e_test_%s.%s", postgresSuffix, tableName) +} + +func (s *PeerFlowE2ETestSuitePG) attachSuffix(input string) string { + return fmt.Sprintf("%s_%s", input, postgresSuffix) +} + +func (s *PeerFlowE2ETestSuitePG) Test_Simple_Flow_PG() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_simple_flow") + dstTableName := s.attachSchemaSuffix("test_simple_flow_dst") + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id SERIAL PRIMARY KEY, + key TEXT NOT NULL, + value TEXT NOT NULL + ); + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_simple_flow"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 2, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and then insert 10 rows into the source table + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + // insert 10 rows into the source table + for i := 0; i < 10; i++ { + testKey := fmt.Sprintf("test_key_%d", i) + testValue := fmt.Sprintf("test_value_%d", i) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(key, value) VALUES ($1, $2) + `, srcTableName), testKey, testValue) + s.NoError(err) + } + fmt.Println("Inserted 10 rows into the source table") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + err = s.comparePGTables(srcTableName, dstTableName) + s.NoError(err) + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuitePG) Test_Simple_Schema_Changes_PG() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_simple_schema_changes") + dstTableName := s.attachSchemaSuffix("test_simple_schema_changes_dst") + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id BIGINT PRIMARY KEY GENERATED ALWAYS AS IDENTITY, + c1 BIGINT + ); + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_simple_schema_changes"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 10, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and then insert and mutate schema repeatedly. + go func() { + // insert first row. + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1) VALUES ($1)`, srcTableName), 1) + s.NoError(err) + fmt.Println("Inserted initial row in the source table") + + // verify we got our first row. + e2e.NormalizeFlowCountQuery(env, connectionGen, 2) + err = s.comparePGTables(srcTableName, dstTableName) + s.NoError(err) + + // alter source table, add column c2 and insert another row. + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + ALTER TABLE %s ADD COLUMN c2 BIGINT`, srcTableName)) + s.NoError(err) + fmt.Println("Altered source table, added column c2") + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1,c2) VALUES ($1,$2)`, srcTableName), 2, 2) + s.NoError(err) + fmt.Println("Inserted row with added c2 in the source table") + + // verify we got our two rows, if schema did not match up it will error. + e2e.NormalizeFlowCountQuery(env, connectionGen, 4) + err = s.comparePGTables(srcTableName, dstTableName) + s.NoError(err) + + // alter source table, add column c3, drop column c2 and insert another row. + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + ALTER TABLE %s DROP COLUMN c2, ADD COLUMN c3 BIGINT`, srcTableName)) + s.NoError(err) + fmt.Println("Altered source table, dropped column c2 and added column c3") + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1,c3) VALUES ($1,$2)`, srcTableName), 3, 3) + s.NoError(err) + fmt.Println("Inserted row with added c3 in the source table") + + // verify we got our two rows, if schema did not match up it will error. + e2e.NormalizeFlowCountQuery(env, connectionGen, 6) + err = s.comparePGTables(srcTableName, dstTableName) + s.NoError(err) + + // alter source table, drop column c3 and insert another row. + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + ALTER TABLE %s DROP COLUMN c3`, srcTableName)) + s.NoError(err) + fmt.Println("Altered source table, dropped column c3") + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1) VALUES ($1)`, srcTableName), 4) + s.NoError(err) + fmt.Println("Inserted row after dropping all columns in the source table") + + // verify we got our two rows, if schema did not match up it will error. + e2e.NormalizeFlowCountQuery(env, connectionGen, 8) + err = s.comparePGTables(srcTableName, dstTableName) + s.NoError(err) + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + env.AssertExpectations(s.T()) +} diff --git a/flow/e2e/postgres/postgres_helper.go b/flow/e2e/postgres/postgres_helper.go new file mode 100644 index 0000000000..6f9b471cc3 --- /dev/null +++ b/flow/e2e/postgres/postgres_helper.go @@ -0,0 +1,15 @@ +package e2e_postgres + +import "github.com/PeerDB-io/peer-flow/generated/protos" + +func generatePGPeer(postgresConfig *protos.PostgresConfig) *protos.Peer { + ret := &protos.Peer{} + ret.Name = "test_pg_peer" + ret.Type = protos.DBType_POSTGRES + + ret.Config = &protos.Peer_PostgresConfig{ + PostgresConfig: postgresConfig, + } + + return ret +} diff --git a/flow/e2e/postgres/qrep_flow_pg_test.go b/flow/e2e/postgres/qrep_flow_pg_test.go index 241659c229..1f084409a2 100644 --- a/flow/e2e/postgres/qrep_flow_pg_test.go +++ b/flow/e2e/postgres/qrep_flow_pg_test.go @@ -3,6 +3,7 @@ package e2e_postgres import ( "context" "fmt" + "strings" "testing" "github.com/PeerDB-io/peer-flow/e2e" @@ -21,6 +22,7 @@ type PeerFlowE2ETestSuitePG struct { testsuite.WorkflowTestSuite pool *pgxpool.Pool + peer *protos.Peer } func TestPeerFlowE2ETestSuitePG(t *testing.T) { @@ -43,6 +45,7 @@ func (s *PeerFlowE2ETestSuitePG) SetupSuite() { s.Fail("failed to setup postgres", err) } s.pool = pool + s.peer = generatePGPeer(e2e.GetTestPostgresConf()) } // Implement TearDownAllSuite interface to tear down the test suite @@ -62,14 +65,28 @@ func (s *PeerFlowE2ETestSuitePG) setupSourceTable(tableName string, rowCount int func (s *PeerFlowE2ETestSuitePG) comparePGTables(srcSchemaQualified, dstSchemaQualified string) error { // Execute the two EXCEPT queries - err := s.compareQuery(srcSchemaQualified, dstSchemaQualified) - if err != nil { - return err + for { + err := s.compareQuery(srcSchemaQualified, dstSchemaQualified) + // while testing, the prepared plan might break due to schema changes + // solution is to retry, prepared statement should be evicted upon the first error + if err != nil && !strings.Contains(err.Error(), "cached plan must not change result type") { + return err + } + if err == nil { + break + } } - err = s.compareQuery(dstSchemaQualified, srcSchemaQualified) - if err != nil { - return err + for { + err := s.compareQuery(dstSchemaQualified, srcSchemaQualified) + // while testing, the prepared plan might break due to schema changes + // solution is to retry, prepared statement should be evicted upon the first error + if err != nil && !strings.Contains(err.Error(), "cached plan must not change result type") { + return err + } + if err == nil { + break + } } // If no error is returned, then the contents of the two tables are the same @@ -79,9 +96,11 @@ func (s *PeerFlowE2ETestSuitePG) comparePGTables(srcSchemaQualified, dstSchemaQu func (s *PeerFlowE2ETestSuitePG) compareQuery(schema1, schema2 string) error { query := fmt.Sprintf("SELECT * FROM %s EXCEPT SELECT * FROM %s", schema1, schema2) rows, _ := s.pool.Query(context.Background(), query) + rowsPresent := false defer rows.Close() for rows.Next() { + rowsPresent = true values, err := rows.Values() if err != nil { return err @@ -95,7 +114,13 @@ func (s *PeerFlowE2ETestSuitePG) compareQuery(schema1, schema2 string) error { fmt.Println("---") } - return rows.Err() + if rows.Err() != nil { + return rows.Err() + } + if rowsPresent { + return fmt.Errorf("comparison failed: rows are not equal") + } + return nil } func (s *PeerFlowE2ETestSuitePG) Test_Complete_QRep_Flow_Multi_Insert_PG() { diff --git a/flow/e2e/snowflake/peer_flow_sf_test.go b/flow/e2e/snowflake/peer_flow_sf_test.go index db55046961..b6090a9df0 100644 --- a/flow/e2e/snowflake/peer_flow_sf_test.go +++ b/flow/e2e/snowflake/peer_flow_sf_test.go @@ -798,3 +798,107 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Multi_Table_SF() { env.AssertExpectations(s.T()) } + +func (s *PeerFlowE2ETestSuiteSF) Test_Simple_Schema_Changes_SF() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_simple_schema_changes") + dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_simple_schema_changes") + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id BIGINT PRIMARY KEY GENERATED ALWAYS AS IDENTITY, + c1 BIGINT + ); + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_simple_schema_changes"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.sfHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 10, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and then insert and mutate schema repeatedly. + go func() { + // insert first row. + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1) VALUES ($1)`, srcTableName), 1) + s.NoError(err) + fmt.Println("Inserted initial row in the source table") + + // verify we got our first row. + e2e.NormalizeFlowCountQuery(env, connectionGen, 2) + s.compareTableSchemasSF("test_simple_schema_changes") + s.compareTableContentsSF("test_simple_schema_changes", "id,c1", false) + + // alter source table, add column c2 and insert another row. + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + ALTER TABLE %s ADD COLUMN c2 BIGINT`, srcTableName)) + s.NoError(err) + fmt.Println("Altered source table, added column c2") + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1,c2) VALUES ($1,$2)`, srcTableName), 2, 2) + s.NoError(err) + fmt.Println("Inserted row with added c2 in the source table") + + // verify we got our two rows, if schema did not match up it will error. + e2e.NormalizeFlowCountQuery(env, connectionGen, 4) + s.compareTableSchemasSF("test_simple_schema_changes") + s.compareTableContentsSF("test_simple_schema_changes", "id,c1,c2", false) + + // alter source table, add column c3, drop column c2 and insert another row. + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + ALTER TABLE %s DROP COLUMN c2, ADD COLUMN c3 BIGINT`, srcTableName)) + s.NoError(err) + fmt.Println("Altered source table, dropped column c2 and added column c3") + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1,c3) VALUES ($1,$2)`, srcTableName), 3, 3) + s.NoError(err) + fmt.Println("Inserted row with added c3 in the source table") + + // verify we got our two rows, if schema did not match up it will error. + e2e.NormalizeFlowCountQuery(env, connectionGen, 6) + s.compareTableSchemasSF("test_simple_schema_changes") + s.compareTableContentsSF("test_simple_schema_changes", "id,c1,c3", false) + + // alter source table, drop column c3 and insert another row. + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + ALTER TABLE %s DROP COLUMN c3`, srcTableName)) + s.NoError(err) + fmt.Println("Altered source table, dropped column c3") + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1) VALUES ($1)`, srcTableName), 4) + s.NoError(err) + fmt.Println("Inserted row after dropping all columns in the source table") + + // verify we got our two rows, if schema did not match up it will error. + e2e.NormalizeFlowCountQuery(env, connectionGen, 8) + s.compareTableSchemasSF("test_simple_schema_changes") + s.compareTableContentsSF("test_simple_schema_changes", "id,c1", false) + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + env.AssertExpectations(s.T()) +} diff --git a/flow/e2e/snowflake/qrep_flow_sf_test.go b/flow/e2e/snowflake/qrep_flow_sf_test.go index 4d42adbcc3..4595a198e9 100644 --- a/flow/e2e/snowflake/qrep_flow_sf_test.go +++ b/flow/e2e/snowflake/qrep_flow_sf_test.go @@ -3,6 +3,8 @@ package e2e_snowflake import ( "context" "fmt" + "sort" + "strings" connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" "github.com/PeerDB-io/peer-flow/e2e" @@ -30,6 +32,33 @@ func (s *PeerFlowE2ETestSuiteSF) setupSFDestinationTable(dstTable string) { fmt.Printf("created table on snowflake: %s.%s. %v\n", s.sfHelper.testSchemaName, dstTable, err) } +func (s *PeerFlowE2ETestSuiteSF) compareTableSchemasSF(tableName string) { + // read rows from source table + pgQueryExecutor := connpostgres.NewQRepQueryExecutor(s.pool, context.Background(), "testflow", "testpart") + pgQueryExecutor.SetTestEnv(true) + pgRows, err := pgQueryExecutor.ExecuteAndProcessQuery( + fmt.Sprintf("SELECT * FROM e2e_test_%s.%s LIMIT 0", snowflakeSuffix, tableName), + ) + require.NoError(s.T(), err) + sort.Slice(pgRows.Schema.Fields, func(i int, j int) bool { + return strings.Compare(pgRows.Schema.Fields[i].Name, pgRows.Schema.Fields[j].Name) == -1 + }) + + // read rows from destination table + qualifiedTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, tableName) + // excluding soft-delete column during schema conversion + sfSelQuery := fmt.Sprintf(`SELECT * EXCLUDE _PEERDB_IS_DELETED FROM %s LIMIT 0`, qualifiedTableName) + fmt.Printf("running query on snowflake: %s\n", sfSelQuery) + + sfRows, err := s.sfHelper.ExecuteAndProcessQuery(sfSelQuery) + require.NoError(s.T(), err) + sort.Slice(sfRows.Schema.Fields, func(i int, j int) bool { + return strings.Compare(sfRows.Schema.Fields[i].Name, sfRows.Schema.Fields[j].Name) == -1 + }) + + s.True(pgRows.Schema.EqualNames(sfRows.Schema), "schemas from source and destination tables are not equal") +} + func (s *PeerFlowE2ETestSuiteSF) compareTableContentsSF(tableName string, selector string, caseSensitive bool) { // read rows from source table pgQueryExecutor := connpostgres.NewQRepQueryExecutor(s.pool, context.Background(), "testflow", "testpart") @@ -49,9 +78,6 @@ func (s *PeerFlowE2ETestSuiteSF) compareTableContentsSF(tableName string, select } fmt.Printf("running query on snowflake: %s\n", sfSelQuery) - // sleep for 1 min for debugging - // time.Sleep(1 * time.Minute) - sfRows, err := s.sfHelper.ExecuteAndProcessQuery(sfSelQuery) require.NoError(s.T(), err) diff --git a/flow/e2e/test_utils.go b/flow/e2e/test_utils.go index 761e0cbf51..2d4b005dff 100644 --- a/flow/e2e/test_utils.go +++ b/flow/e2e/test_utils.go @@ -85,6 +85,36 @@ func SetupPeerFlowStatusQuery(env *testsuite.TestWorkflowEnvironment, } } +func NormalizeFlowCountQuery(env *testsuite.TestWorkflowEnvironment, + connectionGen FlowConnectionGenerationConfig, + minCount int) { + // wait for PeerFlowStatusQuery to finish setup + // sleep for 5 second to allow the workflow to start + time.Sleep(5 * time.Second) + for { + response, err := env.QueryWorkflow( + peerflow.PeerFlowStatusQuery, + connectionGen.FlowJobName, + ) + if err == nil { + var state peerflow.PeerFlowState + err = response.Get(&state) + if err != nil { + log.Errorln(err) + } + + if len(state.NormalizeFlowStatuses) >= minCount { + fmt.Println("query indicates setup is complete") + break + } + } else { + // log the error for informational purposes + log.Errorln(err) + } + time.Sleep(1 * time.Second) + } +} + func CreateSourceTableQRep(pool *pgxpool.Pool, suffix string, tableName string) error { tblFields := []string{ "id UUID NOT NULL PRIMARY KEY", diff --git a/flow/generated/protos/flow.pb.go b/flow/generated/protos/flow.pb.go index 13fca6b7b5..acdc22f25f 100644 --- a/flow/generated/protos/flow.pb.go +++ b/flow/generated/protos/flow.pb.go @@ -169,6 +169,132 @@ func (x *TableNameMapping) GetDestinationTableName() string { return "" } +type RelationMessageColumn struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Flags uint32 `protobuf:"varint,1,opt,name=flags,proto3" json:"flags,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + DataType uint32 `protobuf:"varint,3,opt,name=data_type,json=dataType,proto3" json:"data_type,omitempty"` +} + +func (x *RelationMessageColumn) Reset() { + *x = RelationMessageColumn{} + if protoimpl.UnsafeEnabled { + mi := &file_flow_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RelationMessageColumn) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RelationMessageColumn) ProtoMessage() {} + +func (x *RelationMessageColumn) ProtoReflect() protoreflect.Message { + mi := &file_flow_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RelationMessageColumn.ProtoReflect.Descriptor instead. +func (*RelationMessageColumn) Descriptor() ([]byte, []int) { + return file_flow_proto_rawDescGZIP(), []int{1} +} + +func (x *RelationMessageColumn) GetFlags() uint32 { + if x != nil { + return x.Flags + } + return 0 +} + +func (x *RelationMessageColumn) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *RelationMessageColumn) GetDataType() uint32 { + if x != nil { + return x.DataType + } + return 0 +} + +type RelationMessage struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + RelationId uint32 `protobuf:"varint,1,opt,name=relation_id,json=relationId,proto3" json:"relation_id,omitempty"` + RelationName string `protobuf:"bytes,2,opt,name=relation_name,json=relationName,proto3" json:"relation_name,omitempty"` + Columns []*RelationMessageColumn `protobuf:"bytes,3,rep,name=columns,proto3" json:"columns,omitempty"` +} + +func (x *RelationMessage) Reset() { + *x = RelationMessage{} + if protoimpl.UnsafeEnabled { + mi := &file_flow_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RelationMessage) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RelationMessage) ProtoMessage() {} + +func (x *RelationMessage) ProtoReflect() protoreflect.Message { + mi := &file_flow_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RelationMessage.ProtoReflect.Descriptor instead. +func (*RelationMessage) Descriptor() ([]byte, []int) { + return file_flow_proto_rawDescGZIP(), []int{2} +} + +func (x *RelationMessage) GetRelationId() uint32 { + if x != nil { + return x.RelationId + } + return 0 +} + +func (x *RelationMessage) GetRelationName() string { + if x != nil { + return x.RelationName + } + return "" +} + +func (x *RelationMessage) GetColumns() []*RelationMessageColumn { + if x != nil { + return x.Columns + } + return nil +} + type FlowConnectionConfigs struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -206,7 +332,7 @@ type FlowConnectionConfigs struct { func (x *FlowConnectionConfigs) Reset() { *x = FlowConnectionConfigs{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[1] + mi := &file_flow_proto_msgTypes[3] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -219,7 +345,7 @@ func (x *FlowConnectionConfigs) String() string { func (*FlowConnectionConfigs) ProtoMessage() {} func (x *FlowConnectionConfigs) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[1] + mi := &file_flow_proto_msgTypes[3] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -232,7 +358,7 @@ func (x *FlowConnectionConfigs) ProtoReflect() protoreflect.Message { // Deprecated: Use FlowConnectionConfigs.ProtoReflect.Descriptor instead. func (*FlowConnectionConfigs) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{1} + return file_flow_proto_rawDescGZIP(), []int{3} } func (x *FlowConnectionConfigs) GetSource() *Peer { @@ -394,13 +520,14 @@ type SyncFlowOptions struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - BatchSize int32 `protobuf:"varint,1,opt,name=batch_size,json=batchSize,proto3" json:"batch_size,omitempty"` + BatchSize int32 `protobuf:"varint,1,opt,name=batch_size,json=batchSize,proto3" json:"batch_size,omitempty"` + RelationMessageMapping map[uint32]*RelationMessage `protobuf:"bytes,2,rep,name=relation_message_mapping,json=relationMessageMapping,proto3" json:"relation_message_mapping,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` } func (x *SyncFlowOptions) Reset() { *x = SyncFlowOptions{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[2] + mi := &file_flow_proto_msgTypes[4] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -413,7 +540,7 @@ func (x *SyncFlowOptions) String() string { func (*SyncFlowOptions) ProtoMessage() {} func (x *SyncFlowOptions) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[2] + mi := &file_flow_proto_msgTypes[4] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -426,7 +553,7 @@ func (x *SyncFlowOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use SyncFlowOptions.ProtoReflect.Descriptor instead. func (*SyncFlowOptions) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{2} + return file_flow_proto_rawDescGZIP(), []int{4} } func (x *SyncFlowOptions) GetBatchSize() int32 { @@ -436,6 +563,13 @@ func (x *SyncFlowOptions) GetBatchSize() int32 { return 0 } +func (x *SyncFlowOptions) GetRelationMessageMapping() map[uint32]*RelationMessage { + if x != nil { + return x.RelationMessageMapping + } + return nil +} + type NormalizeFlowOptions struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -447,7 +581,7 @@ type NormalizeFlowOptions struct { func (x *NormalizeFlowOptions) Reset() { *x = NormalizeFlowOptions{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[3] + mi := &file_flow_proto_msgTypes[5] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -460,7 +594,7 @@ func (x *NormalizeFlowOptions) String() string { func (*NormalizeFlowOptions) ProtoMessage() {} func (x *NormalizeFlowOptions) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[3] + mi := &file_flow_proto_msgTypes[5] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -473,7 +607,7 @@ func (x *NormalizeFlowOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use NormalizeFlowOptions.ProtoReflect.Descriptor instead. func (*NormalizeFlowOptions) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{3} + return file_flow_proto_rawDescGZIP(), []int{5} } func (x *NormalizeFlowOptions) GetBatchSize() int32 { @@ -495,7 +629,7 @@ type LastSyncState struct { func (x *LastSyncState) Reset() { *x = LastSyncState{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[4] + mi := &file_flow_proto_msgTypes[6] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -508,7 +642,7 @@ func (x *LastSyncState) String() string { func (*LastSyncState) ProtoMessage() {} func (x *LastSyncState) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[4] + mi := &file_flow_proto_msgTypes[6] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -521,7 +655,7 @@ func (x *LastSyncState) ProtoReflect() protoreflect.Message { // Deprecated: Use LastSyncState.ProtoReflect.Descriptor instead. func (*LastSyncState) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{4} + return file_flow_proto_rawDescGZIP(), []int{6} } func (x *LastSyncState) GetCheckpoint() int64 { @@ -543,15 +677,16 @@ type StartFlowInput struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - LastSyncState *LastSyncState `protobuf:"bytes,1,opt,name=last_sync_state,json=lastSyncState,proto3" json:"last_sync_state,omitempty"` - FlowConnectionConfigs *FlowConnectionConfigs `protobuf:"bytes,2,opt,name=flow_connection_configs,json=flowConnectionConfigs,proto3" json:"flow_connection_configs,omitempty"` - SyncFlowOptions *SyncFlowOptions `protobuf:"bytes,3,opt,name=sync_flow_options,json=syncFlowOptions,proto3" json:"sync_flow_options,omitempty"` + LastSyncState *LastSyncState `protobuf:"bytes,1,opt,name=last_sync_state,json=lastSyncState,proto3" json:"last_sync_state,omitempty"` + FlowConnectionConfigs *FlowConnectionConfigs `protobuf:"bytes,2,opt,name=flow_connection_configs,json=flowConnectionConfigs,proto3" json:"flow_connection_configs,omitempty"` + SyncFlowOptions *SyncFlowOptions `protobuf:"bytes,3,opt,name=sync_flow_options,json=syncFlowOptions,proto3" json:"sync_flow_options,omitempty"` + RelationMessageMapping map[uint32]*RelationMessage `protobuf:"bytes,4,rep,name=relation_message_mapping,json=relationMessageMapping,proto3" json:"relation_message_mapping,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` } func (x *StartFlowInput) Reset() { *x = StartFlowInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[5] + mi := &file_flow_proto_msgTypes[7] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -564,7 +699,7 @@ func (x *StartFlowInput) String() string { func (*StartFlowInput) ProtoMessage() {} func (x *StartFlowInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[5] + mi := &file_flow_proto_msgTypes[7] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -577,7 +712,7 @@ func (x *StartFlowInput) ProtoReflect() protoreflect.Message { // Deprecated: Use StartFlowInput.ProtoReflect.Descriptor instead. func (*StartFlowInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{5} + return file_flow_proto_rawDescGZIP(), []int{7} } func (x *StartFlowInput) GetLastSyncState() *LastSyncState { @@ -601,6 +736,13 @@ func (x *StartFlowInput) GetSyncFlowOptions() *SyncFlowOptions { return nil } +func (x *StartFlowInput) GetRelationMessageMapping() map[uint32]*RelationMessage { + if x != nil { + return x.RelationMessageMapping + } + return nil +} + type StartNormalizeInput struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -612,7 +754,7 @@ type StartNormalizeInput struct { func (x *StartNormalizeInput) Reset() { *x = StartNormalizeInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[6] + mi := &file_flow_proto_msgTypes[8] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -625,7 +767,7 @@ func (x *StartNormalizeInput) String() string { func (*StartNormalizeInput) ProtoMessage() {} func (x *StartNormalizeInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[6] + mi := &file_flow_proto_msgTypes[8] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -638,7 +780,7 @@ func (x *StartNormalizeInput) ProtoReflect() protoreflect.Message { // Deprecated: Use StartNormalizeInput.ProtoReflect.Descriptor instead. func (*StartNormalizeInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{6} + return file_flow_proto_rawDescGZIP(), []int{8} } func (x *StartNormalizeInput) GetFlowConnectionConfigs() *FlowConnectionConfigs { @@ -660,7 +802,7 @@ type GetLastSyncedIDInput struct { func (x *GetLastSyncedIDInput) Reset() { *x = GetLastSyncedIDInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[7] + mi := &file_flow_proto_msgTypes[9] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -673,7 +815,7 @@ func (x *GetLastSyncedIDInput) String() string { func (*GetLastSyncedIDInput) ProtoMessage() {} func (x *GetLastSyncedIDInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[7] + mi := &file_flow_proto_msgTypes[9] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -686,7 +828,7 @@ func (x *GetLastSyncedIDInput) ProtoReflect() protoreflect.Message { // Deprecated: Use GetLastSyncedIDInput.ProtoReflect.Descriptor instead. func (*GetLastSyncedIDInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{7} + return file_flow_proto_rawDescGZIP(), []int{9} } func (x *GetLastSyncedIDInput) GetPeerConnectionConfig() *Peer { @@ -716,7 +858,7 @@ type EnsurePullabilityInput struct { func (x *EnsurePullabilityInput) Reset() { *x = EnsurePullabilityInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[8] + mi := &file_flow_proto_msgTypes[10] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -729,7 +871,7 @@ func (x *EnsurePullabilityInput) String() string { func (*EnsurePullabilityInput) ProtoMessage() {} func (x *EnsurePullabilityInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[8] + mi := &file_flow_proto_msgTypes[10] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -742,7 +884,7 @@ func (x *EnsurePullabilityInput) ProtoReflect() protoreflect.Message { // Deprecated: Use EnsurePullabilityInput.ProtoReflect.Descriptor instead. func (*EnsurePullabilityInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{8} + return file_flow_proto_rawDescGZIP(), []int{10} } func (x *EnsurePullabilityInput) GetPeerConnectionConfig() *Peer { @@ -779,7 +921,7 @@ type EnsurePullabilityBatchInput struct { func (x *EnsurePullabilityBatchInput) Reset() { *x = EnsurePullabilityBatchInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[9] + mi := &file_flow_proto_msgTypes[11] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -792,7 +934,7 @@ func (x *EnsurePullabilityBatchInput) String() string { func (*EnsurePullabilityBatchInput) ProtoMessage() {} func (x *EnsurePullabilityBatchInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[9] + mi := &file_flow_proto_msgTypes[11] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -805,7 +947,7 @@ func (x *EnsurePullabilityBatchInput) ProtoReflect() protoreflect.Message { // Deprecated: Use EnsurePullabilityBatchInput.ProtoReflect.Descriptor instead. func (*EnsurePullabilityBatchInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{9} + return file_flow_proto_rawDescGZIP(), []int{11} } func (x *EnsurePullabilityBatchInput) GetPeerConnectionConfig() *Peer { @@ -840,7 +982,7 @@ type PostgresTableIdentifier struct { func (x *PostgresTableIdentifier) Reset() { *x = PostgresTableIdentifier{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[10] + mi := &file_flow_proto_msgTypes[12] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -853,7 +995,7 @@ func (x *PostgresTableIdentifier) String() string { func (*PostgresTableIdentifier) ProtoMessage() {} func (x *PostgresTableIdentifier) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[10] + mi := &file_flow_proto_msgTypes[12] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -866,7 +1008,7 @@ func (x *PostgresTableIdentifier) ProtoReflect() protoreflect.Message { // Deprecated: Use PostgresTableIdentifier.ProtoReflect.Descriptor instead. func (*PostgresTableIdentifier) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{10} + return file_flow_proto_rawDescGZIP(), []int{12} } func (x *PostgresTableIdentifier) GetRelId() uint32 { @@ -890,7 +1032,7 @@ type TableIdentifier struct { func (x *TableIdentifier) Reset() { *x = TableIdentifier{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[11] + mi := &file_flow_proto_msgTypes[13] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -903,7 +1045,7 @@ func (x *TableIdentifier) String() string { func (*TableIdentifier) ProtoMessage() {} func (x *TableIdentifier) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[11] + mi := &file_flow_proto_msgTypes[13] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -916,7 +1058,7 @@ func (x *TableIdentifier) ProtoReflect() protoreflect.Message { // Deprecated: Use TableIdentifier.ProtoReflect.Descriptor instead. func (*TableIdentifier) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{11} + return file_flow_proto_rawDescGZIP(), []int{13} } func (m *TableIdentifier) GetTableIdentifier() isTableIdentifier_TableIdentifier { @@ -954,7 +1096,7 @@ type EnsurePullabilityOutput struct { func (x *EnsurePullabilityOutput) Reset() { *x = EnsurePullabilityOutput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[12] + mi := &file_flow_proto_msgTypes[14] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -967,7 +1109,7 @@ func (x *EnsurePullabilityOutput) String() string { func (*EnsurePullabilityOutput) ProtoMessage() {} func (x *EnsurePullabilityOutput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[12] + mi := &file_flow_proto_msgTypes[14] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -980,7 +1122,7 @@ func (x *EnsurePullabilityOutput) ProtoReflect() protoreflect.Message { // Deprecated: Use EnsurePullabilityOutput.ProtoReflect.Descriptor instead. func (*EnsurePullabilityOutput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{12} + return file_flow_proto_rawDescGZIP(), []int{14} } func (x *EnsurePullabilityOutput) GetTableIdentifier() *TableIdentifier { @@ -1001,7 +1143,7 @@ type EnsurePullabilityBatchOutput struct { func (x *EnsurePullabilityBatchOutput) Reset() { *x = EnsurePullabilityBatchOutput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[13] + mi := &file_flow_proto_msgTypes[15] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1014,7 +1156,7 @@ func (x *EnsurePullabilityBatchOutput) String() string { func (*EnsurePullabilityBatchOutput) ProtoMessage() {} func (x *EnsurePullabilityBatchOutput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[13] + mi := &file_flow_proto_msgTypes[15] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1027,7 +1169,7 @@ func (x *EnsurePullabilityBatchOutput) ProtoReflect() protoreflect.Message { // Deprecated: Use EnsurePullabilityBatchOutput.ProtoReflect.Descriptor instead. func (*EnsurePullabilityBatchOutput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{13} + return file_flow_proto_rawDescGZIP(), []int{15} } func (x *EnsurePullabilityBatchOutput) GetTableIdentifierMapping() map[string]*TableIdentifier { @@ -1055,7 +1197,7 @@ type SetupReplicationInput struct { func (x *SetupReplicationInput) Reset() { *x = SetupReplicationInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[14] + mi := &file_flow_proto_msgTypes[16] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1068,7 +1210,7 @@ func (x *SetupReplicationInput) String() string { func (*SetupReplicationInput) ProtoMessage() {} func (x *SetupReplicationInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[14] + mi := &file_flow_proto_msgTypes[16] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1081,7 +1223,7 @@ func (x *SetupReplicationInput) ProtoReflect() protoreflect.Message { // Deprecated: Use SetupReplicationInput.ProtoReflect.Descriptor instead. func (*SetupReplicationInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{14} + return file_flow_proto_rawDescGZIP(), []int{16} } func (x *SetupReplicationInput) GetPeerConnectionConfig() *Peer { @@ -1145,7 +1287,7 @@ type SetupReplicationOutput struct { func (x *SetupReplicationOutput) Reset() { *x = SetupReplicationOutput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[15] + mi := &file_flow_proto_msgTypes[17] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1158,7 +1300,7 @@ func (x *SetupReplicationOutput) String() string { func (*SetupReplicationOutput) ProtoMessage() {} func (x *SetupReplicationOutput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[15] + mi := &file_flow_proto_msgTypes[17] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1171,7 +1313,7 @@ func (x *SetupReplicationOutput) ProtoReflect() protoreflect.Message { // Deprecated: Use SetupReplicationOutput.ProtoReflect.Descriptor instead. func (*SetupReplicationOutput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{15} + return file_flow_proto_rawDescGZIP(), []int{17} } func (x *SetupReplicationOutput) GetSlotName() string { @@ -1202,7 +1344,7 @@ type CreateRawTableInput struct { func (x *CreateRawTableInput) Reset() { *x = CreateRawTableInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[16] + mi := &file_flow_proto_msgTypes[18] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1215,7 +1357,7 @@ func (x *CreateRawTableInput) String() string { func (*CreateRawTableInput) ProtoMessage() {} func (x *CreateRawTableInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[16] + mi := &file_flow_proto_msgTypes[18] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1228,7 +1370,7 @@ func (x *CreateRawTableInput) ProtoReflect() protoreflect.Message { // Deprecated: Use CreateRawTableInput.ProtoReflect.Descriptor instead. func (*CreateRawTableInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{16} + return file_flow_proto_rawDescGZIP(), []int{18} } func (x *CreateRawTableInput) GetPeerConnectionConfig() *Peer { @@ -1270,7 +1412,7 @@ type CreateRawTableOutput struct { func (x *CreateRawTableOutput) Reset() { *x = CreateRawTableOutput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[17] + mi := &file_flow_proto_msgTypes[19] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1283,7 +1425,7 @@ func (x *CreateRawTableOutput) String() string { func (*CreateRawTableOutput) ProtoMessage() {} func (x *CreateRawTableOutput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[17] + mi := &file_flow_proto_msgTypes[19] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1296,7 +1438,7 @@ func (x *CreateRawTableOutput) ProtoReflect() protoreflect.Message { // Deprecated: Use CreateRawTableOutput.ProtoReflect.Descriptor instead. func (*CreateRawTableOutput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{17} + return file_flow_proto_rawDescGZIP(), []int{19} } func (x *CreateRawTableOutput) GetTableIdentifier() string { @@ -1321,7 +1463,7 @@ type TableSchema struct { func (x *TableSchema) Reset() { *x = TableSchema{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[18] + mi := &file_flow_proto_msgTypes[20] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1334,7 +1476,7 @@ func (x *TableSchema) String() string { func (*TableSchema) ProtoMessage() {} func (x *TableSchema) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[18] + mi := &file_flow_proto_msgTypes[20] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1347,7 +1489,7 @@ func (x *TableSchema) ProtoReflect() protoreflect.Message { // Deprecated: Use TableSchema.ProtoReflect.Descriptor instead. func (*TableSchema) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{18} + return file_flow_proto_rawDescGZIP(), []int{20} } func (x *TableSchema) GetTableIdentifier() string { @@ -1383,7 +1525,7 @@ type GetTableSchemaBatchInput struct { func (x *GetTableSchemaBatchInput) Reset() { *x = GetTableSchemaBatchInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[19] + mi := &file_flow_proto_msgTypes[21] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1396,7 +1538,7 @@ func (x *GetTableSchemaBatchInput) String() string { func (*GetTableSchemaBatchInput) ProtoMessage() {} func (x *GetTableSchemaBatchInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[19] + mi := &file_flow_proto_msgTypes[21] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1409,7 +1551,7 @@ func (x *GetTableSchemaBatchInput) ProtoReflect() protoreflect.Message { // Deprecated: Use GetTableSchemaBatchInput.ProtoReflect.Descriptor instead. func (*GetTableSchemaBatchInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{19} + return file_flow_proto_rawDescGZIP(), []int{21} } func (x *GetTableSchemaBatchInput) GetPeerConnectionConfig() *Peer { @@ -1437,7 +1579,7 @@ type GetTableSchemaBatchOutput struct { func (x *GetTableSchemaBatchOutput) Reset() { *x = GetTableSchemaBatchOutput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[20] + mi := &file_flow_proto_msgTypes[22] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1450,7 +1592,7 @@ func (x *GetTableSchemaBatchOutput) String() string { func (*GetTableSchemaBatchOutput) ProtoMessage() {} func (x *GetTableSchemaBatchOutput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[20] + mi := &file_flow_proto_msgTypes[22] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1463,7 +1605,7 @@ func (x *GetTableSchemaBatchOutput) ProtoReflect() protoreflect.Message { // Deprecated: Use GetTableSchemaBatchOutput.ProtoReflect.Descriptor instead. func (*GetTableSchemaBatchOutput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{20} + return file_flow_proto_rawDescGZIP(), []int{22} } func (x *GetTableSchemaBatchOutput) GetTableNameSchemaMapping() map[string]*TableSchema { @@ -1486,7 +1628,7 @@ type SetupNormalizedTableInput struct { func (x *SetupNormalizedTableInput) Reset() { *x = SetupNormalizedTableInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[21] + mi := &file_flow_proto_msgTypes[23] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1499,7 +1641,7 @@ func (x *SetupNormalizedTableInput) String() string { func (*SetupNormalizedTableInput) ProtoMessage() {} func (x *SetupNormalizedTableInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[21] + mi := &file_flow_proto_msgTypes[23] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1512,7 +1654,7 @@ func (x *SetupNormalizedTableInput) ProtoReflect() protoreflect.Message { // Deprecated: Use SetupNormalizedTableInput.ProtoReflect.Descriptor instead. func (*SetupNormalizedTableInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{21} + return file_flow_proto_rawDescGZIP(), []int{23} } func (x *SetupNormalizedTableInput) GetPeerConnectionConfig() *Peer { @@ -1548,7 +1690,7 @@ type SetupNormalizedTableBatchInput struct { func (x *SetupNormalizedTableBatchInput) Reset() { *x = SetupNormalizedTableBatchInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[22] + mi := &file_flow_proto_msgTypes[24] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1561,7 +1703,7 @@ func (x *SetupNormalizedTableBatchInput) String() string { func (*SetupNormalizedTableBatchInput) ProtoMessage() {} func (x *SetupNormalizedTableBatchInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[22] + mi := &file_flow_proto_msgTypes[24] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1574,7 +1716,7 @@ func (x *SetupNormalizedTableBatchInput) ProtoReflect() protoreflect.Message { // Deprecated: Use SetupNormalizedTableBatchInput.ProtoReflect.Descriptor instead. func (*SetupNormalizedTableBatchInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{22} + return file_flow_proto_rawDescGZIP(), []int{24} } func (x *SetupNormalizedTableBatchInput) GetPeerConnectionConfig() *Peer { @@ -1603,7 +1745,7 @@ type SetupNormalizedTableOutput struct { func (x *SetupNormalizedTableOutput) Reset() { *x = SetupNormalizedTableOutput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[23] + mi := &file_flow_proto_msgTypes[25] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1616,7 +1758,7 @@ func (x *SetupNormalizedTableOutput) String() string { func (*SetupNormalizedTableOutput) ProtoMessage() {} func (x *SetupNormalizedTableOutput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[23] + mi := &file_flow_proto_msgTypes[25] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1629,7 +1771,7 @@ func (x *SetupNormalizedTableOutput) ProtoReflect() protoreflect.Message { // Deprecated: Use SetupNormalizedTableOutput.ProtoReflect.Descriptor instead. func (*SetupNormalizedTableOutput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{23} + return file_flow_proto_rawDescGZIP(), []int{25} } func (x *SetupNormalizedTableOutput) GetTableIdentifier() string { @@ -1657,7 +1799,7 @@ type SetupNormalizedTableBatchOutput struct { func (x *SetupNormalizedTableBatchOutput) Reset() { *x = SetupNormalizedTableBatchOutput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[24] + mi := &file_flow_proto_msgTypes[26] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1670,7 +1812,7 @@ func (x *SetupNormalizedTableBatchOutput) String() string { func (*SetupNormalizedTableBatchOutput) ProtoMessage() {} func (x *SetupNormalizedTableBatchOutput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[24] + mi := &file_flow_proto_msgTypes[26] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1683,7 +1825,7 @@ func (x *SetupNormalizedTableBatchOutput) ProtoReflect() protoreflect.Message { // Deprecated: Use SetupNormalizedTableBatchOutput.ProtoReflect.Descriptor instead. func (*SetupNormalizedTableBatchOutput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{24} + return file_flow_proto_rawDescGZIP(), []int{26} } func (x *SetupNormalizedTableBatchOutput) GetTableExistsMapping() map[string]bool { @@ -1706,7 +1848,7 @@ type IntPartitionRange struct { func (x *IntPartitionRange) Reset() { *x = IntPartitionRange{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[25] + mi := &file_flow_proto_msgTypes[27] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1719,7 +1861,7 @@ func (x *IntPartitionRange) String() string { func (*IntPartitionRange) ProtoMessage() {} func (x *IntPartitionRange) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[25] + mi := &file_flow_proto_msgTypes[27] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1732,7 +1874,7 @@ func (x *IntPartitionRange) ProtoReflect() protoreflect.Message { // Deprecated: Use IntPartitionRange.ProtoReflect.Descriptor instead. func (*IntPartitionRange) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{25} + return file_flow_proto_rawDescGZIP(), []int{27} } func (x *IntPartitionRange) GetStart() int64 { @@ -1761,7 +1903,7 @@ type TimestampPartitionRange struct { func (x *TimestampPartitionRange) Reset() { *x = TimestampPartitionRange{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[26] + mi := &file_flow_proto_msgTypes[28] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1774,7 +1916,7 @@ func (x *TimestampPartitionRange) String() string { func (*TimestampPartitionRange) ProtoMessage() {} func (x *TimestampPartitionRange) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[26] + mi := &file_flow_proto_msgTypes[28] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1787,7 +1929,7 @@ func (x *TimestampPartitionRange) ProtoReflect() protoreflect.Message { // Deprecated: Use TimestampPartitionRange.ProtoReflect.Descriptor instead. func (*TimestampPartitionRange) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{26} + return file_flow_proto_rawDescGZIP(), []int{28} } func (x *TimestampPartitionRange) GetStart() *timestamppb.Timestamp { @@ -1816,7 +1958,7 @@ type TID struct { func (x *TID) Reset() { *x = TID{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[27] + mi := &file_flow_proto_msgTypes[29] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1829,7 +1971,7 @@ func (x *TID) String() string { func (*TID) ProtoMessage() {} func (x *TID) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[27] + mi := &file_flow_proto_msgTypes[29] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1842,7 +1984,7 @@ func (x *TID) ProtoReflect() protoreflect.Message { // Deprecated: Use TID.ProtoReflect.Descriptor instead. func (*TID) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{27} + return file_flow_proto_rawDescGZIP(), []int{29} } func (x *TID) GetBlockNumber() uint32 { @@ -1871,7 +2013,7 @@ type TIDPartitionRange struct { func (x *TIDPartitionRange) Reset() { *x = TIDPartitionRange{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[28] + mi := &file_flow_proto_msgTypes[30] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1884,7 +2026,7 @@ func (x *TIDPartitionRange) String() string { func (*TIDPartitionRange) ProtoMessage() {} func (x *TIDPartitionRange) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[28] + mi := &file_flow_proto_msgTypes[30] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1897,7 +2039,7 @@ func (x *TIDPartitionRange) ProtoReflect() protoreflect.Message { // Deprecated: Use TIDPartitionRange.ProtoReflect.Descriptor instead. func (*TIDPartitionRange) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{28} + return file_flow_proto_rawDescGZIP(), []int{30} } func (x *TIDPartitionRange) GetStart() *TID { @@ -1932,7 +2074,7 @@ type PartitionRange struct { func (x *PartitionRange) Reset() { *x = PartitionRange{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[29] + mi := &file_flow_proto_msgTypes[31] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1945,7 +2087,7 @@ func (x *PartitionRange) String() string { func (*PartitionRange) ProtoMessage() {} func (x *PartitionRange) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[29] + mi := &file_flow_proto_msgTypes[31] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1958,7 +2100,7 @@ func (x *PartitionRange) ProtoReflect() protoreflect.Message { // Deprecated: Use PartitionRange.ProtoReflect.Descriptor instead. func (*PartitionRange) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{29} + return file_flow_proto_rawDescGZIP(), []int{31} } func (m *PartitionRange) GetRange() isPartitionRange_Range { @@ -2023,7 +2165,7 @@ type QRepWriteMode struct { func (x *QRepWriteMode) Reset() { *x = QRepWriteMode{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[30] + mi := &file_flow_proto_msgTypes[32] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2036,7 +2178,7 @@ func (x *QRepWriteMode) String() string { func (*QRepWriteMode) ProtoMessage() {} func (x *QRepWriteMode) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[30] + mi := &file_flow_proto_msgTypes[32] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2049,7 +2191,7 @@ func (x *QRepWriteMode) ProtoReflect() protoreflect.Message { // Deprecated: Use QRepWriteMode.ProtoReflect.Descriptor instead. func (*QRepWriteMode) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{30} + return file_flow_proto_rawDescGZIP(), []int{32} } func (x *QRepWriteMode) GetWriteType() QRepWriteType { @@ -2102,7 +2244,7 @@ type QRepConfig struct { func (x *QRepConfig) Reset() { *x = QRepConfig{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[31] + mi := &file_flow_proto_msgTypes[33] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2115,7 +2257,7 @@ func (x *QRepConfig) String() string { func (*QRepConfig) ProtoMessage() {} func (x *QRepConfig) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[31] + mi := &file_flow_proto_msgTypes[33] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2128,7 +2270,7 @@ func (x *QRepConfig) ProtoReflect() protoreflect.Message { // Deprecated: Use QRepConfig.ProtoReflect.Descriptor instead. func (*QRepConfig) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{31} + return file_flow_proto_rawDescGZIP(), []int{33} } func (x *QRepConfig) GetFlowJobName() string { @@ -2256,7 +2398,7 @@ type QRepPartition struct { func (x *QRepPartition) Reset() { *x = QRepPartition{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[32] + mi := &file_flow_proto_msgTypes[34] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2269,7 +2411,7 @@ func (x *QRepPartition) String() string { func (*QRepPartition) ProtoMessage() {} func (x *QRepPartition) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[32] + mi := &file_flow_proto_msgTypes[34] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2282,7 +2424,7 @@ func (x *QRepPartition) ProtoReflect() protoreflect.Message { // Deprecated: Use QRepPartition.ProtoReflect.Descriptor instead. func (*QRepPartition) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{32} + return file_flow_proto_rawDescGZIP(), []int{34} } func (x *QRepPartition) GetPartitionId() string { @@ -2318,7 +2460,7 @@ type QRepPartitionBatch struct { func (x *QRepPartitionBatch) Reset() { *x = QRepPartitionBatch{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[33] + mi := &file_flow_proto_msgTypes[35] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2331,7 +2473,7 @@ func (x *QRepPartitionBatch) String() string { func (*QRepPartitionBatch) ProtoMessage() {} func (x *QRepPartitionBatch) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[33] + mi := &file_flow_proto_msgTypes[35] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2344,7 +2486,7 @@ func (x *QRepPartitionBatch) ProtoReflect() protoreflect.Message { // Deprecated: Use QRepPartitionBatch.ProtoReflect.Descriptor instead. func (*QRepPartitionBatch) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{33} + return file_flow_proto_rawDescGZIP(), []int{35} } func (x *QRepPartitionBatch) GetBatchId() int32 { @@ -2372,7 +2514,7 @@ type QRepParitionResult struct { func (x *QRepParitionResult) Reset() { *x = QRepParitionResult{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[34] + mi := &file_flow_proto_msgTypes[36] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2385,7 +2527,7 @@ func (x *QRepParitionResult) String() string { func (*QRepParitionResult) ProtoMessage() {} func (x *QRepParitionResult) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[34] + mi := &file_flow_proto_msgTypes[36] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2398,7 +2540,7 @@ func (x *QRepParitionResult) ProtoReflect() protoreflect.Message { // Deprecated: Use QRepParitionResult.ProtoReflect.Descriptor instead. func (*QRepParitionResult) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{34} + return file_flow_proto_rawDescGZIP(), []int{36} } func (x *QRepParitionResult) GetPartitions() []*QRepPartition { @@ -2419,7 +2561,7 @@ type DropFlowInput struct { func (x *DropFlowInput) Reset() { *x = DropFlowInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[35] + mi := &file_flow_proto_msgTypes[37] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2432,7 +2574,7 @@ func (x *DropFlowInput) String() string { func (*DropFlowInput) ProtoMessage() {} func (x *DropFlowInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[35] + mi := &file_flow_proto_msgTypes[37] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2445,7 +2587,7 @@ func (x *DropFlowInput) ProtoReflect() protoreflect.Message { // Deprecated: Use DropFlowInput.ProtoReflect.Descriptor instead. func (*DropFlowInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{35} + return file_flow_proto_rawDescGZIP(), []int{37} } func (x *DropFlowInput) GetFlowName() string { @@ -2455,6 +2597,187 @@ func (x *DropFlowInput) GetFlowName() string { return "" } +type DeltaAddedColumn struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ColumnName string `protobuf:"bytes,1,opt,name=column_name,json=columnName,proto3" json:"column_name,omitempty"` + ColumnType string `protobuf:"bytes,2,opt,name=column_type,json=columnType,proto3" json:"column_type,omitempty"` +} + +func (x *DeltaAddedColumn) Reset() { + *x = DeltaAddedColumn{} + if protoimpl.UnsafeEnabled { + mi := &file_flow_proto_msgTypes[38] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeltaAddedColumn) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeltaAddedColumn) ProtoMessage() {} + +func (x *DeltaAddedColumn) ProtoReflect() protoreflect.Message { + mi := &file_flow_proto_msgTypes[38] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DeltaAddedColumn.ProtoReflect.Descriptor instead. +func (*DeltaAddedColumn) Descriptor() ([]byte, []int) { + return file_flow_proto_rawDescGZIP(), []int{38} +} + +func (x *DeltaAddedColumn) GetColumnName() string { + if x != nil { + return x.ColumnName + } + return "" +} + +func (x *DeltaAddedColumn) GetColumnType() string { + if x != nil { + return x.ColumnType + } + return "" +} + +type TableSchemaDelta struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + SrcTableName string `protobuf:"bytes,1,opt,name=src_table_name,json=srcTableName,proto3" json:"src_table_name,omitempty"` + DstTableName string `protobuf:"bytes,2,opt,name=dst_table_name,json=dstTableName,proto3" json:"dst_table_name,omitempty"` + AddedColumns []*DeltaAddedColumn `protobuf:"bytes,3,rep,name=added_columns,json=addedColumns,proto3" json:"added_columns,omitempty"` + DroppedColumns []string `protobuf:"bytes,4,rep,name=dropped_columns,json=droppedColumns,proto3" json:"dropped_columns,omitempty"` +} + +func (x *TableSchemaDelta) Reset() { + *x = TableSchemaDelta{} + if protoimpl.UnsafeEnabled { + mi := &file_flow_proto_msgTypes[39] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TableSchemaDelta) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TableSchemaDelta) ProtoMessage() {} + +func (x *TableSchemaDelta) ProtoReflect() protoreflect.Message { + mi := &file_flow_proto_msgTypes[39] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TableSchemaDelta.ProtoReflect.Descriptor instead. +func (*TableSchemaDelta) Descriptor() ([]byte, []int) { + return file_flow_proto_rawDescGZIP(), []int{39} +} + +func (x *TableSchemaDelta) GetSrcTableName() string { + if x != nil { + return x.SrcTableName + } + return "" +} + +func (x *TableSchemaDelta) GetDstTableName() string { + if x != nil { + return x.DstTableName + } + return "" +} + +func (x *TableSchemaDelta) GetAddedColumns() []*DeltaAddedColumn { + if x != nil { + return x.AddedColumns + } + return nil +} + +func (x *TableSchemaDelta) GetDroppedColumns() []string { + if x != nil { + return x.DroppedColumns + } + return nil +} + +type ReplayTableSchemaDeltaInput struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + FlowConnectionConfigs *FlowConnectionConfigs `protobuf:"bytes,1,opt,name=flow_connection_configs,json=flowConnectionConfigs,proto3" json:"flow_connection_configs,omitempty"` + TableSchemaDelta *TableSchemaDelta `protobuf:"bytes,2,opt,name=table_schema_delta,json=tableSchemaDelta,proto3" json:"table_schema_delta,omitempty"` +} + +func (x *ReplayTableSchemaDeltaInput) Reset() { + *x = ReplayTableSchemaDeltaInput{} + if protoimpl.UnsafeEnabled { + mi := &file_flow_proto_msgTypes[40] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ReplayTableSchemaDeltaInput) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ReplayTableSchemaDeltaInput) ProtoMessage() {} + +func (x *ReplayTableSchemaDeltaInput) ProtoReflect() protoreflect.Message { + mi := &file_flow_proto_msgTypes[40] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ReplayTableSchemaDeltaInput.ProtoReflect.Descriptor instead. +func (*ReplayTableSchemaDeltaInput) Descriptor() ([]byte, []int) { + return file_flow_proto_rawDescGZIP(), []int{40} +} + +func (x *ReplayTableSchemaDeltaInput) GetFlowConnectionConfigs() *FlowConnectionConfigs { + if x != nil { + return x.FlowConnectionConfigs + } + return nil +} + +func (x *ReplayTableSchemaDeltaInput) GetTableSchemaDelta() *TableSchemaDelta { + if x != nil { + return x.TableSchemaDelta + } + return nil +} + var File_flow_proto protoreflect.FileDescriptor var file_flow_proto_rawDesc = []byte{ @@ -2469,511 +2792,585 @@ var file_flow_proto_rawDesc = []byte{ 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x34, 0x0a, 0x16, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x14, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xb9, 0x0c, - 0x0a, 0x15, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x12, 0x2a, 0x0a, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, - 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, - 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x06, 0x73, 0x6f, 0x75, - 0x72, 0x63, 0x65, 0x12, 0x34, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0b, 0x64, 0x65, - 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, - 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x3b, 0x0a, - 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x04, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, - 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x0b, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x66, 0x0a, 0x12, 0x74, 0x61, - 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, - 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, - 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, - 0x52, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, - 0x6e, 0x67, 0x12, 0x77, 0x0a, 0x19, 0x73, 0x72, 0x63, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, - 0x69, 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, - 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3d, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, - 0x6c, 0x6f, 0x77, 0x2e, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x2e, 0x53, 0x72, 0x63, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x49, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, - 0x6e, 0x74, 0x72, 0x79, 0x52, 0x15, 0x73, 0x72, 0x63, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, - 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x12, 0x79, 0x0a, 0x19, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, - 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3e, + 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x5e, 0x0a, + 0x15, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6c, 0x61, 0x67, 0x73, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x66, 0x6c, 0x61, 0x67, 0x73, 0x12, 0x12, 0x0a, 0x04, + 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, + 0x12, 0x1b, 0x0a, 0x09, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x0d, 0x52, 0x08, 0x64, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x22, 0x95, 0x01, + 0x0a, 0x0f, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0a, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x72, 0x65, 0x6c, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x3c, 0x0a, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, + 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, + 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x52, 0x07, 0x63, 0x6f, + 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x22, 0xb9, 0x0c, 0x0a, 0x15, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, + 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x12, + 0x2a, 0x0a, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, + 0x65, 0x65, 0x72, 0x52, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x34, 0x0a, 0x0b, 0x64, + 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, + 0x50, 0x65, 0x65, 0x72, 0x52, 0x0b, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, + 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x3b, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x12, 0x66, 0x0a, 0x12, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x73, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, - 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x16, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, - 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x12, 0x37, 0x0a, 0x0d, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, - 0x74, 0x61, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, - 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, - 0x72, 0x52, 0x0c, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x50, 0x65, 0x65, 0x72, 0x12, - 0x24, 0x0a, 0x0e, 0x6d, 0x61, 0x78, 0x5f, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, - 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0c, 0x6d, 0x61, 0x78, 0x42, 0x61, 0x74, 0x63, - 0x68, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x26, 0x0a, 0x0f, 0x64, 0x6f, 0x5f, 0x69, 0x6e, 0x69, 0x74, - 0x69, 0x61, 0x6c, 0x5f, 0x63, 0x6f, 0x70, 0x79, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, - 0x64, 0x6f, 0x49, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x43, 0x6f, 0x70, 0x79, 0x12, 0x29, 0x0a, - 0x10, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, - 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x44, 0x0a, 0x1f, 0x73, 0x6e, 0x61, 0x70, - 0x73, 0x68, 0x6f, 0x74, 0x5f, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x70, 0x65, - 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x0c, 0x20, 0x01, 0x28, - 0x0d, 0x52, 0x1b, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x4e, 0x75, 0x6d, 0x52, 0x6f, - 0x77, 0x73, 0x50, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x41, - 0x0a, 0x1d, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x5f, 0x6d, 0x61, 0x78, 0x5f, 0x70, - 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73, 0x18, - 0x0d, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x1a, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x4d, - 0x61, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, - 0x73, 0x12, 0x44, 0x0a, 0x1f, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x5f, 0x6e, 0x75, - 0x6d, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x5f, 0x69, 0x6e, 0x5f, 0x70, 0x61, 0x72, 0x61, - 0x6c, 0x6c, 0x65, 0x6c, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x1b, 0x73, 0x6e, 0x61, 0x70, - 0x73, 0x68, 0x6f, 0x74, 0x4e, 0x75, 0x6d, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x49, 0x6e, 0x50, - 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x12, 0x47, 0x0a, 0x12, 0x73, 0x6e, 0x61, 0x70, 0x73, - 0x68, 0x6f, 0x74, 0x5f, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x0f, 0x20, - 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, - 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x10, - 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, - 0x12, 0x3d, 0x0a, 0x0d, 0x63, 0x64, 0x63, 0x5f, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6d, 0x6f, 0x64, - 0x65, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, - 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, - 0x64, 0x65, 0x52, 0x0b, 0x63, 0x64, 0x63, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x12, - 0x32, 0x0a, 0x15, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x5f, 0x73, 0x74, 0x61, 0x67, - 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x11, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, - 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x53, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x50, - 0x61, 0x74, 0x68, 0x12, 0x28, 0x0a, 0x10, 0x63, 0x64, 0x63, 0x5f, 0x73, 0x74, 0x61, 0x67, 0x69, - 0x6e, 0x67, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x12, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, - 0x64, 0x63, 0x53, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x50, 0x61, 0x74, 0x68, 0x12, 0x1f, 0x0a, - 0x0b, 0x73, 0x6f, 0x66, 0x74, 0x5f, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x18, 0x13, 0x20, 0x01, - 0x28, 0x08, 0x52, 0x0a, 0x73, 0x6f, 0x66, 0x74, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x12, 0x32, - 0x0a, 0x15, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x6c, - 0x6f, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x14, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x72, - 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x6c, 0x6f, 0x74, 0x4e, 0x61, - 0x6d, 0x65, 0x12, 0x26, 0x0a, 0x0f, 0x70, 0x75, 0x73, 0x68, 0x5f, 0x62, 0x61, 0x74, 0x63, 0x68, - 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x15, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x70, 0x75, 0x73, - 0x68, 0x42, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x70, 0x75, - 0x73, 0x68, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x69, 0x73, 0x6d, 0x18, 0x16, - 0x20, 0x01, 0x28, 0x03, 0x52, 0x0f, 0x70, 0x75, 0x73, 0x68, 0x50, 0x61, 0x72, 0x61, 0x6c, 0x6c, - 0x65, 0x6c, 0x69, 0x73, 0x6d, 0x1a, 0x43, 0x0a, 0x15, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, - 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, - 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, - 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x48, 0x0a, 0x1a, 0x53, 0x72, - 0x63, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, - 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, - 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x63, 0x0a, 0x1b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, - 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, - 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2e, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, - 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x05, - 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x30, 0x0a, 0x0f, 0x53, 0x79, 0x6e, - 0x63, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1d, 0x0a, 0x0a, - 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, - 0x52, 0x09, 0x62, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x22, 0x35, 0x0a, 0x14, 0x4e, - 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, - 0x6f, 0x6e, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, - 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x62, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, - 0x7a, 0x65, 0x22, 0x71, 0x0a, 0x0d, 0x4c, 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x53, 0x74, - 0x61, 0x74, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, - 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, - 0x69, 0x6e, 0x74, 0x12, 0x40, 0x0a, 0x0e, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x73, 0x79, 0x6e, 0x63, - 0x65, 0x64, 0x5f, 0x61, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, - 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, - 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0c, 0x6c, 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, - 0x63, 0x65, 0x64, 0x41, 0x74, 0x22, 0xfa, 0x01, 0x0a, 0x0e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x46, - 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x42, 0x0a, 0x0f, 0x6c, 0x61, 0x73, 0x74, - 0x5f, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, - 0x4c, 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x0d, 0x6c, - 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x5a, 0x0a, 0x17, - 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, - 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, + 0x67, 0x73, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, + 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, + 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x12, 0x77, 0x0a, 0x19, 0x73, 0x72, + 0x63, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, + 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3d, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, - 0x73, 0x52, 0x15, 0x66, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x12, 0x48, 0x0a, 0x11, 0x73, 0x79, 0x6e, 0x63, - 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, - 0x77, 0x2e, 0x53, 0x79, 0x6e, 0x63, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, - 0x73, 0x52, 0x0f, 0x73, 0x79, 0x6e, 0x63, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, - 0x6e, 0x73, 0x22, 0x71, 0x0a, 0x13, 0x53, 0x74, 0x61, 0x72, 0x74, 0x4e, 0x6f, 0x72, 0x6d, 0x61, - 0x6c, 0x69, 0x7a, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x5a, 0x0a, 0x17, 0x66, 0x6c, 0x6f, - 0x77, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, - 0x66, 0x69, 0x67, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x65, 0x65, - 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, - 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x52, 0x15, - 0x66, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x73, 0x22, 0x84, 0x01, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x4c, 0x61, 0x73, - 0x74, 0x53, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x49, 0x44, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, + 0x73, 0x2e, 0x53, 0x72, 0x63, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x4e, 0x61, 0x6d, 0x65, + 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x15, 0x73, 0x72, + 0x63, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, + 0x69, 0x6e, 0x67, 0x12, 0x79, 0x0a, 0x19, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, + 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, + 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, + 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, + 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x16, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, + 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x12, 0x37, + 0x0a, 0x0d, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, + 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, + 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0c, 0x6d, 0x65, 0x74, 0x61, 0x64, + 0x61, 0x74, 0x61, 0x50, 0x65, 0x65, 0x72, 0x12, 0x24, 0x0a, 0x0e, 0x6d, 0x61, 0x78, 0x5f, 0x62, + 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0d, 0x52, + 0x0c, 0x6d, 0x61, 0x78, 0x42, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x26, 0x0a, + 0x0f, 0x64, 0x6f, 0x5f, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x63, 0x6f, 0x70, 0x79, + 0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x64, 0x6f, 0x49, 0x6e, 0x69, 0x74, 0x69, 0x61, + 0x6c, 0x43, 0x6f, 0x70, 0x79, 0x12, 0x29, 0x0a, 0x10, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0f, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, + 0x12, 0x44, 0x0a, 0x1f, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x5f, 0x6e, 0x75, 0x6d, + 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x70, 0x65, 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x1b, 0x73, 0x6e, 0x61, 0x70, 0x73, + 0x68, 0x6f, 0x74, 0x4e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x50, 0x65, 0x72, 0x50, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x41, 0x0a, 0x1d, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, + 0x6f, 0x74, 0x5f, 0x6d, 0x61, 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x5f, + 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x1a, 0x73, + 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x4d, 0x61, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6c, 0x6c, + 0x65, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73, 0x12, 0x44, 0x0a, 0x1f, 0x73, 0x6e, 0x61, + 0x70, 0x73, 0x68, 0x6f, 0x74, 0x5f, 0x6e, 0x75, 0x6d, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, + 0x5f, 0x69, 0x6e, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x18, 0x0e, 0x20, 0x01, + 0x28, 0x0d, 0x52, 0x1b, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x4e, 0x75, 0x6d, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x73, 0x49, 0x6e, 0x50, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x12, + 0x47, 0x0a, 0x12, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x5f, 0x73, 0x79, 0x6e, 0x63, + 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x53, 0x79, + 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x10, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, + 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x3d, 0x0a, 0x0d, 0x63, 0x64, 0x63, 0x5f, + 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0e, 0x32, + 0x19, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, + 0x65, 0x70, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x0b, 0x63, 0x64, 0x63, 0x53, + 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x32, 0x0a, 0x15, 0x73, 0x6e, 0x61, 0x70, 0x73, + 0x68, 0x6f, 0x74, 0x5f, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x61, 0x74, 0x68, + 0x18, 0x11, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, + 0x53, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x50, 0x61, 0x74, 0x68, 0x12, 0x28, 0x0a, 0x10, 0x63, + 0x64, 0x63, 0x5f, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, + 0x12, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x64, 0x63, 0x53, 0x74, 0x61, 0x67, 0x69, 0x6e, + 0x67, 0x50, 0x61, 0x74, 0x68, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x6f, 0x66, 0x74, 0x5f, 0x64, 0x65, + 0x6c, 0x65, 0x74, 0x65, 0x18, 0x13, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x73, 0x6f, 0x66, 0x74, + 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x12, 0x32, 0x0a, 0x15, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, + 0x14, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x53, 0x6c, 0x6f, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x26, 0x0a, 0x0f, 0x70, 0x75, + 0x73, 0x68, 0x5f, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x15, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x0d, 0x70, 0x75, 0x73, 0x68, 0x42, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, + 0x7a, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x70, 0x75, 0x73, 0x68, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6c, + 0x6c, 0x65, 0x6c, 0x69, 0x73, 0x6d, 0x18, 0x16, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0f, 0x70, 0x75, + 0x73, 0x68, 0x50, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x69, 0x73, 0x6d, 0x1a, 0x43, 0x0a, + 0x15, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, + 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, + 0x38, 0x01, 0x1a, 0x48, 0x0a, 0x1a, 0x53, 0x72, 0x63, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, + 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x03, 0x6b, + 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x63, 0x0a, 0x1b, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, + 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, + 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2e, 0x0a, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, + 0x01, 0x22, 0x8d, 0x02, 0x0a, 0x0f, 0x53, 0x79, 0x6e, 0x63, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, + 0x69, 0x7a, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x62, 0x61, 0x74, 0x63, 0x68, + 0x53, 0x69, 0x7a, 0x65, 0x12, 0x72, 0x0a, 0x18, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, + 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, + 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x79, 0x6e, 0x63, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x52, 0x16, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x67, 0x0a, 0x1b, 0x52, 0x65, 0x6c, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, + 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0d, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x32, 0x0a, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, + 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, + 0x01, 0x22, 0x35, 0x0a, 0x14, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x46, 0x6c, + 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x62, 0x61, 0x74, + 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x62, + 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x22, 0x71, 0x0a, 0x0d, 0x4c, 0x61, 0x73, 0x74, + 0x53, 0x79, 0x6e, 0x63, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x68, 0x65, + 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x63, + 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x40, 0x0a, 0x0e, 0x6c, 0x61, 0x73, + 0x74, 0x5f, 0x73, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0c, 0x6c, + 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x41, 0x74, 0x22, 0xd6, 0x03, 0x0a, 0x0e, + 0x53, 0x74, 0x61, 0x72, 0x74, 0x46, 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x42, + 0x0a, 0x0f, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x73, 0x74, 0x61, 0x74, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, + 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x4c, 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x53, 0x74, + 0x61, 0x74, 0x65, 0x52, 0x0d, 0x6c, 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x12, 0x5a, 0x0a, 0x17, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, + 0x77, 0x2e, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x52, 0x15, 0x66, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, + 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x12, 0x48, + 0x0a, 0x11, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6f, 0x70, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, + 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x79, 0x6e, 0x63, 0x46, 0x6c, 0x6f, 0x77, + 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x0f, 0x73, 0x79, 0x6e, 0x63, 0x46, 0x6c, 0x6f, + 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x71, 0x0a, 0x18, 0x72, 0x65, 0x6c, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x6d, 0x61, 0x70, + 0x70, 0x69, 0x6e, 0x67, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x70, 0x65, 0x65, + 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x46, 0x6c, + 0x6f, 0x77, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x52, 0x16, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x67, 0x0a, 0x1b, 0x52, + 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x4d, 0x61, + 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, + 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x32, 0x0a, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x3a, 0x02, 0x38, 0x01, 0x22, 0x71, 0x0a, 0x13, 0x53, 0x74, 0x61, 0x72, 0x74, 0x4e, 0x6f, 0x72, + 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x5a, 0x0a, 0x17, 0x66, + 0x6c, 0x6f, 0x77, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x46, 0x6c, 0x6f, 0x77, 0x43, + 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, + 0x52, 0x15, 0x66, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x22, 0x84, 0x01, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x4c, + 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x49, 0x44, 0x49, 0x6e, 0x70, 0x75, 0x74, + 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, + 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, + 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xbe, + 0x01, 0x0a, 0x16, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, + 0x6c, 0x69, 0x74, 0x79, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, + 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, + 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, + 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, + 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, + 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x36, 0x0a, 0x17, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, + 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x15, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x22, + 0xc5, 0x01, 0x0a, 0x1b, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, + 0x69, 0x6c, 0x69, 0x74, 0x79, 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, + 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, + 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, + 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x38, 0x0a, + 0x18, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, + 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, + 0x16, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, + 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, 0x22, 0x30, 0x0a, 0x17, 0x50, 0x6f, 0x73, 0x74, 0x67, + 0x72, 0x65, 0x73, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, + 0x65, 0x72, 0x12, 0x15, 0x0a, 0x06, 0x72, 0x65, 0x6c, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0d, 0x52, 0x05, 0x72, 0x65, 0x6c, 0x49, 0x64, 0x22, 0x89, 0x01, 0x0a, 0x0f, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x62, 0x0a, + 0x19, 0x70, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, + 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x24, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x50, + 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, + 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x48, 0x00, 0x52, 0x17, 0x70, 0x6f, 0x73, 0x74, 0x67, 0x72, + 0x65, 0x73, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, + 0x72, 0x42, 0x12, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, + 0x69, 0x66, 0x69, 0x65, 0x72, 0x22, 0x62, 0x0a, 0x17, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, + 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, + 0x12, 0x47, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, + 0x66, 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, + 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, + 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, + 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x22, 0x88, 0x02, 0x0a, 0x1c, 0x45, 0x6e, + 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x42, + 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x7f, 0x0a, 0x18, 0x74, 0x61, + 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x5f, 0x6d, + 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x45, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x45, 0x6e, 0x73, 0x75, 0x72, + 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x42, 0x61, 0x74, 0x63, + 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, + 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x52, 0x16, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, + 0x66, 0x69, 0x65, 0x72, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x67, 0x0a, 0x1b, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x4d, 0x61, + 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, + 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x32, 0x0a, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, + 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x3a, 0x02, 0x38, 0x01, 0x22, 0x9a, 0x04, 0x0a, 0x15, 0x53, 0x65, 0x74, 0x75, 0x70, 0x52, 0x65, + 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xbe, 0x01, 0x0a, - 0x16, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, - 0x74, 0x79, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, - 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, - 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, - 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, - 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, - 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x36, 0x0a, 0x17, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x15, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x22, 0xc5, 0x01, - 0x0a, 0x1b, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, 0x6c, - 0x69, 0x74, 0x79, 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, - 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, - 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, - 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, - 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, - 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x38, 0x0a, 0x18, 0x73, - 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, - 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x16, 0x73, - 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, - 0x66, 0x69, 0x65, 0x72, 0x73, 0x22, 0x30, 0x0a, 0x17, 0x50, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, - 0x73, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, - 0x12, 0x15, 0x0a, 0x06, 0x72, 0x65, 0x6c, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, - 0x52, 0x05, 0x72, 0x65, 0x6c, 0x49, 0x64, 0x22, 0x89, 0x01, 0x0a, 0x0f, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x62, 0x0a, 0x19, 0x70, - 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, - 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, - 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x50, 0x6f, 0x73, - 0x74, 0x67, 0x72, 0x65, 0x73, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, - 0x66, 0x69, 0x65, 0x72, 0x48, 0x00, 0x52, 0x17, 0x70, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, - 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x42, - 0x12, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, - 0x69, 0x65, 0x72, 0x22, 0x62, 0x0a, 0x17, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, - 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x47, - 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, - 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, - 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, - 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x22, 0x88, 0x02, 0x0a, 0x1c, 0x45, 0x6e, 0x73, 0x75, - 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x42, 0x61, 0x74, - 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x7f, 0x0a, 0x18, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x5f, 0x6d, 0x61, 0x70, - 0x70, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x45, 0x2e, 0x70, 0x65, 0x65, - 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, - 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, - 0x75, 0x74, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, - 0x69, 0x66, 0x69, 0x65, 0x72, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, - 0x79, 0x52, 0x16, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, - 0x65, 0x72, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x67, 0x0a, 0x1b, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x4d, 0x61, 0x70, 0x70, - 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x32, 0x0a, 0x05, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, - 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, - 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, - 0x38, 0x01, 0x22, 0x9a, 0x04, 0x0a, 0x15, 0x53, 0x65, 0x74, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6c, - 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, - 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, - 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, - 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, - 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, - 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, - 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x66, 0x0a, 0x12, 0x74, 0x61, - 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, - 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, - 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x65, 0x74, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, - 0x52, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, - 0x6e, 0x67, 0x12, 0x3d, 0x0a, 0x10, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, - 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, - 0x52, 0x0f, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x65, 0x65, - 0x72, 0x12, 0x26, 0x0a, 0x0f, 0x64, 0x6f, 0x5f, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, - 0x63, 0x6f, 0x70, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x64, 0x6f, 0x49, 0x6e, - 0x69, 0x74, 0x69, 0x61, 0x6c, 0x43, 0x6f, 0x70, 0x79, 0x12, 0x3a, 0x0a, 0x19, 0x65, 0x78, 0x69, - 0x73, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x17, 0x65, 0x78, - 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x43, 0x0a, 0x1e, 0x65, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, - 0x67, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x6c, - 0x6f, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1b, 0x65, - 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x53, 0x6c, 0x6f, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0x43, 0x0a, 0x15, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, - 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, - 0x5a, 0x0a, 0x16, 0x53, 0x65, 0x74, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x6c, 0x6f, - 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x73, 0x6c, - 0x6f, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, - 0x6f, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x73, - 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xed, 0x02, 0x0a, 0x13, - 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x61, 0x77, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, + 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x66, 0x0a, 0x12, + 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, + 0x6e, 0x67, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, + 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x65, 0x74, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6c, + 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x52, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, + 0x70, 0x69, 0x6e, 0x67, 0x12, 0x3d, 0x0a, 0x10, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, + 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, + 0x65, 0x72, 0x52, 0x0f, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, + 0x65, 0x65, 0x72, 0x12, 0x26, 0x0a, 0x0f, 0x64, 0x6f, 0x5f, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, + 0x6c, 0x5f, 0x63, 0x6f, 0x70, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x64, 0x6f, + 0x49, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x43, 0x6f, 0x70, 0x79, 0x12, 0x3a, 0x0a, 0x19, 0x65, + 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x17, + 0x65, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x43, 0x0a, 0x1e, 0x65, 0x78, 0x69, 0x73, 0x74, + 0x69, 0x6e, 0x67, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x73, 0x6c, 0x6f, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x1b, 0x65, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x6c, 0x6f, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0x43, 0x0a, 0x15, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, + 0x01, 0x22, 0x5a, 0x0a, 0x16, 0x53, 0x65, 0x74, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x73, + 0x6c, 0x6f, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, + 0x73, 0x6c, 0x6f, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x73, 0x6e, 0x61, 0x70, + 0x73, 0x68, 0x6f, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0c, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xed, 0x02, + 0x0a, 0x13, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x61, 0x77, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, + 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, + 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, + 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, + 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, + 0x61, 0x6d, 0x65, 0x12, 0x64, 0x0a, 0x12, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, + 0x65, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x36, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x43, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x52, 0x61, 0x77, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, + 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, + 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, + 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x12, 0x3d, 0x0a, 0x0d, 0x63, 0x64, 0x63, + 0x5f, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x19, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, + 0x52, 0x65, 0x70, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x0b, 0x63, 0x64, 0x63, + 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x1a, 0x43, 0x0a, 0x15, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, + 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x41, 0x0a, + 0x14, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x61, 0x77, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4f, + 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x29, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, + 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, + 0x22, 0xe3, 0x01, 0x0a, 0x0b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x12, 0x29, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, + 0x66, 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x3f, 0x0a, 0x07, 0x63, + 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x52, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x2c, 0x0a, 0x12, + 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, + 0x6d, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, + 0x79, 0x4b, 0x65, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x1a, 0x3a, 0x0a, 0x0c, 0x43, 0x6f, + 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, + 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x91, 0x01, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, - 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, - 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, - 0x65, 0x12, 0x64, 0x0a, 0x12, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, - 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, - 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x43, 0x72, 0x65, 0x61, - 0x74, 0x65, 0x52, 0x61, 0x77, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, - 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, - 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, - 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x12, 0x3d, 0x0a, 0x0d, 0x63, 0x64, 0x63, 0x5f, 0x73, - 0x79, 0x6e, 0x63, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, - 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, - 0x70, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x0b, 0x63, 0x64, 0x63, 0x53, 0x79, - 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x1a, 0x43, 0x0a, 0x15, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, - 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, - 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, - 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x41, 0x0a, 0x14, 0x43, - 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x61, 0x77, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x75, 0x74, - 0x70, 0x75, 0x74, 0x12, 0x29, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, - 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x22, 0xe3, - 0x01, 0x0a, 0x0b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x29, - 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, - 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, - 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x3f, 0x0a, 0x07, 0x63, 0x6f, 0x6c, - 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x70, 0x65, 0x65, - 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, - 0x79, 0x52, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x2c, 0x0a, 0x12, 0x70, 0x72, - 0x69, 0x6d, 0x61, 0x72, 0x79, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x4b, - 0x65, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x1a, 0x3a, 0x0a, 0x0c, 0x43, 0x6f, 0x6c, 0x75, - 0x6d, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, - 0x3a, 0x02, 0x38, 0x01, 0x22, 0x91, 0x01, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, - 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, - 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x2b, 0x0a, 0x11, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, - 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, - 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, 0x22, 0xff, 0x01, 0x0a, 0x19, 0x47, 0x65, 0x74, - 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x61, 0x74, 0x63, 0x68, - 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x7d, 0x0a, 0x19, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, - 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x6d, 0x61, 0x70, 0x70, - 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x42, 0x2e, 0x70, 0x65, 0x65, 0x72, - 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, - 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x16, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, - 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x63, 0x0a, 0x1b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, - 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, - 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2e, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, - 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xda, 0x01, 0x0a, 0x19, 0x53, - 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, - 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, - 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x12, 0x29, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, - 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, - 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x48, 0x0a, - 0x13, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x65, 0x65, - 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x52, 0x11, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0xd4, 0x02, 0x0a, 0x1e, 0x53, 0x65, 0x74, 0x75, - 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, + 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x2b, 0x0a, + 0x11, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, + 0x72, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, + 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, 0x22, 0xff, 0x01, 0x0a, 0x19, 0x47, + 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x61, 0x74, + 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x7d, 0x0a, 0x19, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x6d, 0x61, + 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x42, 0x2e, 0x70, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, + 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, + 0x16, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x63, 0x0a, 0x1b, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, + 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2e, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, + 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, + 0x61, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xda, 0x01, 0x0a, + 0x19, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x12, 0x82, 0x01, 0x0a, 0x19, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, - 0x61, 0x6d, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, - 0x6e, 0x67, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x47, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, - 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, - 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, - 0x79, 0x52, 0x16, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, - 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x63, 0x0a, 0x1b, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, - 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2e, 0x0a, 0x05, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x65, 0x65, 0x72, - 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, - 0x65, 0x6d, 0x61, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x6e, - 0x0a, 0x1a, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, - 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x29, 0x0a, 0x10, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, - 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x25, 0x0a, 0x0e, 0x61, 0x6c, 0x72, 0x65, 0x61, - 0x64, 0x79, 0x5f, 0x65, 0x78, 0x69, 0x73, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, - 0x0d, 0x61, 0x6c, 0x72, 0x65, 0x61, 0x64, 0x79, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x22, 0xe0, - 0x01, 0x0a, 0x1f, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, - 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, - 0x75, 0x74, 0x12, 0x76, 0x0a, 0x14, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x65, 0x78, 0x69, 0x73, - 0x74, 0x73, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x44, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, - 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x2e, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, - 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x12, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, - 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x45, 0x0a, 0x17, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, - 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, - 0x01, 0x22, 0x3b, 0x0a, 0x11, 0x49, 0x6e, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x10, 0x0a, 0x03, - 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0x79, - 0x0a, 0x17, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x30, 0x0a, 0x05, 0x73, 0x74, 0x61, - 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, - 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, - 0x74, 0x61, 0x6d, 0x70, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x2c, 0x0a, 0x03, 0x65, - 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, - 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, - 0x74, 0x61, 0x6d, 0x70, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0x4d, 0x0a, 0x03, 0x54, 0x49, 0x44, - 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0b, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x4e, 0x75, 0x6d, - 0x62, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x5f, 0x6e, 0x75, - 0x6d, 0x62, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0c, 0x6f, 0x66, 0x66, 0x73, - 0x65, 0x74, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x22, 0x5f, 0x0a, 0x11, 0x54, 0x49, 0x44, 0x50, - 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x26, 0x0a, - 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x70, - 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x49, 0x44, 0x52, 0x05, - 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x22, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, - 0x2e, 0x54, 0x49, 0x44, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0xe8, 0x01, 0x0a, 0x0e, 0x50, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x3d, 0x0a, 0x09, - 0x69, 0x6e, 0x74, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x49, 0x6e, - 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x48, - 0x00, 0x52, 0x08, 0x69, 0x6e, 0x74, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x4f, 0x0a, 0x0f, 0x74, - 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, - 0x6f, 0x77, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x50, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x48, 0x00, 0x52, 0x0e, 0x74, 0x69, - 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x3d, 0x0a, 0x09, - 0x74, 0x69, 0x64, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x49, - 0x44, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x48, - 0x00, 0x52, 0x08, 0x74, 0x69, 0x64, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x42, 0x07, 0x0a, 0x05, 0x72, - 0x61, 0x6e, 0x67, 0x65, 0x22, 0x78, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, - 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x39, 0x0a, 0x0a, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x74, - 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, - 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, - 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x77, 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, - 0x12, 0x2c, 0x0a, 0x12, 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x63, - 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x75, 0x70, - 0x73, 0x65, 0x72, 0x74, 0x4b, 0x65, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x22, 0x96, - 0x06, 0x0a, 0x0a, 0x51, 0x52, 0x65, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, - 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, - 0x65, 0x12, 0x33, 0x0a, 0x0b, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x70, 0x65, 0x65, 0x72, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, - 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0a, 0x73, 0x6f, 0x75, 0x72, - 0x63, 0x65, 0x50, 0x65, 0x65, 0x72, 0x12, 0x3d, 0x0a, 0x10, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, - 0x50, 0x65, 0x65, 0x72, 0x52, 0x0f, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x50, 0x65, 0x65, 0x72, 0x12, 0x40, 0x0a, 0x1c, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, - 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1a, 0x64, 0x65, 0x73, - 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, - 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x14, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, - 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x12, 0x27, 0x0a, - 0x0f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, - 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, - 0x6b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, - 0x61, 0x72, 0x6b, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x0f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x43, 0x6f, 0x6c, 0x75, 0x6d, - 0x6e, 0x12, 0x2a, 0x0a, 0x11, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x63, 0x6f, 0x70, - 0x79, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x69, 0x6e, - 0x69, 0x74, 0x69, 0x61, 0x6c, 0x43, 0x6f, 0x70, 0x79, 0x4f, 0x6e, 0x6c, 0x79, 0x12, 0x36, 0x0a, - 0x09, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0e, - 0x32, 0x19, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, - 0x52, 0x65, 0x70, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x08, 0x73, 0x79, 0x6e, - 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x24, 0x0a, 0x0e, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, - 0x69, 0x7a, 0x65, 0x5f, 0x69, 0x6e, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0c, 0x62, - 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x49, 0x6e, 0x74, 0x12, 0x34, 0x0a, 0x16, 0x62, - 0x61, 0x74, 0x63, 0x68, 0x5f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x65, - 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x14, 0x62, 0x61, 0x74, - 0x63, 0x68, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, - 0x73, 0x12, 0x30, 0x0a, 0x14, 0x6d, 0x61, 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, - 0x6c, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0d, 0x52, - 0x12, 0x6d, 0x61, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x57, 0x6f, 0x72, 0x6b, - 0x65, 0x72, 0x73, 0x12, 0x3f, 0x0a, 0x1c, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x62, 0x65, 0x74, 0x77, - 0x65, 0x65, 0x6e, 0x5f, 0x62, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x5f, 0x73, 0x65, 0x63, 0x6f, - 0x6e, 0x64, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x19, 0x77, 0x61, 0x69, 0x74, 0x42, - 0x65, 0x74, 0x77, 0x65, 0x65, 0x6e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x53, 0x65, 0x63, - 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x39, 0x0a, 0x0a, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x6d, 0x6f, - 0x64, 0x65, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, - 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x09, 0x77, 0x72, 0x69, 0x74, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x12, - 0x21, 0x0a, 0x0c, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, - 0x0f, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x50, 0x61, - 0x74, 0x68, 0x12, 0x33, 0x0a, 0x16, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x70, - 0x65, 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x10, 0x20, 0x01, - 0x28, 0x0d, 0x52, 0x13, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x50, 0x65, 0x72, 0x50, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x97, 0x01, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, - 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x31, 0x0a, 0x05, - 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x65, - 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x12, - 0x30, 0x0a, 0x14, 0x66, 0x75, 0x6c, 0x6c, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x70, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x66, - 0x75, 0x6c, 0x6c, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x22, 0x6b, 0x0a, 0x12, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x12, 0x19, 0x0a, 0x08, 0x62, 0x61, 0x74, 0x63, 0x68, - 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x62, 0x61, 0x74, 0x63, 0x68, - 0x49, 0x64, 0x12, 0x3a, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, - 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, - 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x50, - 0x0a, 0x12, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, - 0x73, 0x75, 0x6c, 0x74, 0x12, 0x3a, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, + 0x6e, 0x66, 0x69, 0x67, 0x12, 0x29, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, + 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, + 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, + 0x48, 0x0a, 0x13, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, + 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x11, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0xd4, 0x02, 0x0a, 0x1e, 0x53, 0x65, + 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, + 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, + 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x82, 0x01, 0x0a, 0x19, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x6d, 0x61, 0x70, + 0x70, 0x69, 0x6e, 0x67, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x47, 0x2e, 0x70, 0x65, 0x65, + 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, + 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, 0x74, + 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, + 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x52, 0x16, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x63, 0x0a, 0x1b, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, + 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, + 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2e, 0x0a, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, + 0x22, 0x6e, 0x0a, 0x1a, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, + 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x29, + 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, + 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, + 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x25, 0x0a, 0x0e, 0x61, 0x6c, 0x72, + 0x65, 0x61, 0x64, 0x79, 0x5f, 0x65, 0x78, 0x69, 0x73, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x0d, 0x61, 0x6c, 0x72, 0x65, 0x61, 0x64, 0x79, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, + 0x22, 0xe0, 0x01, 0x0a, 0x1f, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, + 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, + 0x74, 0x70, 0x75, 0x74, 0x12, 0x76, 0x0a, 0x14, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x65, 0x78, + 0x69, 0x73, 0x74, 0x73, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x44, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, + 0x2e, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, + 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, + 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x12, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x45, + 0x78, 0x69, 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x45, 0x0a, 0x17, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, + 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, + 0x02, 0x38, 0x01, 0x22, 0x3b, 0x0a, 0x11, 0x49, 0x6e, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x72, + 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x10, + 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x65, 0x6e, 0x64, + 0x22, 0x79, 0x0a, 0x17, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x50, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x30, 0x0a, 0x05, 0x73, + 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, + 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, + 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x2c, 0x0a, + 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, + 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, + 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0x4d, 0x0a, 0x03, 0x54, + 0x49, 0x44, 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x6e, 0x75, 0x6d, 0x62, + 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0b, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x4e, + 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x5f, + 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0c, 0x6f, 0x66, + 0x66, 0x73, 0x65, 0x74, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x22, 0x5f, 0x0a, 0x11, 0x54, 0x49, + 0x44, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, + 0x26, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, + 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x49, 0x44, + 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x22, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, + 0x6f, 0x77, 0x2e, 0x54, 0x49, 0x44, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0xe8, 0x01, 0x0a, 0x0e, + 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x3d, + 0x0a, 0x09, 0x69, 0x6e, 0x74, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, + 0x49, 0x6e, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, + 0x65, 0x48, 0x00, 0x52, 0x08, 0x69, 0x6e, 0x74, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x4f, 0x0a, + 0x0f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, + 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x50, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x48, 0x00, 0x52, 0x0e, + 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x3d, + 0x0a, 0x09, 0x74, 0x69, 0x64, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, + 0x54, 0x49, 0x44, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, + 0x65, 0x48, 0x00, 0x52, 0x08, 0x74, 0x69, 0x64, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x42, 0x07, 0x0a, + 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x22, 0x78, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, + 0x69, 0x74, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x39, 0x0a, 0x0a, 0x77, 0x72, 0x69, 0x74, 0x65, + 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1a, 0x2e, 0x70, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, + 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x77, 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, + 0x70, 0x65, 0x12, 0x2c, 0x0a, 0x12, 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x6b, 0x65, 0x79, + 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, + 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x4b, 0x65, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, + 0x22, 0x96, 0x06, 0x0a, 0x0a, 0x51, 0x52, 0x65, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, + 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, + 0x61, 0x6d, 0x65, 0x12, 0x33, 0x0a, 0x0b, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x70, 0x65, + 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, + 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0a, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x50, 0x65, 0x65, 0x72, 0x12, 0x3d, 0x0a, 0x10, 0x64, 0x65, 0x73, 0x74, + 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, + 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0f, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x50, 0x65, 0x65, 0x72, 0x12, 0x40, 0x0a, 0x1c, 0x64, 0x65, 0x73, 0x74, 0x69, + 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, + 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1a, 0x64, + 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, + 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x14, 0x0a, 0x05, 0x71, 0x75, 0x65, + 0x72, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x12, + 0x27, 0x0a, 0x0f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, + 0x61, 0x72, 0x6b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x77, 0x61, 0x74, 0x65, + 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x07, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x43, 0x6f, 0x6c, + 0x75, 0x6d, 0x6e, 0x12, 0x2a, 0x0a, 0x11, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x63, + 0x6f, 0x70, 0x79, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, + 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x43, 0x6f, 0x70, 0x79, 0x4f, 0x6e, 0x6c, 0x79, 0x12, + 0x36, 0x0a, 0x09, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x09, 0x20, 0x01, + 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, + 0x2e, 0x51, 0x52, 0x65, 0x70, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x08, 0x73, + 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x24, 0x0a, 0x0e, 0x62, 0x61, 0x74, 0x63, 0x68, + 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, 0x69, 0x6e, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0d, 0x52, + 0x0c, 0x62, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x49, 0x6e, 0x74, 0x12, 0x34, 0x0a, + 0x16, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x14, 0x62, + 0x61, 0x74, 0x63, 0x68, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x63, 0x6f, + 0x6e, 0x64, 0x73, 0x12, 0x30, 0x0a, 0x14, 0x6d, 0x61, 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6c, + 0x6c, 0x65, 0x6c, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73, 0x18, 0x0c, 0x20, 0x01, 0x28, + 0x0d, 0x52, 0x12, 0x6d, 0x61, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x57, 0x6f, + 0x72, 0x6b, 0x65, 0x72, 0x73, 0x12, 0x3f, 0x0a, 0x1c, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x62, 0x65, + 0x74, 0x77, 0x65, 0x65, 0x6e, 0x5f, 0x62, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x5f, 0x73, 0x65, + 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x19, 0x77, 0x61, 0x69, + 0x74, 0x42, 0x65, 0x74, 0x77, 0x65, 0x65, 0x6e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x53, + 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x39, 0x0a, 0x0a, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, + 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, + 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, + 0x74, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x09, 0x77, 0x72, 0x69, 0x74, 0x65, 0x4d, 0x6f, 0x64, + 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x61, 0x74, + 0x68, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, + 0x50, 0x61, 0x74, 0x68, 0x12, 0x33, 0x0a, 0x16, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, + 0x5f, 0x70, 0x65, 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x10, + 0x20, 0x01, 0x28, 0x0d, 0x52, 0x13, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x50, 0x65, 0x72, + 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x97, 0x01, 0x0a, 0x0d, 0x51, 0x52, + 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c, 0x70, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x31, + 0x0a, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, + 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x50, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x05, 0x72, 0x61, 0x6e, 0x67, + 0x65, 0x12, 0x30, 0x0a, 0x14, 0x66, 0x75, 0x6c, 0x6c, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, + 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x12, 0x66, 0x75, 0x6c, 0x6c, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x22, 0x6b, 0x0a, 0x12, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x12, 0x19, 0x0a, 0x08, 0x62, 0x61, 0x74, + 0x63, 0x68, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x62, 0x61, 0x74, + 0x63, 0x68, 0x49, 0x64, 0x12, 0x3a, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, - 0x22, 0x2c, 0x0a, 0x0d, 0x44, 0x72, 0x6f, 0x70, 0x46, 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x70, 0x75, - 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x66, 0x6c, 0x6f, 0x77, 0x4e, 0x61, 0x6d, 0x65, 0x2a, 0x50, - 0x0a, 0x0c, 0x51, 0x52, 0x65, 0x70, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x1f, + 0x22, 0x50, 0x0a, 0x12, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x3a, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, + 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x22, 0x2c, 0x0a, 0x0d, 0x44, 0x72, 0x6f, 0x70, 0x46, 0x6c, 0x6f, 0x77, 0x49, 0x6e, + 0x70, 0x75, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x66, 0x6c, 0x6f, 0x77, 0x4e, 0x61, 0x6d, 0x65, + 0x22, 0x54, 0x0a, 0x10, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x41, 0x64, 0x64, 0x65, 0x64, 0x43, 0x6f, + 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, + 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, + 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, + 0x6d, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x22, 0xcb, 0x01, 0x0a, 0x10, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x12, 0x24, 0x0a, 0x0e, 0x73, + 0x72, 0x63, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x72, 0x63, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, + 0x65, 0x12, 0x24, 0x0a, 0x0e, 0x64, 0x73, 0x74, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x64, 0x73, 0x74, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x42, 0x0a, 0x0d, 0x61, 0x64, 0x64, 0x65, 0x64, + 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, + 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x44, 0x65, 0x6c, + 0x74, 0x61, 0x41, 0x64, 0x64, 0x65, 0x64, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x52, 0x0c, 0x61, + 0x64, 0x64, 0x65, 0x64, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x27, 0x0a, 0x0f, 0x64, + 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x04, + 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x64, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x43, 0x6f, 0x6c, + 0x75, 0x6d, 0x6e, 0x73, 0x22, 0xc6, 0x01, 0x0a, 0x1b, 0x52, 0x65, 0x70, 0x6c, 0x61, 0x79, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x49, + 0x6e, 0x70, 0x75, 0x74, 0x12, 0x5a, 0x0a, 0x17, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x63, 0x6f, 0x6e, + 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, + 0x6c, 0x6f, 0x77, 0x2e, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x52, 0x15, 0x66, 0x6c, 0x6f, 0x77, 0x43, + 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, + 0x12, 0x4b, 0x0a, 0x12, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x5f, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x52, 0x10, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x2a, 0x50, 0x0a, + 0x0c, 0x51, 0x52, 0x65, 0x70, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x1f, 0x0a, + 0x1b, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x53, 0x59, 0x4e, 0x43, 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, + 0x4d, 0x55, 0x4c, 0x54, 0x49, 0x5f, 0x49, 0x4e, 0x53, 0x45, 0x52, 0x54, 0x10, 0x00, 0x12, 0x1f, 0x0a, 0x1b, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x53, 0x59, 0x4e, 0x43, 0x5f, 0x4d, 0x4f, 0x44, 0x45, - 0x5f, 0x4d, 0x55, 0x4c, 0x54, 0x49, 0x5f, 0x49, 0x4e, 0x53, 0x45, 0x52, 0x54, 0x10, 0x00, 0x12, - 0x1f, 0x0a, 0x1b, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x53, 0x59, 0x4e, 0x43, 0x5f, 0x4d, 0x4f, 0x44, - 0x45, 0x5f, 0x53, 0x54, 0x4f, 0x52, 0x41, 0x47, 0x45, 0x5f, 0x41, 0x56, 0x52, 0x4f, 0x10, 0x01, - 0x2a, 0x47, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, - 0x65, 0x12, 0x1a, 0x0a, 0x16, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x5f, - 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x41, 0x50, 0x50, 0x45, 0x4e, 0x44, 0x10, 0x00, 0x12, 0x1a, 0x0a, - 0x16, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x5f, 0x4d, 0x4f, 0x44, 0x45, - 0x5f, 0x55, 0x50, 0x53, 0x45, 0x52, 0x54, 0x10, 0x01, 0x42, 0x76, 0x0a, 0x0f, 0x63, 0x6f, 0x6d, - 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x42, 0x09, 0x46, 0x6c, - 0x6f, 0x77, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x10, 0x67, 0x65, 0x6e, 0x65, 0x72, - 0x61, 0x74, 0x65, 0x64, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x73, 0xa2, 0x02, 0x03, 0x50, 0x58, - 0x58, 0xaa, 0x02, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0xca, 0x02, - 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0xe2, 0x02, 0x16, 0x50, 0x65, - 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, - 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, - 0x77, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x5f, 0x53, 0x54, 0x4f, 0x52, 0x41, 0x47, 0x45, 0x5f, 0x41, 0x56, 0x52, 0x4f, 0x10, 0x01, 0x2a, + 0x47, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, + 0x12, 0x1a, 0x0a, 0x16, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x5f, 0x4d, + 0x4f, 0x44, 0x45, 0x5f, 0x41, 0x50, 0x50, 0x45, 0x4e, 0x44, 0x10, 0x00, 0x12, 0x1a, 0x0a, 0x16, + 0x51, 0x52, 0x45, 0x50, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, + 0x55, 0x50, 0x53, 0x45, 0x52, 0x54, 0x10, 0x01, 0x42, 0x76, 0x0a, 0x0f, 0x63, 0x6f, 0x6d, 0x2e, + 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x42, 0x09, 0x46, 0x6c, 0x6f, + 0x77, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x10, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, + 0x74, 0x65, 0x64, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x73, 0xa2, 0x02, 0x03, 0x50, 0x58, 0x58, + 0xaa, 0x02, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0xca, 0x02, 0x0a, + 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0xe2, 0x02, 0x16, 0x50, 0x65, 0x65, + 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, + 0x61, 0x74, 0x61, 0xea, 0x02, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, + 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -2989,118 +3386,133 @@ func file_flow_proto_rawDescGZIP() []byte { } var file_flow_proto_enumTypes = make([]protoimpl.EnumInfo, 2) -var file_flow_proto_msgTypes = make([]protoimpl.MessageInfo, 46) +var file_flow_proto_msgTypes = make([]protoimpl.MessageInfo, 53) var file_flow_proto_goTypes = []interface{}{ (QRepSyncMode)(0), // 0: peerdb_flow.QRepSyncMode (QRepWriteType)(0), // 1: peerdb_flow.QRepWriteType (*TableNameMapping)(nil), // 2: peerdb_flow.TableNameMapping - (*FlowConnectionConfigs)(nil), // 3: peerdb_flow.FlowConnectionConfigs - (*SyncFlowOptions)(nil), // 4: peerdb_flow.SyncFlowOptions - (*NormalizeFlowOptions)(nil), // 5: peerdb_flow.NormalizeFlowOptions - (*LastSyncState)(nil), // 6: peerdb_flow.LastSyncState - (*StartFlowInput)(nil), // 7: peerdb_flow.StartFlowInput - (*StartNormalizeInput)(nil), // 8: peerdb_flow.StartNormalizeInput - (*GetLastSyncedIDInput)(nil), // 9: peerdb_flow.GetLastSyncedIDInput - (*EnsurePullabilityInput)(nil), // 10: peerdb_flow.EnsurePullabilityInput - (*EnsurePullabilityBatchInput)(nil), // 11: peerdb_flow.EnsurePullabilityBatchInput - (*PostgresTableIdentifier)(nil), // 12: peerdb_flow.PostgresTableIdentifier - (*TableIdentifier)(nil), // 13: peerdb_flow.TableIdentifier - (*EnsurePullabilityOutput)(nil), // 14: peerdb_flow.EnsurePullabilityOutput - (*EnsurePullabilityBatchOutput)(nil), // 15: peerdb_flow.EnsurePullabilityBatchOutput - (*SetupReplicationInput)(nil), // 16: peerdb_flow.SetupReplicationInput - (*SetupReplicationOutput)(nil), // 17: peerdb_flow.SetupReplicationOutput - (*CreateRawTableInput)(nil), // 18: peerdb_flow.CreateRawTableInput - (*CreateRawTableOutput)(nil), // 19: peerdb_flow.CreateRawTableOutput - (*TableSchema)(nil), // 20: peerdb_flow.TableSchema - (*GetTableSchemaBatchInput)(nil), // 21: peerdb_flow.GetTableSchemaBatchInput - (*GetTableSchemaBatchOutput)(nil), // 22: peerdb_flow.GetTableSchemaBatchOutput - (*SetupNormalizedTableInput)(nil), // 23: peerdb_flow.SetupNormalizedTableInput - (*SetupNormalizedTableBatchInput)(nil), // 24: peerdb_flow.SetupNormalizedTableBatchInput - (*SetupNormalizedTableOutput)(nil), // 25: peerdb_flow.SetupNormalizedTableOutput - (*SetupNormalizedTableBatchOutput)(nil), // 26: peerdb_flow.SetupNormalizedTableBatchOutput - (*IntPartitionRange)(nil), // 27: peerdb_flow.IntPartitionRange - (*TimestampPartitionRange)(nil), // 28: peerdb_flow.TimestampPartitionRange - (*TID)(nil), // 29: peerdb_flow.TID - (*TIDPartitionRange)(nil), // 30: peerdb_flow.TIDPartitionRange - (*PartitionRange)(nil), // 31: peerdb_flow.PartitionRange - (*QRepWriteMode)(nil), // 32: peerdb_flow.QRepWriteMode - (*QRepConfig)(nil), // 33: peerdb_flow.QRepConfig - (*QRepPartition)(nil), // 34: peerdb_flow.QRepPartition - (*QRepPartitionBatch)(nil), // 35: peerdb_flow.QRepPartitionBatch - (*QRepParitionResult)(nil), // 36: peerdb_flow.QRepParitionResult - (*DropFlowInput)(nil), // 37: peerdb_flow.DropFlowInput - nil, // 38: peerdb_flow.FlowConnectionConfigs.TableNameMappingEntry - nil, // 39: peerdb_flow.FlowConnectionConfigs.SrcTableIdNameMappingEntry - nil, // 40: peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry - nil, // 41: peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry - nil, // 42: peerdb_flow.SetupReplicationInput.TableNameMappingEntry - nil, // 43: peerdb_flow.CreateRawTableInput.TableNameMappingEntry - nil, // 44: peerdb_flow.TableSchema.ColumnsEntry - nil, // 45: peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry - nil, // 46: peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry - nil, // 47: peerdb_flow.SetupNormalizedTableBatchOutput.TableExistsMappingEntry - (*Peer)(nil), // 48: peerdb_peers.Peer - (*timestamppb.Timestamp)(nil), // 49: google.protobuf.Timestamp + (*RelationMessageColumn)(nil), // 3: peerdb_flow.RelationMessageColumn + (*RelationMessage)(nil), // 4: peerdb_flow.RelationMessage + (*FlowConnectionConfigs)(nil), // 5: peerdb_flow.FlowConnectionConfigs + (*SyncFlowOptions)(nil), // 6: peerdb_flow.SyncFlowOptions + (*NormalizeFlowOptions)(nil), // 7: peerdb_flow.NormalizeFlowOptions + (*LastSyncState)(nil), // 8: peerdb_flow.LastSyncState + (*StartFlowInput)(nil), // 9: peerdb_flow.StartFlowInput + (*StartNormalizeInput)(nil), // 10: peerdb_flow.StartNormalizeInput + (*GetLastSyncedIDInput)(nil), // 11: peerdb_flow.GetLastSyncedIDInput + (*EnsurePullabilityInput)(nil), // 12: peerdb_flow.EnsurePullabilityInput + (*EnsurePullabilityBatchInput)(nil), // 13: peerdb_flow.EnsurePullabilityBatchInput + (*PostgresTableIdentifier)(nil), // 14: peerdb_flow.PostgresTableIdentifier + (*TableIdentifier)(nil), // 15: peerdb_flow.TableIdentifier + (*EnsurePullabilityOutput)(nil), // 16: peerdb_flow.EnsurePullabilityOutput + (*EnsurePullabilityBatchOutput)(nil), // 17: peerdb_flow.EnsurePullabilityBatchOutput + (*SetupReplicationInput)(nil), // 18: peerdb_flow.SetupReplicationInput + (*SetupReplicationOutput)(nil), // 19: peerdb_flow.SetupReplicationOutput + (*CreateRawTableInput)(nil), // 20: peerdb_flow.CreateRawTableInput + (*CreateRawTableOutput)(nil), // 21: peerdb_flow.CreateRawTableOutput + (*TableSchema)(nil), // 22: peerdb_flow.TableSchema + (*GetTableSchemaBatchInput)(nil), // 23: peerdb_flow.GetTableSchemaBatchInput + (*GetTableSchemaBatchOutput)(nil), // 24: peerdb_flow.GetTableSchemaBatchOutput + (*SetupNormalizedTableInput)(nil), // 25: peerdb_flow.SetupNormalizedTableInput + (*SetupNormalizedTableBatchInput)(nil), // 26: peerdb_flow.SetupNormalizedTableBatchInput + (*SetupNormalizedTableOutput)(nil), // 27: peerdb_flow.SetupNormalizedTableOutput + (*SetupNormalizedTableBatchOutput)(nil), // 28: peerdb_flow.SetupNormalizedTableBatchOutput + (*IntPartitionRange)(nil), // 29: peerdb_flow.IntPartitionRange + (*TimestampPartitionRange)(nil), // 30: peerdb_flow.TimestampPartitionRange + (*TID)(nil), // 31: peerdb_flow.TID + (*TIDPartitionRange)(nil), // 32: peerdb_flow.TIDPartitionRange + (*PartitionRange)(nil), // 33: peerdb_flow.PartitionRange + (*QRepWriteMode)(nil), // 34: peerdb_flow.QRepWriteMode + (*QRepConfig)(nil), // 35: peerdb_flow.QRepConfig + (*QRepPartition)(nil), // 36: peerdb_flow.QRepPartition + (*QRepPartitionBatch)(nil), // 37: peerdb_flow.QRepPartitionBatch + (*QRepParitionResult)(nil), // 38: peerdb_flow.QRepParitionResult + (*DropFlowInput)(nil), // 39: peerdb_flow.DropFlowInput + (*DeltaAddedColumn)(nil), // 40: peerdb_flow.DeltaAddedColumn + (*TableSchemaDelta)(nil), // 41: peerdb_flow.TableSchemaDelta + (*ReplayTableSchemaDeltaInput)(nil), // 42: peerdb_flow.ReplayTableSchemaDeltaInput + nil, // 43: peerdb_flow.FlowConnectionConfigs.TableNameMappingEntry + nil, // 44: peerdb_flow.FlowConnectionConfigs.SrcTableIdNameMappingEntry + nil, // 45: peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry + nil, // 46: peerdb_flow.SyncFlowOptions.RelationMessageMappingEntry + nil, // 47: peerdb_flow.StartFlowInput.RelationMessageMappingEntry + nil, // 48: peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry + nil, // 49: peerdb_flow.SetupReplicationInput.TableNameMappingEntry + nil, // 50: peerdb_flow.CreateRawTableInput.TableNameMappingEntry + nil, // 51: peerdb_flow.TableSchema.ColumnsEntry + nil, // 52: peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry + nil, // 53: peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry + nil, // 54: peerdb_flow.SetupNormalizedTableBatchOutput.TableExistsMappingEntry + (*Peer)(nil), // 55: peerdb_peers.Peer + (*timestamppb.Timestamp)(nil), // 56: google.protobuf.Timestamp } var file_flow_proto_depIdxs = []int32{ - 48, // 0: peerdb_flow.FlowConnectionConfigs.source:type_name -> peerdb_peers.Peer - 48, // 1: peerdb_flow.FlowConnectionConfigs.destination:type_name -> peerdb_peers.Peer - 20, // 2: peerdb_flow.FlowConnectionConfigs.table_schema:type_name -> peerdb_flow.TableSchema - 38, // 3: peerdb_flow.FlowConnectionConfigs.table_name_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.TableNameMappingEntry - 39, // 4: peerdb_flow.FlowConnectionConfigs.src_table_id_name_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.SrcTableIdNameMappingEntry - 40, // 5: peerdb_flow.FlowConnectionConfigs.table_name_schema_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry - 48, // 6: peerdb_flow.FlowConnectionConfigs.metadata_peer:type_name -> peerdb_peers.Peer - 0, // 7: peerdb_flow.FlowConnectionConfigs.snapshot_sync_mode:type_name -> peerdb_flow.QRepSyncMode - 0, // 8: peerdb_flow.FlowConnectionConfigs.cdc_sync_mode:type_name -> peerdb_flow.QRepSyncMode - 49, // 9: peerdb_flow.LastSyncState.last_synced_at:type_name -> google.protobuf.Timestamp - 6, // 10: peerdb_flow.StartFlowInput.last_sync_state:type_name -> peerdb_flow.LastSyncState - 3, // 11: peerdb_flow.StartFlowInput.flow_connection_configs:type_name -> peerdb_flow.FlowConnectionConfigs - 4, // 12: peerdb_flow.StartFlowInput.sync_flow_options:type_name -> peerdb_flow.SyncFlowOptions - 3, // 13: peerdb_flow.StartNormalizeInput.flow_connection_configs:type_name -> peerdb_flow.FlowConnectionConfigs - 48, // 14: peerdb_flow.GetLastSyncedIDInput.peer_connection_config:type_name -> peerdb_peers.Peer - 48, // 15: peerdb_flow.EnsurePullabilityInput.peer_connection_config:type_name -> peerdb_peers.Peer - 48, // 16: peerdb_flow.EnsurePullabilityBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer - 12, // 17: peerdb_flow.TableIdentifier.postgres_table_identifier:type_name -> peerdb_flow.PostgresTableIdentifier - 13, // 18: peerdb_flow.EnsurePullabilityOutput.table_identifier:type_name -> peerdb_flow.TableIdentifier - 41, // 19: peerdb_flow.EnsurePullabilityBatchOutput.table_identifier_mapping:type_name -> peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry - 48, // 20: peerdb_flow.SetupReplicationInput.peer_connection_config:type_name -> peerdb_peers.Peer - 42, // 21: peerdb_flow.SetupReplicationInput.table_name_mapping:type_name -> peerdb_flow.SetupReplicationInput.TableNameMappingEntry - 48, // 22: peerdb_flow.SetupReplicationInput.destination_peer:type_name -> peerdb_peers.Peer - 48, // 23: peerdb_flow.CreateRawTableInput.peer_connection_config:type_name -> peerdb_peers.Peer - 43, // 24: peerdb_flow.CreateRawTableInput.table_name_mapping:type_name -> peerdb_flow.CreateRawTableInput.TableNameMappingEntry - 0, // 25: peerdb_flow.CreateRawTableInput.cdc_sync_mode:type_name -> peerdb_flow.QRepSyncMode - 44, // 26: peerdb_flow.TableSchema.columns:type_name -> peerdb_flow.TableSchema.ColumnsEntry - 48, // 27: peerdb_flow.GetTableSchemaBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer - 45, // 28: peerdb_flow.GetTableSchemaBatchOutput.table_name_schema_mapping:type_name -> peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry - 48, // 29: peerdb_flow.SetupNormalizedTableInput.peer_connection_config:type_name -> peerdb_peers.Peer - 20, // 30: peerdb_flow.SetupNormalizedTableInput.source_table_schema:type_name -> peerdb_flow.TableSchema - 48, // 31: peerdb_flow.SetupNormalizedTableBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer - 46, // 32: peerdb_flow.SetupNormalizedTableBatchInput.table_name_schema_mapping:type_name -> peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry - 47, // 33: peerdb_flow.SetupNormalizedTableBatchOutput.table_exists_mapping:type_name -> peerdb_flow.SetupNormalizedTableBatchOutput.TableExistsMappingEntry - 49, // 34: peerdb_flow.TimestampPartitionRange.start:type_name -> google.protobuf.Timestamp - 49, // 35: peerdb_flow.TimestampPartitionRange.end:type_name -> google.protobuf.Timestamp - 29, // 36: peerdb_flow.TIDPartitionRange.start:type_name -> peerdb_flow.TID - 29, // 37: peerdb_flow.TIDPartitionRange.end:type_name -> peerdb_flow.TID - 27, // 38: peerdb_flow.PartitionRange.int_range:type_name -> peerdb_flow.IntPartitionRange - 28, // 39: peerdb_flow.PartitionRange.timestamp_range:type_name -> peerdb_flow.TimestampPartitionRange - 30, // 40: peerdb_flow.PartitionRange.tid_range:type_name -> peerdb_flow.TIDPartitionRange - 1, // 41: peerdb_flow.QRepWriteMode.write_type:type_name -> peerdb_flow.QRepWriteType - 48, // 42: peerdb_flow.QRepConfig.source_peer:type_name -> peerdb_peers.Peer - 48, // 43: peerdb_flow.QRepConfig.destination_peer:type_name -> peerdb_peers.Peer - 0, // 44: peerdb_flow.QRepConfig.sync_mode:type_name -> peerdb_flow.QRepSyncMode - 32, // 45: peerdb_flow.QRepConfig.write_mode:type_name -> peerdb_flow.QRepWriteMode - 31, // 46: peerdb_flow.QRepPartition.range:type_name -> peerdb_flow.PartitionRange - 34, // 47: peerdb_flow.QRepPartitionBatch.partitions:type_name -> peerdb_flow.QRepPartition - 34, // 48: peerdb_flow.QRepParitionResult.partitions:type_name -> peerdb_flow.QRepPartition - 20, // 49: peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry.value:type_name -> peerdb_flow.TableSchema - 13, // 50: peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry.value:type_name -> peerdb_flow.TableIdentifier - 20, // 51: peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry.value:type_name -> peerdb_flow.TableSchema - 20, // 52: peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry.value:type_name -> peerdb_flow.TableSchema - 53, // [53:53] is the sub-list for method output_type - 53, // [53:53] is the sub-list for method input_type - 53, // [53:53] is the sub-list for extension type_name - 53, // [53:53] is the sub-list for extension extendee - 0, // [0:53] is the sub-list for field type_name + 3, // 0: peerdb_flow.RelationMessage.columns:type_name -> peerdb_flow.RelationMessageColumn + 55, // 1: peerdb_flow.FlowConnectionConfigs.source:type_name -> peerdb_peers.Peer + 55, // 2: peerdb_flow.FlowConnectionConfigs.destination:type_name -> peerdb_peers.Peer + 22, // 3: peerdb_flow.FlowConnectionConfigs.table_schema:type_name -> peerdb_flow.TableSchema + 43, // 4: peerdb_flow.FlowConnectionConfigs.table_name_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.TableNameMappingEntry + 44, // 5: peerdb_flow.FlowConnectionConfigs.src_table_id_name_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.SrcTableIdNameMappingEntry + 45, // 6: peerdb_flow.FlowConnectionConfigs.table_name_schema_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry + 55, // 7: peerdb_flow.FlowConnectionConfigs.metadata_peer:type_name -> peerdb_peers.Peer + 0, // 8: peerdb_flow.FlowConnectionConfigs.snapshot_sync_mode:type_name -> peerdb_flow.QRepSyncMode + 0, // 9: peerdb_flow.FlowConnectionConfigs.cdc_sync_mode:type_name -> peerdb_flow.QRepSyncMode + 46, // 10: peerdb_flow.SyncFlowOptions.relation_message_mapping:type_name -> peerdb_flow.SyncFlowOptions.RelationMessageMappingEntry + 56, // 11: peerdb_flow.LastSyncState.last_synced_at:type_name -> google.protobuf.Timestamp + 8, // 12: peerdb_flow.StartFlowInput.last_sync_state:type_name -> peerdb_flow.LastSyncState + 5, // 13: peerdb_flow.StartFlowInput.flow_connection_configs:type_name -> peerdb_flow.FlowConnectionConfigs + 6, // 14: peerdb_flow.StartFlowInput.sync_flow_options:type_name -> peerdb_flow.SyncFlowOptions + 47, // 15: peerdb_flow.StartFlowInput.relation_message_mapping:type_name -> peerdb_flow.StartFlowInput.RelationMessageMappingEntry + 5, // 16: peerdb_flow.StartNormalizeInput.flow_connection_configs:type_name -> peerdb_flow.FlowConnectionConfigs + 55, // 17: peerdb_flow.GetLastSyncedIDInput.peer_connection_config:type_name -> peerdb_peers.Peer + 55, // 18: peerdb_flow.EnsurePullabilityInput.peer_connection_config:type_name -> peerdb_peers.Peer + 55, // 19: peerdb_flow.EnsurePullabilityBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer + 14, // 20: peerdb_flow.TableIdentifier.postgres_table_identifier:type_name -> peerdb_flow.PostgresTableIdentifier + 15, // 21: peerdb_flow.EnsurePullabilityOutput.table_identifier:type_name -> peerdb_flow.TableIdentifier + 48, // 22: peerdb_flow.EnsurePullabilityBatchOutput.table_identifier_mapping:type_name -> peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry + 55, // 23: peerdb_flow.SetupReplicationInput.peer_connection_config:type_name -> peerdb_peers.Peer + 49, // 24: peerdb_flow.SetupReplicationInput.table_name_mapping:type_name -> peerdb_flow.SetupReplicationInput.TableNameMappingEntry + 55, // 25: peerdb_flow.SetupReplicationInput.destination_peer:type_name -> peerdb_peers.Peer + 55, // 26: peerdb_flow.CreateRawTableInput.peer_connection_config:type_name -> peerdb_peers.Peer + 50, // 27: peerdb_flow.CreateRawTableInput.table_name_mapping:type_name -> peerdb_flow.CreateRawTableInput.TableNameMappingEntry + 0, // 28: peerdb_flow.CreateRawTableInput.cdc_sync_mode:type_name -> peerdb_flow.QRepSyncMode + 51, // 29: peerdb_flow.TableSchema.columns:type_name -> peerdb_flow.TableSchema.ColumnsEntry + 55, // 30: peerdb_flow.GetTableSchemaBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer + 52, // 31: peerdb_flow.GetTableSchemaBatchOutput.table_name_schema_mapping:type_name -> peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry + 55, // 32: peerdb_flow.SetupNormalizedTableInput.peer_connection_config:type_name -> peerdb_peers.Peer + 22, // 33: peerdb_flow.SetupNormalizedTableInput.source_table_schema:type_name -> peerdb_flow.TableSchema + 55, // 34: peerdb_flow.SetupNormalizedTableBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer + 53, // 35: peerdb_flow.SetupNormalizedTableBatchInput.table_name_schema_mapping:type_name -> peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry + 54, // 36: peerdb_flow.SetupNormalizedTableBatchOutput.table_exists_mapping:type_name -> peerdb_flow.SetupNormalizedTableBatchOutput.TableExistsMappingEntry + 56, // 37: peerdb_flow.TimestampPartitionRange.start:type_name -> google.protobuf.Timestamp + 56, // 38: peerdb_flow.TimestampPartitionRange.end:type_name -> google.protobuf.Timestamp + 31, // 39: peerdb_flow.TIDPartitionRange.start:type_name -> peerdb_flow.TID + 31, // 40: peerdb_flow.TIDPartitionRange.end:type_name -> peerdb_flow.TID + 29, // 41: peerdb_flow.PartitionRange.int_range:type_name -> peerdb_flow.IntPartitionRange + 30, // 42: peerdb_flow.PartitionRange.timestamp_range:type_name -> peerdb_flow.TimestampPartitionRange + 32, // 43: peerdb_flow.PartitionRange.tid_range:type_name -> peerdb_flow.TIDPartitionRange + 1, // 44: peerdb_flow.QRepWriteMode.write_type:type_name -> peerdb_flow.QRepWriteType + 55, // 45: peerdb_flow.QRepConfig.source_peer:type_name -> peerdb_peers.Peer + 55, // 46: peerdb_flow.QRepConfig.destination_peer:type_name -> peerdb_peers.Peer + 0, // 47: peerdb_flow.QRepConfig.sync_mode:type_name -> peerdb_flow.QRepSyncMode + 34, // 48: peerdb_flow.QRepConfig.write_mode:type_name -> peerdb_flow.QRepWriteMode + 33, // 49: peerdb_flow.QRepPartition.range:type_name -> peerdb_flow.PartitionRange + 36, // 50: peerdb_flow.QRepPartitionBatch.partitions:type_name -> peerdb_flow.QRepPartition + 36, // 51: peerdb_flow.QRepParitionResult.partitions:type_name -> peerdb_flow.QRepPartition + 40, // 52: peerdb_flow.TableSchemaDelta.added_columns:type_name -> peerdb_flow.DeltaAddedColumn + 5, // 53: peerdb_flow.ReplayTableSchemaDeltaInput.flow_connection_configs:type_name -> peerdb_flow.FlowConnectionConfigs + 41, // 54: peerdb_flow.ReplayTableSchemaDeltaInput.table_schema_delta:type_name -> peerdb_flow.TableSchemaDelta + 22, // 55: peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry.value:type_name -> peerdb_flow.TableSchema + 4, // 56: peerdb_flow.SyncFlowOptions.RelationMessageMappingEntry.value:type_name -> peerdb_flow.RelationMessage + 4, // 57: peerdb_flow.StartFlowInput.RelationMessageMappingEntry.value:type_name -> peerdb_flow.RelationMessage + 15, // 58: peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry.value:type_name -> peerdb_flow.TableIdentifier + 22, // 59: peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry.value:type_name -> peerdb_flow.TableSchema + 22, // 60: peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry.value:type_name -> peerdb_flow.TableSchema + 61, // [61:61] is the sub-list for method output_type + 61, // [61:61] is the sub-list for method input_type + 61, // [61:61] is the sub-list for extension type_name + 61, // [61:61] is the sub-list for extension extendee + 0, // [0:61] is the sub-list for field type_name } func init() { file_flow_proto_init() } @@ -3123,7 +3535,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*FlowConnectionConfigs); i { + switch v := v.(*RelationMessageColumn); i { case 0: return &v.state case 1: @@ -3135,7 +3547,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SyncFlowOptions); i { + switch v := v.(*RelationMessage); i { case 0: return &v.state case 1: @@ -3147,7 +3559,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*NormalizeFlowOptions); i { + switch v := v.(*FlowConnectionConfigs); i { case 0: return &v.state case 1: @@ -3159,7 +3571,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*LastSyncState); i { + switch v := v.(*SyncFlowOptions); i { case 0: return &v.state case 1: @@ -3171,7 +3583,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StartFlowInput); i { + switch v := v.(*NormalizeFlowOptions); i { case 0: return &v.state case 1: @@ -3183,7 +3595,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StartNormalizeInput); i { + switch v := v.(*LastSyncState); i { case 0: return &v.state case 1: @@ -3195,7 +3607,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetLastSyncedIDInput); i { + switch v := v.(*StartFlowInput); i { case 0: return &v.state case 1: @@ -3207,7 +3619,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*EnsurePullabilityInput); i { + switch v := v.(*StartNormalizeInput); i { case 0: return &v.state case 1: @@ -3219,7 +3631,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*EnsurePullabilityBatchInput); i { + switch v := v.(*GetLastSyncedIDInput); i { case 0: return &v.state case 1: @@ -3231,7 +3643,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PostgresTableIdentifier); i { + switch v := v.(*EnsurePullabilityInput); i { case 0: return &v.state case 1: @@ -3243,7 +3655,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TableIdentifier); i { + switch v := v.(*EnsurePullabilityBatchInput); i { case 0: return &v.state case 1: @@ -3255,7 +3667,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*EnsurePullabilityOutput); i { + switch v := v.(*PostgresTableIdentifier); i { case 0: return &v.state case 1: @@ -3267,7 +3679,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*EnsurePullabilityBatchOutput); i { + switch v := v.(*TableIdentifier); i { case 0: return &v.state case 1: @@ -3279,7 +3691,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SetupReplicationInput); i { + switch v := v.(*EnsurePullabilityOutput); i { case 0: return &v.state case 1: @@ -3291,7 +3703,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SetupReplicationOutput); i { + switch v := v.(*EnsurePullabilityBatchOutput); i { case 0: return &v.state case 1: @@ -3303,7 +3715,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CreateRawTableInput); i { + switch v := v.(*SetupReplicationInput); i { case 0: return &v.state case 1: @@ -3315,7 +3727,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CreateRawTableOutput); i { + switch v := v.(*SetupReplicationOutput); i { case 0: return &v.state case 1: @@ -3327,7 +3739,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TableSchema); i { + switch v := v.(*CreateRawTableInput); i { case 0: return &v.state case 1: @@ -3339,7 +3751,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetTableSchemaBatchInput); i { + switch v := v.(*CreateRawTableOutput); i { case 0: return &v.state case 1: @@ -3351,7 +3763,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetTableSchemaBatchOutput); i { + switch v := v.(*TableSchema); i { case 0: return &v.state case 1: @@ -3363,7 +3775,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SetupNormalizedTableInput); i { + switch v := v.(*GetTableSchemaBatchInput); i { case 0: return &v.state case 1: @@ -3375,7 +3787,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SetupNormalizedTableBatchInput); i { + switch v := v.(*GetTableSchemaBatchOutput); i { case 0: return &v.state case 1: @@ -3387,7 +3799,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SetupNormalizedTableOutput); i { + switch v := v.(*SetupNormalizedTableInput); i { case 0: return &v.state case 1: @@ -3399,7 +3811,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SetupNormalizedTableBatchOutput); i { + switch v := v.(*SetupNormalizedTableBatchInput); i { case 0: return &v.state case 1: @@ -3411,7 +3823,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*IntPartitionRange); i { + switch v := v.(*SetupNormalizedTableOutput); i { case 0: return &v.state case 1: @@ -3423,7 +3835,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TimestampPartitionRange); i { + switch v := v.(*SetupNormalizedTableBatchOutput); i { case 0: return &v.state case 1: @@ -3435,7 +3847,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TID); i { + switch v := v.(*IntPartitionRange); i { case 0: return &v.state case 1: @@ -3447,7 +3859,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TIDPartitionRange); i { + switch v := v.(*TimestampPartitionRange); i { case 0: return &v.state case 1: @@ -3459,7 +3871,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PartitionRange); i { + switch v := v.(*TID); i { case 0: return &v.state case 1: @@ -3471,7 +3883,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*QRepWriteMode); i { + switch v := v.(*TIDPartitionRange); i { case 0: return &v.state case 1: @@ -3483,7 +3895,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*QRepConfig); i { + switch v := v.(*PartitionRange); i { case 0: return &v.state case 1: @@ -3495,7 +3907,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*QRepPartition); i { + switch v := v.(*QRepWriteMode); i { case 0: return &v.state case 1: @@ -3507,7 +3919,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*QRepPartitionBatch); i { + switch v := v.(*QRepConfig); i { case 0: return &v.state case 1: @@ -3519,7 +3931,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*QRepParitionResult); i { + switch v := v.(*QRepPartition); i { case 0: return &v.state case 1: @@ -3531,6 +3943,30 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*QRepPartitionBatch); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_flow_proto_msgTypes[36].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*QRepParitionResult); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_flow_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*DropFlowInput); i { case 0: return &v.state @@ -3542,11 +3978,47 @@ func file_flow_proto_init() { return nil } } + file_flow_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DeltaAddedColumn); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_flow_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*TableSchemaDelta); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_flow_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ReplayTableSchemaDeltaInput); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } } - file_flow_proto_msgTypes[11].OneofWrappers = []interface{}{ + file_flow_proto_msgTypes[13].OneofWrappers = []interface{}{ (*TableIdentifier_PostgresTableIdentifier)(nil), } - file_flow_proto_msgTypes[29].OneofWrappers = []interface{}{ + file_flow_proto_msgTypes[31].OneofWrappers = []interface{}{ (*PartitionRange_IntRange)(nil), (*PartitionRange_TimestampRange)(nil), (*PartitionRange_TidRange)(nil), @@ -3557,7 +4029,7 @@ func file_flow_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_flow_proto_rawDesc, NumEnums: 2, - NumMessages: 46, + NumMessages: 53, NumExtensions: 0, NumServices: 0, }, diff --git a/flow/model/model.go b/flow/model/model.go index ad2f22ef7b..9bc5877cba 100644 --- a/flow/model/model.go +++ b/flow/model/model.go @@ -28,6 +28,8 @@ type PullRecordsRequest struct { OverridePublicationName string // override replication slot name OverrideReplicationSlotName string + // for supporting schema changes + RelationMessageMapping RelationMessageMapping } type Record interface { @@ -201,6 +203,10 @@ type SyncResponse struct { CurrentSyncBatchID int64 // TableNameRowsMapping tells how many records need to be synced to each destination table. TableNameRowsMapping map[string]uint32 + // to be carried to NormalizeFlow + TableSchemaDelta *protos.TableSchemaDelta + // to be stored in state for future PullFlows + RelationMessageMapping RelationMessageMapping } type NormalizeResponse struct { @@ -209,3 +215,31 @@ type NormalizeResponse struct { StartBatchID int64 EndBatchID int64 } + +// sync all the records normally, then apply the schema delta after NormalizeFlow. +type RecordsWithTableSchemaDelta struct { + RecordBatch *RecordBatch + TableSchemaDelta *protos.TableSchemaDelta + RelationMessageMapping RelationMessageMapping +} + +// being clever and passing the delta back as a regular record instead of heavy CDC refactoring. +type RelationRecord struct { + CheckPointID int64 + TableSchemaDelta *protos.TableSchemaDelta +} + +// Implement Record interface for RelationRecord. +func (r *RelationRecord) GetCheckPointID() int64 { + return r.CheckPointID +} + +func (r *RelationRecord) GetTableName() string { + return r.TableSchemaDelta.SrcTableName +} + +func (r *RelationRecord) GetItems() RecordItems { + return nil +} + +type RelationMessageMapping map[uint32]*protos.RelationMessage diff --git a/flow/workflows/peer_flow.go b/flow/workflows/peer_flow.go index e6cde4378c..e509e59e52 100644 --- a/flow/workflows/peer_flow.go +++ b/flow/workflows/peer_flow.go @@ -67,6 +67,9 @@ type PeerFlowState struct { SyncFlowErrors error // Errors encountered during child sync flow executions. NormalizeFlowErrors error + // Global mapping of relation IDs to RelationMessages sent as a part of logical replication. + // Needed to support schema changes. + RelationMessageMapping model.RelationMessageMapping } // returns a new empty PeerFlowState @@ -79,6 +82,13 @@ func NewStartedPeerFlowState() *PeerFlowState { SetupComplete: false, SyncFlowErrors: nil, NormalizeFlowErrors: nil, + // WORKAROUND: empty maps are protobufed into nil maps for reasons beyond me + RelationMessageMapping: model.RelationMessageMapping{ + 0: &protos.RelationMessage{ + RelationId: 0, + RelationName: "protobuf_workaround", + }, + }, } } @@ -325,6 +335,7 @@ func PeerFlowWorkflowWithConfig( }, } ctx = workflow.WithChildOptions(ctx, childSyncFlowOpts) + syncFlowOptions.RelationMessageMapping = state.RelationMessageMapping childSyncFlowFuture := workflow.ExecuteChildWorkflow( ctx, SyncFlowWorkflow, @@ -332,16 +343,16 @@ func PeerFlowWorkflowWithConfig( syncFlowOptions, ) - selector.AddFuture(childSyncFlowFuture, func(f workflow.Future) { - var childSyncFlowRes *model.SyncResponse - if err := f.Get(ctx, &childSyncFlowRes); err != nil { - w.logger.Error("failed to execute sync flow: ", err) - state.SyncFlowErrors = multierror.Append(state.SyncFlowErrors, err) - } else { - state.SyncFlowStatuses = append(state.SyncFlowStatuses, childSyncFlowRes) + var childSyncFlowRes *model.SyncResponse + if err := childSyncFlowFuture.Get(ctx, &childSyncFlowRes); err != nil { + w.logger.Error("failed to execute sync flow: ", err) + state.SyncFlowErrors = multierror.Append(state.SyncFlowErrors, err) + } else { + state.SyncFlowStatuses = append(state.SyncFlowStatuses, childSyncFlowRes) + if childSyncFlowRes != nil { + state.RelationMessageMapping = childSyncFlowRes.RelationMessageMapping } - }) - selector.Select(ctx) + } normalizeFlowID, err := GetChildWorkflowID(ctx, "normalize-flow", cfg.FlowJobName) if err != nil { @@ -356,10 +367,17 @@ func PeerFlowWorkflowWithConfig( }, } ctx = workflow.WithChildOptions(ctx, childNormalizeFlowOpts) + + var tableSchemaDelta *protos.TableSchemaDelta = nil + if childSyncFlowRes != nil { + tableSchemaDelta = childSyncFlowRes.TableSchemaDelta + } + childNormalizeFlowFuture := workflow.ExecuteChildWorkflow( ctx, NormalizeFlowWorkflow, cfg, + tableSchemaDelta, ) selector.AddFuture(childNormalizeFlowFuture, func(f workflow.Future) { @@ -372,6 +390,27 @@ func PeerFlowWorkflowWithConfig( } }) selector.Select(ctx) + + // slightly hacky: table schema mapping is cached, so we need to manually update it if schema changes. + if tableSchemaDelta != nil { + getModifiedSchemaCtx := workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ + StartToCloseTimeout: 5 * time.Minute, + }) + getModifiedSchemaFuture := workflow.ExecuteActivity(getModifiedSchemaCtx, flowable.GetTableSchema, + &protos.GetTableSchemaBatchInput{ + PeerConnectionConfig: cfg.Source, + TableIdentifiers: []string{tableSchemaDelta.SrcTableName}, + }) + + var getModifiedSchemaRes *protos.GetTableSchemaBatchOutput + if err := getModifiedSchemaFuture.Get(ctx, &getModifiedSchemaRes); err != nil { + w.logger.Error("failed to execute schema update at source: ", err) + state.SyncFlowErrors = multierror.Append(state.SyncFlowErrors, err) + } else { + cfg.TableNameSchemaMapping[tableSchemaDelta.DstTableName] = + getModifiedSchemaRes.TableNameSchemaMapping[tableSchemaDelta.SrcTableName] + } + } } state.TruncateProgress() diff --git a/flow/workflows/sync_flow.go b/flow/workflows/sync_flow.go index 284bbf4f36..a2be62af9e 100644 --- a/flow/workflows/sync_flow.go +++ b/flow/workflows/sync_flow.go @@ -54,6 +54,7 @@ func (s *SyncFlowExecution) executeSyncFlow( ctx workflow.Context, config *protos.FlowConnectionConfigs, opts *protos.SyncFlowOptions, + relationMessageMapping model.RelationMessageMapping, ) (*model.SyncResponse, error) { s.logger.Info("executing sync flow - ", s.PeerFlowName) @@ -89,9 +90,10 @@ func (s *SyncFlowExecution) executeSyncFlow( // execute StartFlow on the peers to start the flow startFlowInput := &protos.StartFlowInput{ - FlowConnectionConfigs: config, - LastSyncState: dstSyncState, - SyncFlowOptions: opts, + FlowConnectionConfigs: config, + LastSyncState: dstSyncState, + SyncFlowOptions: opts, + RelationMessageMapping: relationMessageMapping, } fStartFlow := workflow.ExecuteActivity(startFlowCtx, flowable.StartFlow, startFlowInput) @@ -115,23 +117,25 @@ func SyncFlowWorkflow(ctx workflow.Context, Progress: []string{}, }) - return s.executeSyncFlow(ctx, config, options) + return s.executeSyncFlow(ctx, config, options, options.RelationMessageMapping) } func NormalizeFlowWorkflow(ctx workflow.Context, config *protos.FlowConnectionConfigs, + tableSchemaDelta *protos.TableSchemaDelta, ) (*model.NormalizeResponse, error) { s := NewNormalizeFlowExecution(ctx, &NormalizeFlowState{ PeerFlowName: config.FlowJobName, Progress: []string{}, }) - return s.executeNormalizeFlow(ctx, config) + return s.executeNormalizeFlow(ctx, config, tableSchemaDelta) } func (s *NormalizeFlowExecution) executeNormalizeFlow( ctx workflow.Context, config *protos.FlowConnectionConfigs, + tableSchemaDelta *protos.TableSchemaDelta, ) (*model.NormalizeResponse, error) { s.logger.Info("executing normalize flow - ", s.PeerFlowName) @@ -151,5 +155,16 @@ func (s *NormalizeFlowExecution) executeNormalizeFlow( return nil, fmt.Errorf("failed to flow: %w", err) } + replayTableSchemaInput := &protos.ReplayTableSchemaDeltaInput{ + FlowConnectionConfigs: config, + TableSchemaDelta: tableSchemaDelta, + } + + fReplayTableSchemaDelta := workflow.ExecuteActivity(normalizeFlowCtx, flowable.ReplayTableSchemaDelta, + replayTableSchemaInput) + if err := fReplayTableSchemaDelta.Get(normalizeFlowCtx, nil); err != nil { + return nil, fmt.Errorf("failed to replay schema delta: %w", err) + } + return normalizeResponse, nil } diff --git a/nexus/pt/src/peerdb_flow.rs b/nexus/pt/src/peerdb_flow.rs index 8612d7632c..5a55396853 100644 --- a/nexus/pt/src/peerdb_flow.rs +++ b/nexus/pt/src/peerdb_flow.rs @@ -9,6 +9,26 @@ pub struct TableNameMapping { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct RelationMessageColumn { + #[prost(uint32, tag="1")] + pub flags: u32, + #[prost(string, tag="2")] + pub name: ::prost::alloc::string::String, + #[prost(uint32, tag="3")] + pub data_type: u32, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct RelationMessage { + #[prost(uint32, tag="1")] + pub relation_id: u32, + #[prost(string, tag="2")] + pub relation_name: ::prost::alloc::string::String, + #[prost(message, repeated, tag="3")] + pub columns: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct FlowConnectionConfigs { #[prost(message, optional, tag="1")] pub source: ::core::option::Option, @@ -65,6 +85,8 @@ pub struct FlowConnectionConfigs { pub struct SyncFlowOptions { #[prost(int32, tag="1")] pub batch_size: i32, + #[prost(map="uint32, message", tag="2")] + pub relation_message_mapping: ::std::collections::HashMap, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -89,6 +111,8 @@ pub struct StartFlowInput { pub flow_connection_configs: ::core::option::Option, #[prost(message, optional, tag="3")] pub sync_flow_options: ::core::option::Option, + #[prost(map="uint32, message", tag="4")] + pub relation_message_mapping: ::std::collections::HashMap, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -398,6 +422,34 @@ pub struct DropFlowInput { #[prost(string, tag="1")] pub flow_name: ::prost::alloc::string::String, } +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct DeltaAddedColumn { + #[prost(string, tag="1")] + pub column_name: ::prost::alloc::string::String, + #[prost(string, tag="2")] + pub column_type: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct TableSchemaDelta { + #[prost(string, tag="1")] + pub src_table_name: ::prost::alloc::string::String, + #[prost(string, tag="2")] + pub dst_table_name: ::prost::alloc::string::String, + #[prost(message, repeated, tag="3")] + pub added_columns: ::prost::alloc::vec::Vec, + #[prost(string, repeated, tag="4")] + pub dropped_columns: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ReplayTableSchemaDeltaInput { + #[prost(message, optional, tag="1")] + pub flow_connection_configs: ::core::option::Option, + #[prost(message, optional, tag="2")] + pub table_schema_delta: ::core::option::Option, +} /// protos for qrep #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] diff --git a/nexus/pt/src/peerdb_flow.serde.rs b/nexus/pt/src/peerdb_flow.serde.rs index 3470c31dd9..8d6aabe5b0 100644 --- a/nexus/pt/src/peerdb_flow.serde.rs +++ b/nexus/pt/src/peerdb_flow.serde.rs @@ -249,6 +249,120 @@ impl<'de> serde::Deserialize<'de> for CreateRawTableOutput { deserializer.deserialize_struct("peerdb_flow.CreateRawTableOutput", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for DeltaAddedColumn { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.column_name.is_empty() { + len += 1; + } + if !self.column_type.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_flow.DeltaAddedColumn", len)?; + if !self.column_name.is_empty() { + struct_ser.serialize_field("columnName", &self.column_name)?; + } + if !self.column_type.is_empty() { + struct_ser.serialize_field("columnType", &self.column_type)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for DeltaAddedColumn { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "column_name", + "columnName", + "column_type", + "columnType", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + ColumnName, + ColumnType, + __SkipField__, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "columnName" | "column_name" => Ok(GeneratedField::ColumnName), + "columnType" | "column_type" => Ok(GeneratedField::ColumnType), + _ => Ok(GeneratedField::__SkipField__), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = DeltaAddedColumn; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct peerdb_flow.DeltaAddedColumn") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut column_name__ = None; + let mut column_type__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::ColumnName => { + if column_name__.is_some() { + return Err(serde::de::Error::duplicate_field("columnName")); + } + column_name__ = Some(map.next_value()?); + } + GeneratedField::ColumnType => { + if column_type__.is_some() { + return Err(serde::de::Error::duplicate_field("columnType")); + } + column_type__ = Some(map.next_value()?); + } + GeneratedField::__SkipField__ => { + let _ = map.next_value::()?; + } + } + } + Ok(DeltaAddedColumn { + column_name: column_name__.unwrap_or_default(), + column_type: column_type__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("peerdb_flow.DeltaAddedColumn", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for DropFlowInput { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result @@ -3160,6 +3274,387 @@ impl<'de> serde::Deserialize<'de> for QRepWriteType { deserializer.deserialize_any(GeneratedVisitor) } } +impl serde::Serialize for RelationMessage { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.relation_id != 0 { + len += 1; + } + if !self.relation_name.is_empty() { + len += 1; + } + if !self.columns.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_flow.RelationMessage", len)?; + if self.relation_id != 0 { + struct_ser.serialize_field("relationId", &self.relation_id)?; + } + if !self.relation_name.is_empty() { + struct_ser.serialize_field("relationName", &self.relation_name)?; + } + if !self.columns.is_empty() { + struct_ser.serialize_field("columns", &self.columns)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for RelationMessage { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "relation_id", + "relationId", + "relation_name", + "relationName", + "columns", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + RelationId, + RelationName, + Columns, + __SkipField__, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "relationId" | "relation_id" => Ok(GeneratedField::RelationId), + "relationName" | "relation_name" => Ok(GeneratedField::RelationName), + "columns" => Ok(GeneratedField::Columns), + _ => Ok(GeneratedField::__SkipField__), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = RelationMessage; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct peerdb_flow.RelationMessage") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut relation_id__ = None; + let mut relation_name__ = None; + let mut columns__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::RelationId => { + if relation_id__.is_some() { + return Err(serde::de::Error::duplicate_field("relationId")); + } + relation_id__ = + Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::RelationName => { + if relation_name__.is_some() { + return Err(serde::de::Error::duplicate_field("relationName")); + } + relation_name__ = Some(map.next_value()?); + } + GeneratedField::Columns => { + if columns__.is_some() { + return Err(serde::de::Error::duplicate_field("columns")); + } + columns__ = Some(map.next_value()?); + } + GeneratedField::__SkipField__ => { + let _ = map.next_value::()?; + } + } + } + Ok(RelationMessage { + relation_id: relation_id__.unwrap_or_default(), + relation_name: relation_name__.unwrap_or_default(), + columns: columns__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("peerdb_flow.RelationMessage", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for RelationMessageColumn { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.flags != 0 { + len += 1; + } + if !self.name.is_empty() { + len += 1; + } + if self.data_type != 0 { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_flow.RelationMessageColumn", len)?; + if self.flags != 0 { + struct_ser.serialize_field("flags", &self.flags)?; + } + if !self.name.is_empty() { + struct_ser.serialize_field("name", &self.name)?; + } + if self.data_type != 0 { + struct_ser.serialize_field("dataType", &self.data_type)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for RelationMessageColumn { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "flags", + "name", + "data_type", + "dataType", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Flags, + Name, + DataType, + __SkipField__, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "flags" => Ok(GeneratedField::Flags), + "name" => Ok(GeneratedField::Name), + "dataType" | "data_type" => Ok(GeneratedField::DataType), + _ => Ok(GeneratedField::__SkipField__), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = RelationMessageColumn; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct peerdb_flow.RelationMessageColumn") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut flags__ = None; + let mut name__ = None; + let mut data_type__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::Flags => { + if flags__.is_some() { + return Err(serde::de::Error::duplicate_field("flags")); + } + flags__ = + Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::Name => { + if name__.is_some() { + return Err(serde::de::Error::duplicate_field("name")); + } + name__ = Some(map.next_value()?); + } + GeneratedField::DataType => { + if data_type__.is_some() { + return Err(serde::de::Error::duplicate_field("dataType")); + } + data_type__ = + Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::__SkipField__ => { + let _ = map.next_value::()?; + } + } + } + Ok(RelationMessageColumn { + flags: flags__.unwrap_or_default(), + name: name__.unwrap_or_default(), + data_type: data_type__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("peerdb_flow.RelationMessageColumn", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for ReplayTableSchemaDeltaInput { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.flow_connection_configs.is_some() { + len += 1; + } + if self.table_schema_delta.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_flow.ReplayTableSchemaDeltaInput", len)?; + if let Some(v) = self.flow_connection_configs.as_ref() { + struct_ser.serialize_field("flowConnectionConfigs", v)?; + } + if let Some(v) = self.table_schema_delta.as_ref() { + struct_ser.serialize_field("tableSchemaDelta", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for ReplayTableSchemaDeltaInput { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "flow_connection_configs", + "flowConnectionConfigs", + "table_schema_delta", + "tableSchemaDelta", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + FlowConnectionConfigs, + TableSchemaDelta, + __SkipField__, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "flowConnectionConfigs" | "flow_connection_configs" => Ok(GeneratedField::FlowConnectionConfigs), + "tableSchemaDelta" | "table_schema_delta" => Ok(GeneratedField::TableSchemaDelta), + _ => Ok(GeneratedField::__SkipField__), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = ReplayTableSchemaDeltaInput; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct peerdb_flow.ReplayTableSchemaDeltaInput") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut flow_connection_configs__ = None; + let mut table_schema_delta__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::FlowConnectionConfigs => { + if flow_connection_configs__.is_some() { + return Err(serde::de::Error::duplicate_field("flowConnectionConfigs")); + } + flow_connection_configs__ = map.next_value()?; + } + GeneratedField::TableSchemaDelta => { + if table_schema_delta__.is_some() { + return Err(serde::de::Error::duplicate_field("tableSchemaDelta")); + } + table_schema_delta__ = map.next_value()?; + } + GeneratedField::__SkipField__ => { + let _ = map.next_value::()?; + } + } + } + Ok(ReplayTableSchemaDeltaInput { + flow_connection_configs: flow_connection_configs__, + table_schema_delta: table_schema_delta__, + }) + } + } + deserializer.deserialize_struct("peerdb_flow.ReplayTableSchemaDeltaInput", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for SetupNormalizedTableBatchInput { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result @@ -3957,6 +4452,9 @@ impl serde::Serialize for StartFlowInput { if self.sync_flow_options.is_some() { len += 1; } + if !self.relation_message_mapping.is_empty() { + len += 1; + } let mut struct_ser = serializer.serialize_struct("peerdb_flow.StartFlowInput", len)?; if let Some(v) = self.last_sync_state.as_ref() { struct_ser.serialize_field("lastSyncState", v)?; @@ -3967,6 +4465,9 @@ impl serde::Serialize for StartFlowInput { if let Some(v) = self.sync_flow_options.as_ref() { struct_ser.serialize_field("syncFlowOptions", v)?; } + if !self.relation_message_mapping.is_empty() { + struct_ser.serialize_field("relationMessageMapping", &self.relation_message_mapping)?; + } struct_ser.end() } } @@ -3983,6 +4484,8 @@ impl<'de> serde::Deserialize<'de> for StartFlowInput { "flowConnectionConfigs", "sync_flow_options", "syncFlowOptions", + "relation_message_mapping", + "relationMessageMapping", ]; #[allow(clippy::enum_variant_names)] @@ -3990,6 +4493,7 @@ impl<'de> serde::Deserialize<'de> for StartFlowInput { LastSyncState, FlowConnectionConfigs, SyncFlowOptions, + RelationMessageMapping, __SkipField__, } impl<'de> serde::Deserialize<'de> for GeneratedField { @@ -4015,6 +4519,7 @@ impl<'de> serde::Deserialize<'de> for StartFlowInput { "lastSyncState" | "last_sync_state" => Ok(GeneratedField::LastSyncState), "flowConnectionConfigs" | "flow_connection_configs" => Ok(GeneratedField::FlowConnectionConfigs), "syncFlowOptions" | "sync_flow_options" => Ok(GeneratedField::SyncFlowOptions), + "relationMessageMapping" | "relation_message_mapping" => Ok(GeneratedField::RelationMessageMapping), _ => Ok(GeneratedField::__SkipField__), } } @@ -4037,6 +4542,7 @@ impl<'de> serde::Deserialize<'de> for StartFlowInput { let mut last_sync_state__ = None; let mut flow_connection_configs__ = None; let mut sync_flow_options__ = None; + let mut relation_message_mapping__ = None; while let Some(k) = map.next_key()? { match k { GeneratedField::LastSyncState => { @@ -4057,6 +4563,15 @@ impl<'de> serde::Deserialize<'de> for StartFlowInput { } sync_flow_options__ = map.next_value()?; } + GeneratedField::RelationMessageMapping => { + if relation_message_mapping__.is_some() { + return Err(serde::de::Error::duplicate_field("relationMessageMapping")); + } + relation_message_mapping__ = Some( + map.next_value::, _>>()? + .into_iter().map(|(k,v)| (k.0, v)).collect() + ); + } GeneratedField::__SkipField__ => { let _ = map.next_value::()?; } @@ -4066,6 +4581,7 @@ impl<'de> serde::Deserialize<'de> for StartFlowInput { last_sync_state: last_sync_state__, flow_connection_configs: flow_connection_configs__, sync_flow_options: sync_flow_options__, + relation_message_mapping: relation_message_mapping__.unwrap_or_default(), }) } } @@ -4179,10 +4695,16 @@ impl serde::Serialize for SyncFlowOptions { if self.batch_size != 0 { len += 1; } + if !self.relation_message_mapping.is_empty() { + len += 1; + } let mut struct_ser = serializer.serialize_struct("peerdb_flow.SyncFlowOptions", len)?; if self.batch_size != 0 { struct_ser.serialize_field("batchSize", &self.batch_size)?; } + if !self.relation_message_mapping.is_empty() { + struct_ser.serialize_field("relationMessageMapping", &self.relation_message_mapping)?; + } struct_ser.end() } } @@ -4195,11 +4717,14 @@ impl<'de> serde::Deserialize<'de> for SyncFlowOptions { const FIELDS: &[&str] = &[ "batch_size", "batchSize", + "relation_message_mapping", + "relationMessageMapping", ]; #[allow(clippy::enum_variant_names)] enum GeneratedField { BatchSize, + RelationMessageMapping, __SkipField__, } impl<'de> serde::Deserialize<'de> for GeneratedField { @@ -4223,6 +4748,7 @@ impl<'de> serde::Deserialize<'de> for SyncFlowOptions { { match value { "batchSize" | "batch_size" => Ok(GeneratedField::BatchSize), + "relationMessageMapping" | "relation_message_mapping" => Ok(GeneratedField::RelationMessageMapping), _ => Ok(GeneratedField::__SkipField__), } } @@ -4243,6 +4769,7 @@ impl<'de> serde::Deserialize<'de> for SyncFlowOptions { V: serde::de::MapAccess<'de>, { let mut batch_size__ = None; + let mut relation_message_mapping__ = None; while let Some(k) = map.next_key()? { match k { GeneratedField::BatchSize => { @@ -4253,6 +4780,15 @@ impl<'de> serde::Deserialize<'de> for SyncFlowOptions { Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } + GeneratedField::RelationMessageMapping => { + if relation_message_mapping__.is_some() { + return Err(serde::de::Error::duplicate_field("relationMessageMapping")); + } + relation_message_mapping__ = Some( + map.next_value::, _>>()? + .into_iter().map(|(k,v)| (k.0, v)).collect() + ); + } GeneratedField::__SkipField__ => { let _ = map.next_value::()?; } @@ -4260,6 +4796,7 @@ impl<'de> serde::Deserialize<'de> for SyncFlowOptions { } Ok(SyncFlowOptions { batch_size: batch_size__.unwrap_or_default(), + relation_message_mapping: relation_message_mapping__.unwrap_or_default(), }) } } @@ -4844,6 +5381,156 @@ impl<'de> serde::Deserialize<'de> for TableSchema { deserializer.deserialize_struct("peerdb_flow.TableSchema", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for TableSchemaDelta { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.src_table_name.is_empty() { + len += 1; + } + if !self.dst_table_name.is_empty() { + len += 1; + } + if !self.added_columns.is_empty() { + len += 1; + } + if !self.dropped_columns.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_flow.TableSchemaDelta", len)?; + if !self.src_table_name.is_empty() { + struct_ser.serialize_field("srcTableName", &self.src_table_name)?; + } + if !self.dst_table_name.is_empty() { + struct_ser.serialize_field("dstTableName", &self.dst_table_name)?; + } + if !self.added_columns.is_empty() { + struct_ser.serialize_field("addedColumns", &self.added_columns)?; + } + if !self.dropped_columns.is_empty() { + struct_ser.serialize_field("droppedColumns", &self.dropped_columns)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for TableSchemaDelta { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "src_table_name", + "srcTableName", + "dst_table_name", + "dstTableName", + "added_columns", + "addedColumns", + "dropped_columns", + "droppedColumns", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + SrcTableName, + DstTableName, + AddedColumns, + DroppedColumns, + __SkipField__, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "srcTableName" | "src_table_name" => Ok(GeneratedField::SrcTableName), + "dstTableName" | "dst_table_name" => Ok(GeneratedField::DstTableName), + "addedColumns" | "added_columns" => Ok(GeneratedField::AddedColumns), + "droppedColumns" | "dropped_columns" => Ok(GeneratedField::DroppedColumns), + _ => Ok(GeneratedField::__SkipField__), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = TableSchemaDelta; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct peerdb_flow.TableSchemaDelta") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut src_table_name__ = None; + let mut dst_table_name__ = None; + let mut added_columns__ = None; + let mut dropped_columns__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::SrcTableName => { + if src_table_name__.is_some() { + return Err(serde::de::Error::duplicate_field("srcTableName")); + } + src_table_name__ = Some(map.next_value()?); + } + GeneratedField::DstTableName => { + if dst_table_name__.is_some() { + return Err(serde::de::Error::duplicate_field("dstTableName")); + } + dst_table_name__ = Some(map.next_value()?); + } + GeneratedField::AddedColumns => { + if added_columns__.is_some() { + return Err(serde::de::Error::duplicate_field("addedColumns")); + } + added_columns__ = Some(map.next_value()?); + } + GeneratedField::DroppedColumns => { + if dropped_columns__.is_some() { + return Err(serde::de::Error::duplicate_field("droppedColumns")); + } + dropped_columns__ = Some(map.next_value()?); + } + GeneratedField::__SkipField__ => { + let _ = map.next_value::()?; + } + } + } + Ok(TableSchemaDelta { + src_table_name: src_table_name__.unwrap_or_default(), + dst_table_name: dst_table_name__.unwrap_or_default(), + added_columns: added_columns__.unwrap_or_default(), + dropped_columns: dropped_columns__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("peerdb_flow.TableSchemaDelta", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for TimestampPartitionRange { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result diff --git a/protos/flow.proto b/protos/flow.proto index 7ddd873975..07d5af491a 100644 --- a/protos/flow.proto +++ b/protos/flow.proto @@ -10,6 +10,18 @@ message TableNameMapping { string destination_table_name = 2; } +message RelationMessageColumn { + uint32 flags = 1; + string name = 2; + uint32 data_type = 3; +} + +message RelationMessage { + uint32 relation_id = 1; + string relation_name = 2; + repeated RelationMessageColumn columns = 3; +} + message FlowConnectionConfigs { peerdb_peers.Peer source = 1; peerdb_peers.Peer destination = 2; @@ -46,7 +58,10 @@ message FlowConnectionConfigs { int64 push_parallelism = 22; } -message SyncFlowOptions { int32 batch_size = 1; } +message SyncFlowOptions { + int32 batch_size = 1; + map relation_message_mapping = 2; +} message NormalizeFlowOptions { int32 batch_size = 1; @@ -61,6 +76,7 @@ message StartFlowInput { LastSyncState last_sync_state = 1; FlowConnectionConfigs flow_connection_configs = 2; SyncFlowOptions sync_flow_options = 3; + map relation_message_mapping = 4; } message StartNormalizeInput { @@ -268,3 +284,20 @@ message QRepParitionResult { message DropFlowInput { string flow_name = 1; } + +message DeltaAddedColumn { + string column_name = 1; + string column_type = 2; +} + +message TableSchemaDelta { + string src_table_name = 1; + string dst_table_name = 2; + repeated DeltaAddedColumn added_columns = 3; + repeated string dropped_columns = 4; +} + +message ReplayTableSchemaDeltaInput { + FlowConnectionConfigs flow_connection_configs = 1; + TableSchemaDelta table_schema_delta = 2; +} diff --git a/stacks/flow-api.Dockerfile b/stacks/flow-api.Dockerfile index fc95cd4a7d..f84c5da352 100644 --- a/stacks/flow-api.Dockerfile +++ b/stacks/flow-api.Dockerfile @@ -1,7 +1,7 @@ # syntax=docker/dockerfile:1.2 # Start from the latest Golang base image -FROM golang:1.20-alpine AS builder +FROM golang:1.21-alpine AS builder WORKDIR /root/flow # first copy only go.mod and go.sum to cache dependencies diff --git a/stacks/flow-snapshot-worker.Dockerfile b/stacks/flow-snapshot-worker.Dockerfile index a58e800d50..cf3a788818 100644 --- a/stacks/flow-snapshot-worker.Dockerfile +++ b/stacks/flow-snapshot-worker.Dockerfile @@ -1,7 +1,7 @@ # syntax=docker/dockerfile:1.2 # Start from the latest Golang base image -FROM golang:1.20-alpine AS builder +FROM golang:1.21-alpine AS builder WORKDIR /root/ diff --git a/stacks/flow-worker.Dockerfile b/stacks/flow-worker.Dockerfile index b5226ff9ed..cd6725b9fb 100644 --- a/stacks/flow-worker.Dockerfile +++ b/stacks/flow-worker.Dockerfile @@ -1,7 +1,7 @@ # syntax=docker/dockerfile:1.2 # Start from the latest Golang base image -FROM golang:1.20-alpine AS builder +FROM golang:1.21-alpine AS builder WORKDIR /root/ From c1d153a87e6bffb92f272d8f8f1b6718497e2838 Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Thu, 7 Sep 2023 10:43:58 -0400 Subject: [PATCH 33/37] Basic UI for PeerDB (only connectors page is wired) (#377) - This brings the various UI elements to our lib - Also wires up the gRPC from FE to BE. - Connectors page is wired, but lots of other work left. - Serves as a prototype for other pages. --- .dockerignore | 5 + buf.gen.yaml | 5 + docker-compose.yml | 18 +- flow/cmd/api.go | 10 +- flow/cmd/handler.go | 104 +- flow/cmd/worker.go | 51 +- flow/connectors/utils/catalog/env.go | 61 + flow/generated/protos/route.pb.go | 220 +- flow/generated/protos/route_grpc.pb.go | 37 + nexus/pt/src/peerdb_route.rs | 10 + nexus/pt/src/peerdb_route.serde.rs | 167 + nexus/pt/src/peerdb_route.tonic.rs | 78 + protos/route.proto | 8 + stacks/ui-entrypoint.sh | 3 + stacks/ui.Dockerfile | 55 + ui/.eslintignore | 1 + ui/.eslintrc.json | 7 + ui/.gitignore | 38 + ui/.nvmrc | 1 + ui/.prettierignore | 2 + ui/.prettierrc.json | 8 + ui/.storybook/decorators.tsx | 8 + ui/.storybook/main.ts | 57 + ui/.storybook/preview.ts | 17 + ui/.storybook/public/.gitignore | 4 + ui/.yarnrc.yml | 5 + ui/README.md | 50 + ui/app/connectors/create/page.tsx | 41 + ui/app/connectors/edit/[connectorId]/page.tsx | 290 + ui/app/connectors/layout.tsx | 7 + ui/app/connectors/page.tsx | 123 + ui/app/dashboard/layout.tsx | 7 + ui/app/dashboard/page.tsx | 195 + ui/app/favicon.ico | Bin 0 -> 4663 bytes ui/app/layout.tsx | 22 + ui/app/mirrors/create/page.tsx | 119 + ui/app/mirrors/edit/[mirrorId]/page.tsx | 195 + ui/app/mirrors/layout.tsx | 7 + ui/app/mirrors/page.tsx | 150 + ui/app/page.tsx | 13 + ui/components/SidebarComponent.tsx | 48 + ui/grpc_generated/flow.ts | 4744 +++++++ .../google/protobuf/timestamp.ts | 232 + ui/grpc_generated/peers.ts | 1320 ++ ui/grpc_generated/route.ts | 704 + ui/lib/Action/Action.stories.tsx | 18 + ui/lib/Action/Action.styles.ts | 29 + ui/lib/Action/Action.tsx | 26 + ui/lib/Action/index.ts | 1 + ui/lib/AppTheme/AppThemeProvider.tsx | 34 + ui/lib/AppTheme/StyledComponentsRegistry.tsx | 29 + ui/lib/AppTheme/appTheme.ts | 364 + ui/lib/AppTheme/appThemeColors.ts | 132 + ui/lib/AppTheme/appThemeDropShadow.ts | 25 + ui/lib/AppTheme/appThemeFont.ts | 22 + ui/lib/AppTheme/appThemeRadius.ts | 9 + ui/lib/AppTheme/index.ts | 2 + ui/lib/AppTheme/styled.d.ts | 6 + ui/lib/AppTheme/tailwind.css | 3 + ui/lib/Avatar/Avatar.stories.tsx | 43 + ui/lib/Avatar/Avatar.styles.ts | 86 + ui/lib/Avatar/Avatar.tsx | 58 + ui/lib/Avatar/checker.png | Bin 0 -> 284 bytes ui/lib/Avatar/index.ts | 1 + ui/lib/Badge/Badge.stories.tsx | 34 + ui/lib/Badge/Badge.styles.ts | 73 + ui/lib/Badge/Badge.tsx | 16 + ui/lib/Badge/index.ts | 1 + ui/lib/BrandLogo/BrandLogo.tsx | 56 + ui/lib/BrandLogo/index.ts | 1 + ui/lib/Button/Button.stories.tsx | 76 + ui/lib/Button/Button.styles.ts | 130 + ui/lib/Button/Button.tsx | 32 + ui/lib/Button/index.ts | 1 + ui/lib/ButtonGroup/ButtonGroup.stories.tsx | 38 + ui/lib/ButtonGroup/ButtonGroup.styles.ts | 7 + ui/lib/ButtonGroup/ButtonGroup.tsx | 12 + ui/lib/ButtonGroup/index.ts | 1 + ui/lib/Chart/TrackerChart.stories.tsx | 43 + ui/lib/Chart/TrackerChart.styles.ts | 13 + ui/lib/Chart/TrackerChart.tsx | 24 + ui/lib/Chart/index.ts | 1 + ui/lib/Checkbox/Checkbox.stories.tsx | 35 + ui/lib/Checkbox/Checkbox.styles.ts | 51 + ui/lib/Checkbox/Checkbox.tsx | 29 + ui/lib/Checkbox/index.ts | 1 + ui/lib/CodeSnippet/CodeSnippet.stories.tsx | 30 + ui/lib/CodeSnippet/CodeSnippet.styles.ts | 21 + ui/lib/CodeSnippet/CodeSnippet.tsx | 11 + ui/lib/CodeSnippet/index.ts | 1 + ui/lib/Color/Color.stories.tsx | 18 + ui/lib/Color/Color.tsx | 52 + ui/lib/Color/index.ts | 1 + ui/lib/Dialog/Dialog.stories.tsx | 29 + ui/lib/Dialog/Dialog.styles.ts | 3 + ui/lib/Dialog/Dialog.tsx | 35 + ui/lib/Dialog/DialogContent.stories.tsx | 54 + ui/lib/Dialog/DialogContent.styles.ts | 54 + ui/lib/Dialog/DialogContent.tsx | 20 + ui/lib/Dialog/index.ts | 1 + ui/lib/Header/Header.stories.tsx | 65 + ui/lib/Header/Header.styles.tsx | 18 + ui/lib/Header/Header.tsx | 27 + ui/lib/Header/index.ts | 1 + ui/lib/Icon/DeterminateProgressCircle.tsx | 31 + ui/lib/Icon/Icon.stories.tsx | 21 + ui/lib/Icon/Icon.tsx | 61 + ui/lib/Icon/IntermediateProgressCircle.tsx | 95 + ui/lib/Icon/index.ts | 1 + ui/lib/Label/Label.stories.tsx | 66 + ui/lib/Label/Label.styles.ts | 55 + ui/lib/Label/Label.tsx | 42 + ui/lib/Label/index.ts | 1 + ui/lib/Layout/BaseLayout.styles.ts | 51 + ui/lib/Layout/BaseLayout.ts | 9 + ui/lib/Layout/BaseLayoutColumn.tsx | 63 + ui/lib/Layout/BaseLayoutRow.tsx | 72 + ui/lib/Layout/ColumnWithTextField.stories.tsx | 29 + ui/lib/Layout/Layout.stories.tsx | 50 + ui/lib/Layout/Layout.styles.ts | 58 + ui/lib/Layout/Layout.tsx | 135 + ui/lib/Layout/Row.stories.tsx | 51 + ui/lib/Layout/Row.styles.ts | 74 + ui/lib/Layout/Row.tsx | 100 + ui/lib/Layout/RowWIthCheckbox.stories.tsx | 18 + ui/lib/Layout/RowWithButton.stories.tsx | 18 + ui/lib/Layout/RowWithProgressBar.stories.tsx | 31 + ui/lib/Layout/RowWithRadioButton.stories.tsx | 23 + ui/lib/Layout/RowWithSelect.stories.tsx | 32 + ui/lib/Layout/RowWithSwitch.stories.tsx | 18 + ui/lib/Layout/RowWithTextField.stories.tsx | 24 + ui/lib/Layout/RowWithToggleGroup.stories.tsx | 30 + ui/lib/Layout/checker.png | Bin 0 -> 284 bytes ui/lib/Layout/index.ts | 2 + ui/lib/Media/Media.stories.tsx | 85 + ui/lib/Media/Media.styles.ts | 38 + ui/lib/Media/Media.tsx | 16 + ui/lib/Media/checker.png | Bin 0 -> 284 bytes ui/lib/Media/index.ts | 0 ui/lib/Panel/Panel.tsx | 9 + ui/lib/Panel/index.ts | 1 + ui/lib/ProgressBar/ProgressBar.stories.tsx | 14 + ui/lib/ProgressBar/ProgressBar.styles.ts | 31 + ui/lib/ProgressBar/ProgressBar.tsx | 30 + ui/lib/ProgressBar/index.ts | 1 + .../ProgressCircle/ProgressCircle.stories.tsx | 21 + .../ProgressCircle/ProgressCircle.styles.ts | 44 + ui/lib/ProgressCircle/ProgressCircle.tsx | 11 + ui/lib/ProgressCircle/index.ts | 1 + .../RadioButtonGroup.stories.tsx | 21 + .../RadioButtonGroup.styles.ts | 53 + ui/lib/RadioButtonGroup/RadioButtonGroup.tsx | 35 + ui/lib/RadioButtonGroup/index.ts | 1 + ui/lib/SearchField/SearchField.stories.tsx | 20 + ui/lib/SearchField/SearchField.styles.ts | 43 + ui/lib/SearchField/SearchField.tsx | 18 + ui/lib/SearchField/index.ts | 1 + ui/lib/Select/Select.stories.tsx | 36 + ui/lib/Select/Select.styles.ts | 50 + ui/lib/Select/Select.tsx | 34 + ui/lib/Select/SelectItem.stories.tsx | 36 + ui/lib/Select/SelectItem.styles.ts | 32 + ui/lib/Select/SelectItem.tsx | 17 + ui/lib/Select/SelectMenu.stories.tsx | 28 + ui/lib/Select/SelectMenu.styles.ts | 12 + ui/lib/Select/SelectMenu.tsx | 13 + ui/lib/Select/index.ts | 2 + ui/lib/Separator/Separator.stories.tsx | 48 + ui/lib/Separator/Separator.styles.ts | 70 + ui/lib/Separator/Separator.tsx | 22 + ui/lib/Separator/index.ts | 1 + ui/lib/Sidebar/Sidebar.stories.tsx | 41 + ui/lib/Sidebar/Sidebar.styles.ts | 25 + ui/lib/Sidebar/Sidebar.tsx | 66 + ui/lib/Sidebar/SidebarItem.stories.tsx | 35 + ui/lib/Sidebar/SidebarItem.styles.ts | 73 + ui/lib/Sidebar/SidebarItem.tsx | 49 + ui/lib/Sidebar/index.ts | 2 + ui/lib/Slot/Slot.stories.tsx | 15 + ui/lib/Slot/Slot.styles.ts | 7 + ui/lib/Slot/Slot.tsx | 16 + ui/lib/Slot/index.ts | 1 + ui/lib/Switch/Switch.stories.tsx | 26 + ui/lib/Switch/Switch.styles.ts | 56 + ui/lib/Switch/Switch.tsx | 17 + ui/lib/Switch/index.ts | 1 + ui/lib/Table/Table.stories.tsx | 173 + ui/lib/Table/Table.styles.ts | 34 + ui/lib/Table/Table.tsx | 86 + ui/lib/Table/TableCell.stories.tsx | 44 + ui/lib/Table/TableCell.styles.ts | 26 + ui/lib/Table/TableCell.tsx | 13 + ui/lib/Table/TableRow.stories.tsx | 51 + ui/lib/Table/TableRow.tsx | 4 + ui/lib/Table/index.ts | 3 + ui/lib/TextField/TextField.stories.tsx | 26 + ui/lib/TextField/TextField.styles.ts | 48 + ui/lib/TextField/TextField.tsx | 29 + ui/lib/TextField/index.ts | 1 + ui/lib/Thumbnail/Thumbnail.stories.tsx | 37 + ui/lib/Thumbnail/Thumbnail.styles.ts | 40 + ui/lib/Thumbnail/Thumbnail.tsx | 17 + ui/lib/Thumbnail/checker.png | Bin 0 -> 284 bytes ui/lib/Thumbnail/index.ts | 1 + ui/lib/Toast/Toast.stories.tsx | 31 + ui/lib/Toast/Toast.styles.ts | 103 + ui/lib/Toast/Toast.tsx | 54 + ui/lib/Toast/index.ts | 1 + .../ToggleButton/ToggleButton.stories.tsx | 54 + .../ToggleButton/ToggleButton.styles.ts | 7 + ui/lib/Toggle/ToggleButton/ToggleButton.tsx | 18 + ui/lib/Toggle/ToggleButton/index.ts | 1 + .../ToggleGroup/ToggleGroup.stories.tsx | 35 + .../Toggle/ToggleGroup/ToggleGroup.styles.ts | 15 + ui/lib/Toggle/ToggleGroup/ToggleGroup.tsx | 24 + ui/lib/Toggle/ToggleGroup/index.ts | 2 + ui/lib/Toggle/index.ts | 2 + ui/lib/Toggle/styles.ts | 38 + ui/lib/Tooltip/Tooltip.stories.tsx | 18 + ui/lib/Tooltip/Tooltip.styles.ts | 25 + ui/lib/Tooltip/Tooltip.tsx | 37 + ui/lib/Tooltip/index.ts | 1 + ui/lib/types.ts | 18 + ui/lib/utils/isDefined.ts | 3 + ui/lib/utils/renderObjectWith.ts | 33 + ui/next.config.js | 11 + ui/package.json | 70 + ui/postcss.config.js | 6 + ui/public/images/peerdb-combinedMark.svg | 18 + ui/rpc/promisify.ts | 53 + ui/rpc/rpc.ts | 12 + ui/tailwind.config.ts | 105 + ui/tsconfig.json | 28 + ui/yarn.lock | 10768 ++++++++++++++++ 234 files changed, 25961 insertions(+), 100 deletions(-) create mode 100644 flow/connectors/utils/catalog/env.go create mode 100755 stacks/ui-entrypoint.sh create mode 100644 stacks/ui.Dockerfile create mode 100644 ui/.eslintignore create mode 100644 ui/.eslintrc.json create mode 100644 ui/.gitignore create mode 100644 ui/.nvmrc create mode 100644 ui/.prettierignore create mode 100644 ui/.prettierrc.json create mode 100644 ui/.storybook/decorators.tsx create mode 100644 ui/.storybook/main.ts create mode 100644 ui/.storybook/preview.ts create mode 100644 ui/.storybook/public/.gitignore create mode 100644 ui/.yarnrc.yml create mode 100644 ui/README.md create mode 100644 ui/app/connectors/create/page.tsx create mode 100644 ui/app/connectors/edit/[connectorId]/page.tsx create mode 100644 ui/app/connectors/layout.tsx create mode 100644 ui/app/connectors/page.tsx create mode 100644 ui/app/dashboard/layout.tsx create mode 100644 ui/app/dashboard/page.tsx create mode 100644 ui/app/favicon.ico create mode 100644 ui/app/layout.tsx create mode 100644 ui/app/mirrors/create/page.tsx create mode 100644 ui/app/mirrors/edit/[mirrorId]/page.tsx create mode 100644 ui/app/mirrors/layout.tsx create mode 100644 ui/app/mirrors/page.tsx create mode 100644 ui/app/page.tsx create mode 100644 ui/components/SidebarComponent.tsx create mode 100644 ui/grpc_generated/flow.ts create mode 100644 ui/grpc_generated/google/protobuf/timestamp.ts create mode 100644 ui/grpc_generated/peers.ts create mode 100644 ui/grpc_generated/route.ts create mode 100644 ui/lib/Action/Action.stories.tsx create mode 100644 ui/lib/Action/Action.styles.ts create mode 100644 ui/lib/Action/Action.tsx create mode 100644 ui/lib/Action/index.ts create mode 100644 ui/lib/AppTheme/AppThemeProvider.tsx create mode 100644 ui/lib/AppTheme/StyledComponentsRegistry.tsx create mode 100644 ui/lib/AppTheme/appTheme.ts create mode 100644 ui/lib/AppTheme/appThemeColors.ts create mode 100644 ui/lib/AppTheme/appThemeDropShadow.ts create mode 100644 ui/lib/AppTheme/appThemeFont.ts create mode 100644 ui/lib/AppTheme/appThemeRadius.ts create mode 100644 ui/lib/AppTheme/index.ts create mode 100644 ui/lib/AppTheme/styled.d.ts create mode 100644 ui/lib/AppTheme/tailwind.css create mode 100644 ui/lib/Avatar/Avatar.stories.tsx create mode 100644 ui/lib/Avatar/Avatar.styles.ts create mode 100644 ui/lib/Avatar/Avatar.tsx create mode 100644 ui/lib/Avatar/checker.png create mode 100644 ui/lib/Avatar/index.ts create mode 100644 ui/lib/Badge/Badge.stories.tsx create mode 100644 ui/lib/Badge/Badge.styles.ts create mode 100644 ui/lib/Badge/Badge.tsx create mode 100644 ui/lib/Badge/index.ts create mode 100644 ui/lib/BrandLogo/BrandLogo.tsx create mode 100644 ui/lib/BrandLogo/index.ts create mode 100644 ui/lib/Button/Button.stories.tsx create mode 100644 ui/lib/Button/Button.styles.ts create mode 100644 ui/lib/Button/Button.tsx create mode 100644 ui/lib/Button/index.ts create mode 100644 ui/lib/ButtonGroup/ButtonGroup.stories.tsx create mode 100644 ui/lib/ButtonGroup/ButtonGroup.styles.ts create mode 100644 ui/lib/ButtonGroup/ButtonGroup.tsx create mode 100644 ui/lib/ButtonGroup/index.ts create mode 100644 ui/lib/Chart/TrackerChart.stories.tsx create mode 100644 ui/lib/Chart/TrackerChart.styles.ts create mode 100644 ui/lib/Chart/TrackerChart.tsx create mode 100644 ui/lib/Chart/index.ts create mode 100644 ui/lib/Checkbox/Checkbox.stories.tsx create mode 100644 ui/lib/Checkbox/Checkbox.styles.ts create mode 100644 ui/lib/Checkbox/Checkbox.tsx create mode 100644 ui/lib/Checkbox/index.ts create mode 100644 ui/lib/CodeSnippet/CodeSnippet.stories.tsx create mode 100644 ui/lib/CodeSnippet/CodeSnippet.styles.ts create mode 100644 ui/lib/CodeSnippet/CodeSnippet.tsx create mode 100644 ui/lib/CodeSnippet/index.ts create mode 100644 ui/lib/Color/Color.stories.tsx create mode 100644 ui/lib/Color/Color.tsx create mode 100644 ui/lib/Color/index.ts create mode 100644 ui/lib/Dialog/Dialog.stories.tsx create mode 100644 ui/lib/Dialog/Dialog.styles.ts create mode 100644 ui/lib/Dialog/Dialog.tsx create mode 100644 ui/lib/Dialog/DialogContent.stories.tsx create mode 100644 ui/lib/Dialog/DialogContent.styles.ts create mode 100644 ui/lib/Dialog/DialogContent.tsx create mode 100644 ui/lib/Dialog/index.ts create mode 100644 ui/lib/Header/Header.stories.tsx create mode 100644 ui/lib/Header/Header.styles.tsx create mode 100644 ui/lib/Header/Header.tsx create mode 100644 ui/lib/Header/index.ts create mode 100644 ui/lib/Icon/DeterminateProgressCircle.tsx create mode 100644 ui/lib/Icon/Icon.stories.tsx create mode 100644 ui/lib/Icon/Icon.tsx create mode 100644 ui/lib/Icon/IntermediateProgressCircle.tsx create mode 100644 ui/lib/Icon/index.ts create mode 100644 ui/lib/Label/Label.stories.tsx create mode 100644 ui/lib/Label/Label.styles.ts create mode 100644 ui/lib/Label/Label.tsx create mode 100644 ui/lib/Label/index.ts create mode 100644 ui/lib/Layout/BaseLayout.styles.ts create mode 100644 ui/lib/Layout/BaseLayout.ts create mode 100644 ui/lib/Layout/BaseLayoutColumn.tsx create mode 100644 ui/lib/Layout/BaseLayoutRow.tsx create mode 100644 ui/lib/Layout/ColumnWithTextField.stories.tsx create mode 100644 ui/lib/Layout/Layout.stories.tsx create mode 100644 ui/lib/Layout/Layout.styles.ts create mode 100644 ui/lib/Layout/Layout.tsx create mode 100644 ui/lib/Layout/Row.stories.tsx create mode 100644 ui/lib/Layout/Row.styles.ts create mode 100644 ui/lib/Layout/Row.tsx create mode 100644 ui/lib/Layout/RowWIthCheckbox.stories.tsx create mode 100644 ui/lib/Layout/RowWithButton.stories.tsx create mode 100644 ui/lib/Layout/RowWithProgressBar.stories.tsx create mode 100644 ui/lib/Layout/RowWithRadioButton.stories.tsx create mode 100644 ui/lib/Layout/RowWithSelect.stories.tsx create mode 100644 ui/lib/Layout/RowWithSwitch.stories.tsx create mode 100644 ui/lib/Layout/RowWithTextField.stories.tsx create mode 100644 ui/lib/Layout/RowWithToggleGroup.stories.tsx create mode 100644 ui/lib/Layout/checker.png create mode 100644 ui/lib/Layout/index.ts create mode 100644 ui/lib/Media/Media.stories.tsx create mode 100644 ui/lib/Media/Media.styles.ts create mode 100644 ui/lib/Media/Media.tsx create mode 100644 ui/lib/Media/checker.png create mode 100644 ui/lib/Media/index.ts create mode 100644 ui/lib/Panel/Panel.tsx create mode 100644 ui/lib/Panel/index.ts create mode 100644 ui/lib/ProgressBar/ProgressBar.stories.tsx create mode 100644 ui/lib/ProgressBar/ProgressBar.styles.ts create mode 100644 ui/lib/ProgressBar/ProgressBar.tsx create mode 100644 ui/lib/ProgressBar/index.ts create mode 100644 ui/lib/ProgressCircle/ProgressCircle.stories.tsx create mode 100644 ui/lib/ProgressCircle/ProgressCircle.styles.ts create mode 100644 ui/lib/ProgressCircle/ProgressCircle.tsx create mode 100644 ui/lib/ProgressCircle/index.ts create mode 100644 ui/lib/RadioButtonGroup/RadioButtonGroup.stories.tsx create mode 100644 ui/lib/RadioButtonGroup/RadioButtonGroup.styles.ts create mode 100644 ui/lib/RadioButtonGroup/RadioButtonGroup.tsx create mode 100644 ui/lib/RadioButtonGroup/index.ts create mode 100644 ui/lib/SearchField/SearchField.stories.tsx create mode 100644 ui/lib/SearchField/SearchField.styles.ts create mode 100644 ui/lib/SearchField/SearchField.tsx create mode 100644 ui/lib/SearchField/index.ts create mode 100644 ui/lib/Select/Select.stories.tsx create mode 100644 ui/lib/Select/Select.styles.ts create mode 100644 ui/lib/Select/Select.tsx create mode 100644 ui/lib/Select/SelectItem.stories.tsx create mode 100644 ui/lib/Select/SelectItem.styles.ts create mode 100644 ui/lib/Select/SelectItem.tsx create mode 100644 ui/lib/Select/SelectMenu.stories.tsx create mode 100644 ui/lib/Select/SelectMenu.styles.ts create mode 100644 ui/lib/Select/SelectMenu.tsx create mode 100644 ui/lib/Select/index.ts create mode 100644 ui/lib/Separator/Separator.stories.tsx create mode 100644 ui/lib/Separator/Separator.styles.ts create mode 100644 ui/lib/Separator/Separator.tsx create mode 100644 ui/lib/Separator/index.ts create mode 100644 ui/lib/Sidebar/Sidebar.stories.tsx create mode 100644 ui/lib/Sidebar/Sidebar.styles.ts create mode 100644 ui/lib/Sidebar/Sidebar.tsx create mode 100644 ui/lib/Sidebar/SidebarItem.stories.tsx create mode 100644 ui/lib/Sidebar/SidebarItem.styles.ts create mode 100644 ui/lib/Sidebar/SidebarItem.tsx create mode 100644 ui/lib/Sidebar/index.ts create mode 100644 ui/lib/Slot/Slot.stories.tsx create mode 100644 ui/lib/Slot/Slot.styles.ts create mode 100644 ui/lib/Slot/Slot.tsx create mode 100644 ui/lib/Slot/index.ts create mode 100644 ui/lib/Switch/Switch.stories.tsx create mode 100644 ui/lib/Switch/Switch.styles.ts create mode 100644 ui/lib/Switch/Switch.tsx create mode 100644 ui/lib/Switch/index.ts create mode 100644 ui/lib/Table/Table.stories.tsx create mode 100644 ui/lib/Table/Table.styles.ts create mode 100644 ui/lib/Table/Table.tsx create mode 100644 ui/lib/Table/TableCell.stories.tsx create mode 100644 ui/lib/Table/TableCell.styles.ts create mode 100644 ui/lib/Table/TableCell.tsx create mode 100644 ui/lib/Table/TableRow.stories.tsx create mode 100644 ui/lib/Table/TableRow.tsx create mode 100644 ui/lib/Table/index.ts create mode 100644 ui/lib/TextField/TextField.stories.tsx create mode 100644 ui/lib/TextField/TextField.styles.ts create mode 100644 ui/lib/TextField/TextField.tsx create mode 100644 ui/lib/TextField/index.ts create mode 100644 ui/lib/Thumbnail/Thumbnail.stories.tsx create mode 100644 ui/lib/Thumbnail/Thumbnail.styles.ts create mode 100644 ui/lib/Thumbnail/Thumbnail.tsx create mode 100644 ui/lib/Thumbnail/checker.png create mode 100644 ui/lib/Thumbnail/index.ts create mode 100644 ui/lib/Toast/Toast.stories.tsx create mode 100644 ui/lib/Toast/Toast.styles.ts create mode 100644 ui/lib/Toast/Toast.tsx create mode 100644 ui/lib/Toast/index.ts create mode 100644 ui/lib/Toggle/ToggleButton/ToggleButton.stories.tsx create mode 100644 ui/lib/Toggle/ToggleButton/ToggleButton.styles.ts create mode 100644 ui/lib/Toggle/ToggleButton/ToggleButton.tsx create mode 100644 ui/lib/Toggle/ToggleButton/index.ts create mode 100644 ui/lib/Toggle/ToggleGroup/ToggleGroup.stories.tsx create mode 100644 ui/lib/Toggle/ToggleGroup/ToggleGroup.styles.ts create mode 100644 ui/lib/Toggle/ToggleGroup/ToggleGroup.tsx create mode 100644 ui/lib/Toggle/ToggleGroup/index.ts create mode 100644 ui/lib/Toggle/index.ts create mode 100644 ui/lib/Toggle/styles.ts create mode 100644 ui/lib/Tooltip/Tooltip.stories.tsx create mode 100644 ui/lib/Tooltip/Tooltip.styles.ts create mode 100644 ui/lib/Tooltip/Tooltip.tsx create mode 100644 ui/lib/Tooltip/index.ts create mode 100644 ui/lib/types.ts create mode 100644 ui/lib/utils/isDefined.ts create mode 100644 ui/lib/utils/renderObjectWith.ts create mode 100644 ui/next.config.js create mode 100644 ui/package.json create mode 100644 ui/postcss.config.js create mode 100644 ui/public/images/peerdb-combinedMark.svg create mode 100644 ui/rpc/promisify.ts create mode 100644 ui/rpc/rpc.ts create mode 100644 ui/tailwind.config.ts create mode 100644 ui/tsconfig.json create mode 100644 ui/yarn.lock diff --git a/.dockerignore b/.dockerignore index de0143f167..c182d1d56d 100644 --- a/.dockerignore +++ b/.dockerignore @@ -4,6 +4,11 @@ nexus/target thirdparty/**/* thirdparty +ui/node_modules/**/* +ui/node_modules +ui/.next/**/* +ui/.next + .git .gitignore .github diff --git a/buf.gen.yaml b/buf.gen.yaml index fd24bbc5b8..4ba8ac2851 100644 --- a/buf.gen.yaml +++ b/buf.gen.yaml @@ -22,3 +22,8 @@ plugins: out: nexus/pt/src opt: - ignore_unknown_fields=true + - plugin: buf.build/community/stephenh-ts-proto:v1.156.8 + out: ui/grpc_generated + opt: + - esModuleInterop=true + - outputServices=grpc-js diff --git a/docker-compose.yml b/docker-compose.yml index 8f65b14b4d..9472443616 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -12,11 +12,11 @@ x-flow-worker-env: &flow-worker-env AWS_ACCESS_KEY_ID: ${AWS_ACCESS_KEY_ID:-""} AWS_SECRET_ACCESS_KEY: ${AWS_SECRET_ACCESS_KEY:-""} AWS_REGION: ${AWS_REGION:-""} -# enables worker profiling using Go's pprof + # enables worker profiling using Go's pprof ENABLE_PROFILING: "true" -# enables exporting of mirror metrics to Prometheus for visualization using Grafana + # enables exporting of mirror metrics to Prometheus for visualization using Grafana ENABLE_METRICS: "true" -# enables exporting of mirror metrics to Catalog in the PEERDB_STATS schema. + # enables exporting of mirror metrics to Catalog in the PEERDB_STATS schema. ENABLE_STATS: "true" services: @@ -96,6 +96,7 @@ services: ports: - 8112:8112 environment: + <<: [*catalog-config] TEMPORAL_HOST_PORT: temporal:7233 depends_on: temporal-admin-tools: @@ -211,6 +212,17 @@ services: - multi-metrics - metrics + peerdb_ui: + container_name: peerdb_ui + build: + context: . + dockerfile: stacks/ui.Dockerfile + ports: + - 3000:3000 + environment: + <<: *catalog-config + PEERDB_FLOW_SERVER_ADDRESS: flow_api:8112 + volumes: pgdata: prometheusdata: diff --git a/flow/cmd/api.go b/flow/cmd/api.go index ccca98e853..d5c0946c05 100644 --- a/flow/cmd/api.go +++ b/flow/cmd/api.go @@ -5,6 +5,7 @@ import ( "fmt" "net" + utils "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" "github.com/PeerDB-io/peer-flow/generated/protos" log "github.com/sirupsen/logrus" "google.golang.org/grpc" @@ -32,7 +33,14 @@ func APIMain(args *APIServerParams) error { } grpcServer := grpc.NewServer() - flowHandler := NewFlowRequestHandler(tc) + + catalogConn, err := utils.GetCatalogConnectionPoolFromEnv() + if err != nil { + return fmt.Errorf("unable to get catalog connection pool: %w", err) + } + + flowHandler := NewFlowRequestHandler(tc, catalogConn) + defer flowHandler.Close() protos.RegisterFlowServiceServer(grpcServer, flowHandler) grpc_health_v1.RegisterHealthServer(grpcServer, health.NewServer()) diff --git a/flow/cmd/handler.go b/flow/cmd/handler.go index 5bfa19bcb7..2f16cb808c 100644 --- a/flow/cmd/handler.go +++ b/flow/cmd/handler.go @@ -8,18 +8,30 @@ import ( "github.com/PeerDB-io/peer-flow/shared" peerflow "github.com/PeerDB-io/peer-flow/workflows" "github.com/google/uuid" + "github.com/jackc/pgx/v5/pgxpool" + log "github.com/sirupsen/logrus" "go.temporal.io/sdk/client" + "google.golang.org/protobuf/proto" ) // grpc server implementation type FlowRequestHandler struct { temporalClient client.Client + pool *pgxpool.Pool protos.UnimplementedFlowServiceServer } -func NewFlowRequestHandler(temporalClient client.Client) *FlowRequestHandler { +func NewFlowRequestHandler(temporalClient client.Client, pool *pgxpool.Pool) *FlowRequestHandler { return &FlowRequestHandler{ temporalClient: temporalClient, + pool: pool, + } +} + +// Close closes the connection pool +func (h *FlowRequestHandler) Close() { + if h.pool != nil { + h.pool.Close() } } @@ -130,3 +142,93 @@ func (h *FlowRequestHandler) ShutdownFlow( Ok: true, }, nil } + +func (h *FlowRequestHandler) ListPeers( + ctx context.Context, + req *protos.ListPeersRequest, +) (*protos.ListPeersResponse, error) { + rows, err := h.pool.Query(ctx, "SELECT * FROM peers") + if err != nil { + return nil, fmt.Errorf("unable to query peers: %w", err) + } + defer rows.Close() + + peers := []*protos.Peer{} + for rows.Next() { + var id int + var name string + var peerType int + var options []byte + if err := rows.Scan(&id, &name, &peerType, &options); err != nil { + return nil, fmt.Errorf("unable to scan peer row: %w", err) + } + + dbtype := protos.DBType(peerType) + var peer *protos.Peer + switch dbtype { + case protos.DBType_POSTGRES: + var pgOptions protos.PostgresConfig + err := proto.Unmarshal(options, &pgOptions) + if err != nil { + return nil, fmt.Errorf("unable to unmarshal postgres options: %w", err) + } + peer = &protos.Peer{ + Name: name, + Type: dbtype, + Config: &protos.Peer_PostgresConfig{PostgresConfig: &pgOptions}, + } + case protos.DBType_BIGQUERY: + var bqOptions protos.BigqueryConfig + err := proto.Unmarshal(options, &bqOptions) + if err != nil { + return nil, fmt.Errorf("unable to unmarshal bigquery options: %w", err) + } + peer = &protos.Peer{ + Name: name, + Type: dbtype, + Config: &protos.Peer_BigqueryConfig{BigqueryConfig: &bqOptions}, + } + case protos.DBType_SNOWFLAKE: + var sfOptions protos.SnowflakeConfig + err := proto.Unmarshal(options, &sfOptions) + if err != nil { + return nil, fmt.Errorf("unable to unmarshal snowflake options: %w", err) + } + peer = &protos.Peer{ + Name: name, + Type: dbtype, + Config: &protos.Peer_SnowflakeConfig{SnowflakeConfig: &sfOptions}, + } + case protos.DBType_EVENTHUB: + var ehOptions protos.EventHubConfig + err := proto.Unmarshal(options, &ehOptions) + if err != nil { + return nil, fmt.Errorf("unable to unmarshal eventhub options: %w", err) + } + peer = &protos.Peer{ + Name: name, + Type: dbtype, + Config: &protos.Peer_EventhubConfig{EventhubConfig: &ehOptions}, + } + case protos.DBType_SQLSERVER: + var ssOptions protos.SqlServerConfig + err := proto.Unmarshal(options, &ssOptions) + if err != nil { + return nil, fmt.Errorf("unable to unmarshal sqlserver options: %w", err) + } + peer = &protos.Peer{ + Name: name, + Type: dbtype, + Config: &protos.Peer_SqlserverConfig{SqlserverConfig: &ssOptions}, + } + default: + log.Errorf("unsupported peer type for peer '%s': %v", name, dbtype) + } + + peers = append(peers, peer) + } + + return &protos.ListPeersResponse{ + Peers: peers, + }, nil +} diff --git a/flow/cmd/worker.go b/flow/cmd/worker.go index 66968bc10f..6abdd7e05f 100644 --- a/flow/cmd/worker.go +++ b/flow/cmd/worker.go @@ -1,13 +1,11 @@ package main import ( - "context" "fmt" "net/http" "os" "os/signal" "runtime" - "strconv" "syscall" "time" @@ -15,12 +13,10 @@ import ( _ "net/http/pprof" "github.com/PeerDB-io/peer-flow/activities" - "github.com/PeerDB-io/peer-flow/connectors/utils" + utils "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" "github.com/PeerDB-io/peer-flow/connectors/utils/monitoring" - "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/shared" peerflow "github.com/PeerDB-io/peer-flow/workflows" - "github.com/jackc/pgx/v5/pgxpool" "github.com/uber-go/tally/v4" "github.com/uber-go/tally/v4/prometheus" @@ -88,15 +84,11 @@ func WorkerMain(opts *WorkerOptions) error { catalogMirrorMonitor := monitoring.NewCatalogMirrorMonitor(nil) if opts.EnableMonitoring { - catalogConnectionString, err := genCatalogConnectionString() + conn, err := utils.GetCatalogConnectionPoolFromEnv() if err != nil { - log.Fatal(err) + return fmt.Errorf("unable to create catalog connection pool: %w", err) } - catalogConn, err := pgxpool.New(context.Background(), catalogConnectionString) - if err != nil { - return fmt.Errorf("unable to establish connection with catalog: %w", err) - } - catalogMirrorMonitor = monitoring.NewCatalogMirrorMonitor(catalogConn) + catalogMirrorMonitor = monitoring.NewCatalogMirrorMonitor(conn) } defer catalogMirrorMonitor.Close() @@ -153,38 +145,3 @@ func newPrometheusScope(c prometheus.Configuration) tally.Scope { log.Println("prometheus metrics scope created") return scope } - -func genCatalogConnectionString() (string, error) { - host, ok := os.LookupEnv("PEERDB_CATALOG_HOST") - if !ok { - return "", fmt.Errorf("PEERDB_CATALOG_HOST is not set") - } - portStr, ok := os.LookupEnv("PEERDB_CATALOG_PORT") - if !ok { - return "", fmt.Errorf("PEERDB_CATALOG_PORT is not set") - } - port, err := strconv.ParseUint(portStr, 10, 32) - if err != nil { - return "", fmt.Errorf("unable to parse PEERDB_CATALOG_PORT as unsigned integer") - } - user, ok := os.LookupEnv("PEERDB_CATALOG_USER") - if !ok { - return "", fmt.Errorf("PEERDB_CATALOG_USER is not set") - } - password, ok := os.LookupEnv("PEERDB_CATALOG_PASSWORD") - if !ok { - return "", fmt.Errorf("PEERDB_CATALOG_PASSWORD is not set") - } - database, ok := os.LookupEnv("PEERDB_CATALOG_DATABASE") - if !ok { - return "", fmt.Errorf("PEERDB_CATALOG_DATABASE is not set") - } - - return utils.GetPGConnectionString(&protos.PostgresConfig{ - Host: host, - Port: uint32(port), - User: user, - Password: password, - Database: database, - }), nil -} diff --git a/flow/connectors/utils/catalog/env.go b/flow/connectors/utils/catalog/env.go new file mode 100644 index 0000000000..3e29e82a6b --- /dev/null +++ b/flow/connectors/utils/catalog/env.go @@ -0,0 +1,61 @@ +package utils + +import ( + "context" + "fmt" + "os" + "strconv" + + "github.com/PeerDB-io/peer-flow/connectors/utils" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/jackc/pgx/v5/pgxpool" +) + +func GetCatalogConnectionPoolFromEnv() (*pgxpool.Pool, error) { + catalogConnectionString, err := genCatalogConnectionString() + if err != nil { + return nil, fmt.Errorf("unable to generate catalog connection string: %w", err) + } + + catalogConn, err := pgxpool.New(context.Background(), catalogConnectionString) + if err != nil { + return nil, fmt.Errorf("unable to establish connection with catalog: %w", err) + } + + return catalogConn, nil +} + +func genCatalogConnectionString() (string, error) { + host, ok := os.LookupEnv("PEERDB_CATALOG_HOST") + if !ok { + return "", fmt.Errorf("PEERDB_CATALOG_HOST is not set") + } + portStr, ok := os.LookupEnv("PEERDB_CATALOG_PORT") + if !ok { + return "", fmt.Errorf("PEERDB_CATALOG_PORT is not set") + } + port, err := strconv.ParseUint(portStr, 10, 32) + if err != nil { + return "", fmt.Errorf("unable to parse PEERDB_CATALOG_PORT as unsigned integer") + } + user, ok := os.LookupEnv("PEERDB_CATALOG_USER") + if !ok { + return "", fmt.Errorf("PEERDB_CATALOG_USER is not set") + } + password, ok := os.LookupEnv("PEERDB_CATALOG_PASSWORD") + if !ok { + return "", fmt.Errorf("PEERDB_CATALOG_PASSWORD is not set") + } + database, ok := os.LookupEnv("PEERDB_CATALOG_DATABASE") + if !ok { + return "", fmt.Errorf("PEERDB_CATALOG_DATABASE is not set") + } + + return utils.GetPGConnectionString(&protos.PostgresConfig{ + Host: host, + Port: uint32(port), + User: user, + Password: password, + Database: database, + }), nil +} diff --git a/flow/generated/protos/route.pb.go b/flow/generated/protos/route.pb.go index 9a4fda5b15..850d871d1e 100644 --- a/flow/generated/protos/route.pb.go +++ b/flow/generated/protos/route.pb.go @@ -335,6 +335,91 @@ func (x *ShutdownResponse) GetErrorMessage() string { return "" } +type ListPeersRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *ListPeersRequest) Reset() { + *x = ListPeersRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_route_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListPeersRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListPeersRequest) ProtoMessage() {} + +func (x *ListPeersRequest) ProtoReflect() protoreflect.Message { + mi := &file_route_proto_msgTypes[6] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListPeersRequest.ProtoReflect.Descriptor instead. +func (*ListPeersRequest) Descriptor() ([]byte, []int) { + return file_route_proto_rawDescGZIP(), []int{6} +} + +type ListPeersResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Peers []*Peer `protobuf:"bytes,1,rep,name=peers,proto3" json:"peers,omitempty"` +} + +func (x *ListPeersResponse) Reset() { + *x = ListPeersResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_route_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListPeersResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListPeersResponse) ProtoMessage() {} + +func (x *ListPeersResponse) ProtoReflect() protoreflect.Message { + mi := &file_route_proto_msgTypes[7] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ListPeersResponse.ProtoReflect.Descriptor instead. +func (*ListPeersResponse) Descriptor() ([]byte, []int) { + return file_route_proto_rawDescGZIP(), []int{7} +} + +func (x *ListPeersResponse) GetPeers() []*Peer { + if x != nil { + return x.Peers + } + return nil +} + var File_route_proto protoreflect.FileDescriptor var file_route_proto_rawDesc = []byte{ @@ -379,34 +464,44 @@ var file_route_proto_rawDesc = []byte{ 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x6f, 0x6b, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x02, 0x6f, 0x6b, 0x12, 0x23, 0x0a, 0x0d, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, - 0x65, 0x72, 0x72, 0x6f, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x32, 0x9c, 0x02, 0x0a, - 0x0b, 0x46, 0x6c, 0x6f, 0x77, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x5d, 0x0a, 0x0e, - 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x65, 0x65, 0x72, 0x46, 0x6c, 0x6f, 0x77, 0x12, 0x23, - 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x43, 0x72, - 0x65, 0x61, 0x74, 0x65, 0x50, 0x65, 0x65, 0x72, 0x46, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, - 0x74, 0x65, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x65, 0x65, 0x72, 0x46, 0x6c, 0x6f, - 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5d, 0x0a, 0x0e, 0x43, - 0x72, 0x65, 0x61, 0x74, 0x65, 0x51, 0x52, 0x65, 0x70, 0x46, 0x6c, 0x6f, 0x77, 0x12, 0x23, 0x2e, - 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x43, 0x72, 0x65, - 0x61, 0x74, 0x65, 0x51, 0x52, 0x65, 0x70, 0x46, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, - 0x65, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x51, 0x52, 0x65, 0x70, 0x46, 0x6c, 0x6f, 0x77, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4f, 0x0a, 0x0c, 0x53, 0x68, - 0x75, 0x74, 0x64, 0x6f, 0x77, 0x6e, 0x46, 0x6c, 0x6f, 0x77, 0x12, 0x1d, 0x2e, 0x70, 0x65, 0x65, - 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x53, 0x68, 0x75, 0x74, 0x64, 0x6f, - 0x77, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, - 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x53, 0x68, 0x75, 0x74, 0x64, 0x6f, 0x77, - 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x7c, 0x0a, 0x10, 0x63, - 0x6f, 0x6d, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x42, - 0x0a, 0x52, 0x6f, 0x75, 0x74, 0x65, 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, 0x0b, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x52, 0x6f, - 0x75, 0x74, 0x65, 0xca, 0x02, 0x0b, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x52, 0x6f, 0x75, 0x74, - 0x65, 0xe2, 0x02, 0x17, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x5c, - 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x0b, 0x50, 0x65, - 0x65, 0x72, 0x64, 0x62, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x33, + 0x65, 0x72, 0x72, 0x6f, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x12, 0x0a, 0x10, + 0x4c, 0x69, 0x73, 0x74, 0x50, 0x65, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x22, 0x3d, 0x0a, 0x11, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x65, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x28, 0x0a, 0x05, 0x70, 0x65, 0x65, 0x72, 0x73, 0x18, 0x01, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, + 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x05, 0x70, 0x65, 0x65, 0x72, 0x73, 0x32, + 0xec, 0x02, 0x0a, 0x0b, 0x46, 0x6c, 0x6f, 0x77, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, + 0x4e, 0x0a, 0x09, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x65, 0x65, 0x72, 0x73, 0x12, 0x1e, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x4c, 0x69, 0x73, 0x74, + 0x50, 0x65, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x4c, 0x69, 0x73, 0x74, + 0x50, 0x65, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x5d, 0x0a, 0x0e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x65, 0x65, 0x72, 0x46, 0x6c, 0x6f, + 0x77, 0x12, 0x23, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, + 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x65, 0x65, 0x72, 0x46, 0x6c, 0x6f, 0x77, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, + 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x65, 0x65, 0x72, + 0x46, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5d, + 0x0a, 0x0e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x51, 0x52, 0x65, 0x70, 0x46, 0x6c, 0x6f, 0x77, + 0x12, 0x23, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, + 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x51, 0x52, 0x65, 0x70, 0x46, 0x6c, 0x6f, 0x77, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, + 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x51, 0x52, 0x65, 0x70, 0x46, + 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4f, 0x0a, + 0x0c, 0x53, 0x68, 0x75, 0x74, 0x64, 0x6f, 0x77, 0x6e, 0x46, 0x6c, 0x6f, 0x77, 0x12, 0x1d, 0x2e, + 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x53, 0x68, 0x75, + 0x74, 0x64, 0x6f, 0x77, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x53, 0x68, 0x75, 0x74, + 0x64, 0x6f, 0x77, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x7c, + 0x0a, 0x10, 0x63, 0x6f, 0x6d, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, + 0x74, 0x65, 0x42, 0x0a, 0x52, 0x6f, 0x75, 0x74, 0x65, 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, 0x0b, 0x50, 0x65, 0x65, 0x72, 0x64, + 0x62, 0x52, 0x6f, 0x75, 0x74, 0x65, 0xca, 0x02, 0x0b, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x52, + 0x6f, 0x75, 0x74, 0x65, 0xe2, 0x02, 0x17, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x52, 0x6f, 0x75, + 0x74, 0x65, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, + 0x0b, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x62, 0x06, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -421,7 +516,7 @@ func file_route_proto_rawDescGZIP() []byte { return file_route_proto_rawDescData } -var file_route_proto_msgTypes = make([]protoimpl.MessageInfo, 6) +var file_route_proto_msgTypes = make([]protoimpl.MessageInfo, 8) var file_route_proto_goTypes = []interface{}{ (*CreatePeerFlowRequest)(nil), // 0: peerdb_route.CreatePeerFlowRequest (*CreatePeerFlowResponse)(nil), // 1: peerdb_route.CreatePeerFlowResponse @@ -429,26 +524,31 @@ var file_route_proto_goTypes = []interface{}{ (*CreateQRepFlowResponse)(nil), // 3: peerdb_route.CreateQRepFlowResponse (*ShutdownRequest)(nil), // 4: peerdb_route.ShutdownRequest (*ShutdownResponse)(nil), // 5: peerdb_route.ShutdownResponse - (*FlowConnectionConfigs)(nil), // 6: peerdb_flow.FlowConnectionConfigs - (*QRepConfig)(nil), // 7: peerdb_flow.QRepConfig - (*Peer)(nil), // 8: peerdb_peers.Peer + (*ListPeersRequest)(nil), // 6: peerdb_route.ListPeersRequest + (*ListPeersResponse)(nil), // 7: peerdb_route.ListPeersResponse + (*FlowConnectionConfigs)(nil), // 8: peerdb_flow.FlowConnectionConfigs + (*QRepConfig)(nil), // 9: peerdb_flow.QRepConfig + (*Peer)(nil), // 10: peerdb_peers.Peer } var file_route_proto_depIdxs = []int32{ - 6, // 0: peerdb_route.CreatePeerFlowRequest.connection_configs:type_name -> peerdb_flow.FlowConnectionConfigs - 7, // 1: peerdb_route.CreateQRepFlowRequest.qrep_config:type_name -> peerdb_flow.QRepConfig - 8, // 2: peerdb_route.ShutdownRequest.source_peer:type_name -> peerdb_peers.Peer - 8, // 3: peerdb_route.ShutdownRequest.destination_peer:type_name -> peerdb_peers.Peer - 0, // 4: peerdb_route.FlowService.CreatePeerFlow:input_type -> peerdb_route.CreatePeerFlowRequest - 2, // 5: peerdb_route.FlowService.CreateQRepFlow:input_type -> peerdb_route.CreateQRepFlowRequest - 4, // 6: peerdb_route.FlowService.ShutdownFlow:input_type -> peerdb_route.ShutdownRequest - 1, // 7: peerdb_route.FlowService.CreatePeerFlow:output_type -> peerdb_route.CreatePeerFlowResponse - 3, // 8: peerdb_route.FlowService.CreateQRepFlow:output_type -> peerdb_route.CreateQRepFlowResponse - 5, // 9: peerdb_route.FlowService.ShutdownFlow:output_type -> peerdb_route.ShutdownResponse - 7, // [7:10] is the sub-list for method output_type - 4, // [4:7] is the sub-list for method input_type - 4, // [4:4] is the sub-list for extension type_name - 4, // [4:4] is the sub-list for extension extendee - 0, // [0:4] is the sub-list for field type_name + 8, // 0: peerdb_route.CreatePeerFlowRequest.connection_configs:type_name -> peerdb_flow.FlowConnectionConfigs + 9, // 1: peerdb_route.CreateQRepFlowRequest.qrep_config:type_name -> peerdb_flow.QRepConfig + 10, // 2: peerdb_route.ShutdownRequest.source_peer:type_name -> peerdb_peers.Peer + 10, // 3: peerdb_route.ShutdownRequest.destination_peer:type_name -> peerdb_peers.Peer + 10, // 4: peerdb_route.ListPeersResponse.peers:type_name -> peerdb_peers.Peer + 6, // 5: peerdb_route.FlowService.ListPeers:input_type -> peerdb_route.ListPeersRequest + 0, // 6: peerdb_route.FlowService.CreatePeerFlow:input_type -> peerdb_route.CreatePeerFlowRequest + 2, // 7: peerdb_route.FlowService.CreateQRepFlow:input_type -> peerdb_route.CreateQRepFlowRequest + 4, // 8: peerdb_route.FlowService.ShutdownFlow:input_type -> peerdb_route.ShutdownRequest + 7, // 9: peerdb_route.FlowService.ListPeers:output_type -> peerdb_route.ListPeersResponse + 1, // 10: peerdb_route.FlowService.CreatePeerFlow:output_type -> peerdb_route.CreatePeerFlowResponse + 3, // 11: peerdb_route.FlowService.CreateQRepFlow:output_type -> peerdb_route.CreateQRepFlowResponse + 5, // 12: peerdb_route.FlowService.ShutdownFlow:output_type -> peerdb_route.ShutdownResponse + 9, // [9:13] is the sub-list for method output_type + 5, // [5:9] is the sub-list for method input_type + 5, // [5:5] is the sub-list for extension type_name + 5, // [5:5] is the sub-list for extension extendee + 0, // [0:5] is the sub-list for field type_name } func init() { file_route_proto_init() } @@ -531,6 +631,30 @@ func file_route_proto_init() { return nil } } + file_route_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ListPeersRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_route_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ListPeersResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } } type x struct{} out := protoimpl.TypeBuilder{ @@ -538,7 +662,7 @@ func file_route_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_route_proto_rawDesc, NumEnums: 0, - NumMessages: 6, + NumMessages: 8, NumExtensions: 0, NumServices: 1, }, diff --git a/flow/generated/protos/route_grpc.pb.go b/flow/generated/protos/route_grpc.pb.go index ec54177470..d688dc3337 100644 --- a/flow/generated/protos/route_grpc.pb.go +++ b/flow/generated/protos/route_grpc.pb.go @@ -19,6 +19,7 @@ import ( const _ = grpc.SupportPackageIsVersion7 const ( + FlowService_ListPeers_FullMethodName = "/peerdb_route.FlowService/ListPeers" FlowService_CreatePeerFlow_FullMethodName = "/peerdb_route.FlowService/CreatePeerFlow" FlowService_CreateQRepFlow_FullMethodName = "/peerdb_route.FlowService/CreateQRepFlow" FlowService_ShutdownFlow_FullMethodName = "/peerdb_route.FlowService/ShutdownFlow" @@ -28,6 +29,7 @@ const ( // // For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. type FlowServiceClient interface { + ListPeers(ctx context.Context, in *ListPeersRequest, opts ...grpc.CallOption) (*ListPeersResponse, error) CreatePeerFlow(ctx context.Context, in *CreatePeerFlowRequest, opts ...grpc.CallOption) (*CreatePeerFlowResponse, error) CreateQRepFlow(ctx context.Context, in *CreateQRepFlowRequest, opts ...grpc.CallOption) (*CreateQRepFlowResponse, error) ShutdownFlow(ctx context.Context, in *ShutdownRequest, opts ...grpc.CallOption) (*ShutdownResponse, error) @@ -41,6 +43,15 @@ func NewFlowServiceClient(cc grpc.ClientConnInterface) FlowServiceClient { return &flowServiceClient{cc} } +func (c *flowServiceClient) ListPeers(ctx context.Context, in *ListPeersRequest, opts ...grpc.CallOption) (*ListPeersResponse, error) { + out := new(ListPeersResponse) + err := c.cc.Invoke(ctx, FlowService_ListPeers_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *flowServiceClient) CreatePeerFlow(ctx context.Context, in *CreatePeerFlowRequest, opts ...grpc.CallOption) (*CreatePeerFlowResponse, error) { out := new(CreatePeerFlowResponse) err := c.cc.Invoke(ctx, FlowService_CreatePeerFlow_FullMethodName, in, out, opts...) @@ -72,6 +83,7 @@ func (c *flowServiceClient) ShutdownFlow(ctx context.Context, in *ShutdownReques // All implementations must embed UnimplementedFlowServiceServer // for forward compatibility type FlowServiceServer interface { + ListPeers(context.Context, *ListPeersRequest) (*ListPeersResponse, error) CreatePeerFlow(context.Context, *CreatePeerFlowRequest) (*CreatePeerFlowResponse, error) CreateQRepFlow(context.Context, *CreateQRepFlowRequest) (*CreateQRepFlowResponse, error) ShutdownFlow(context.Context, *ShutdownRequest) (*ShutdownResponse, error) @@ -82,6 +94,9 @@ type FlowServiceServer interface { type UnimplementedFlowServiceServer struct { } +func (UnimplementedFlowServiceServer) ListPeers(context.Context, *ListPeersRequest) (*ListPeersResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ListPeers not implemented") +} func (UnimplementedFlowServiceServer) CreatePeerFlow(context.Context, *CreatePeerFlowRequest) (*CreatePeerFlowResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method CreatePeerFlow not implemented") } @@ -104,6 +119,24 @@ func RegisterFlowServiceServer(s grpc.ServiceRegistrar, srv FlowServiceServer) { s.RegisterService(&FlowService_ServiceDesc, srv) } +func _FlowService_ListPeers_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ListPeersRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(FlowServiceServer).ListPeers(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: FlowService_ListPeers_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(FlowServiceServer).ListPeers(ctx, req.(*ListPeersRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _FlowService_CreatePeerFlow_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(CreatePeerFlowRequest) if err := dec(in); err != nil { @@ -165,6 +198,10 @@ var FlowService_ServiceDesc = grpc.ServiceDesc{ ServiceName: "peerdb_route.FlowService", HandlerType: (*FlowServiceServer)(nil), Methods: []grpc.MethodDesc{ + { + MethodName: "ListPeers", + Handler: _FlowService_ListPeers_Handler, + }, { MethodName: "CreatePeerFlow", Handler: _FlowService_CreatePeerFlow_Handler, diff --git a/nexus/pt/src/peerdb_route.rs b/nexus/pt/src/peerdb_route.rs index 2e3386a5a5..5a9a17e5d9 100644 --- a/nexus/pt/src/peerdb_route.rs +++ b/nexus/pt/src/peerdb_route.rs @@ -43,6 +43,16 @@ pub struct ShutdownResponse { #[prost(string, tag="2")] pub error_message: ::prost::alloc::string::String, } +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ListPeersRequest { +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ListPeersResponse { + #[prost(message, repeated, tag="1")] + pub peers: ::prost::alloc::vec::Vec, +} include!("peerdb_route.tonic.rs"); include!("peerdb_route.serde.rs"); // @@protoc_insertion_point(module) \ No newline at end of file diff --git a/nexus/pt/src/peerdb_route.serde.rs b/nexus/pt/src/peerdb_route.serde.rs index 1de74cb8f3..c9311a96a5 100644 --- a/nexus/pt/src/peerdb_route.serde.rs +++ b/nexus/pt/src/peerdb_route.serde.rs @@ -383,6 +383,173 @@ impl<'de> serde::Deserialize<'de> for CreateQRepFlowResponse { deserializer.deserialize_struct("peerdb_route.CreateQRepFlowResponse", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for ListPeersRequest { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let len = 0; + let struct_ser = serializer.serialize_struct("peerdb_route.ListPeersRequest", len)?; + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for ListPeersRequest { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + __SkipField__, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + Ok(GeneratedField::__SkipField__) + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = ListPeersRequest; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct peerdb_route.ListPeersRequest") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + while map.next_key::()?.is_some() { + let _ = map.next_value::()?; + } + Ok(ListPeersRequest { + }) + } + } + deserializer.deserialize_struct("peerdb_route.ListPeersRequest", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for ListPeersResponse { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.peers.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_route.ListPeersResponse", len)?; + if !self.peers.is_empty() { + struct_ser.serialize_field("peers", &self.peers)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for ListPeersResponse { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "peers", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Peers, + __SkipField__, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "peers" => Ok(GeneratedField::Peers), + _ => Ok(GeneratedField::__SkipField__), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = ListPeersResponse; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct peerdb_route.ListPeersResponse") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut peers__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::Peers => { + if peers__.is_some() { + return Err(serde::de::Error::duplicate_field("peers")); + } + peers__ = Some(map.next_value()?); + } + GeneratedField::__SkipField__ => { + let _ = map.next_value::()?; + } + } + } + Ok(ListPeersResponse { + peers: peers__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("peerdb_route.ListPeersResponse", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for ShutdownRequest { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result diff --git a/nexus/pt/src/peerdb_route.tonic.rs b/nexus/pt/src/peerdb_route.tonic.rs index 571965ea8c..62c5b9c58b 100644 --- a/nexus/pt/src/peerdb_route.tonic.rs +++ b/nexus/pt/src/peerdb_route.tonic.rs @@ -86,6 +86,32 @@ pub mod flow_service_client { self } /// + pub async fn list_peers( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/peerdb_route.FlowService/ListPeers", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("peerdb_route.FlowService", "ListPeers")); + self.inner.unary(req, path, codec).await + } + /// pub async fn create_peer_flow( &mut self, request: impl tonic::IntoRequest, @@ -172,6 +198,14 @@ pub mod flow_service_server { /// Generated trait containing gRPC methods that should be implemented for use with FlowServiceServer. #[async_trait] pub trait FlowService: Send + Sync + 'static { + /// + async fn list_peers( + &self, + request: tonic::Request, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + >; /// async fn create_peer_flow( &self, @@ -277,6 +311,50 @@ pub mod flow_service_server { fn call(&mut self, req: http::Request) -> Self::Future { let inner = self.inner.clone(); match req.uri().path() { + "/peerdb_route.FlowService/ListPeers" => { + #[allow(non_camel_case_types)] + struct ListPeersSvc(pub Arc); + impl< + T: FlowService, + > tonic::server::UnaryService + for ListPeersSvc { + type Response = super::ListPeersResponse; + type Future = BoxFuture< + tonic::Response, + tonic::Status, + >; + fn call( + &mut self, + request: tonic::Request, + ) -> Self::Future { + let inner = Arc::clone(&self.0); + let fut = async move { (*inner).list_peers(request).await }; + Box::pin(fut) + } + } + let accept_compression_encodings = self.accept_compression_encodings; + let send_compression_encodings = self.send_compression_encodings; + let max_decoding_message_size = self.max_decoding_message_size; + let max_encoding_message_size = self.max_encoding_message_size; + let inner = self.inner.clone(); + let fut = async move { + let inner = inner.0; + let method = ListPeersSvc(inner); + let codec = tonic::codec::ProstCodec::default(); + let mut grpc = tonic::server::Grpc::new(codec) + .apply_compression_config( + accept_compression_encodings, + send_compression_encodings, + ) + .apply_max_message_size_config( + max_decoding_message_size, + max_encoding_message_size, + ); + let res = grpc.unary(method, req).await; + Ok(res) + }; + Box::pin(fut) + } "/peerdb_route.FlowService/CreatePeerFlow" => { #[allow(non_camel_case_types)] struct CreatePeerFlowSvc(pub Arc); diff --git a/protos/route.proto b/protos/route.proto index 4665ee64fb..98c7d5e0e3 100644 --- a/protos/route.proto +++ b/protos/route.proto @@ -34,7 +34,15 @@ message ShutdownResponse { string error_message = 2; } +message ListPeersRequest { +} + +message ListPeersResponse { + repeated peerdb_peers.Peer peers = 1; +} + service FlowService { + rpc ListPeers(ListPeersRequest) returns (ListPeersResponse) {} rpc CreatePeerFlow(CreatePeerFlowRequest) returns (CreatePeerFlowResponse) {} rpc CreateQRepFlow(CreateQRepFlowRequest) returns (CreateQRepFlowResponse) {} rpc ShutdownFlow(ShutdownRequest) returns (ShutdownResponse) {} diff --git a/stacks/ui-entrypoint.sh b/stacks/ui-entrypoint.sh new file mode 100755 index 0000000000..9bee304074 --- /dev/null +++ b/stacks/ui-entrypoint.sh @@ -0,0 +1,3 @@ +#!/bin/sh +printenv > /app/.env.production +exec "$@" diff --git a/stacks/ui.Dockerfile b/stacks/ui.Dockerfile new file mode 100644 index 0000000000..3e58b5e646 --- /dev/null +++ b/stacks/ui.Dockerfile @@ -0,0 +1,55 @@ +# syntax=docker/dockerfile:1.2 + +# Base stage +FROM node:18-bookworm-slim AS base +WORKDIR /app + +# Dependencies stage +FROM base AS deps +WORKDIR /app +# Copy package.json and yarn.lock from ui folder to current directory in image +COPY ui/package.json ui/yarn.lock* ui/.yarnrc.yml ./ +RUN yarn --frozen-lockfile + +FROM base AS builder +WORKDIR /app + +COPY --from=deps /app/node_modules ./node_modules +COPY ui/ ./ + +ENV NEXT_TELEMETRY_DISABLED 1 +RUN yarn build + +# Builder stage +FROM base AS runner +WORKDIR /app +ENV NODE_ENV production +ENV NEXT_TELEMETRY_DISABLED 1 +RUN addgroup --system --gid 1001 nodejs +RUN adduser --system --uid 1001 nextjs + +COPY --from=builder /app/public ./public + +# Set the correct permission for prerender cache +RUN mkdir .next +RUN chown nextjs:nodejs .next + +# Automatically leverage output traces to reduce image size +# https://nextjs.org/docs/advanced-features/output-file-tracing +COPY --from=builder --chown=nextjs:nodejs /app/.next/standalone ./ +COPY --from=builder --chown=nextjs:nodejs /app/.next/static ./.next/static +COPY stacks/ui-entrypoint.sh /app/entrypoint.sh + +# allow permissions for nextjs user to do anything in /app +RUN chown -R nextjs:nodejs /app + +USER nextjs + +EXPOSE 3000 + +ENV PORT 3000 +# set hostname to localhost +ENV HOSTNAME "0.0.0.0" + +ENTRYPOINT ["/app/entrypoint.sh"] +CMD ["node", "server.js"] diff --git a/ui/.eslintignore b/ui/.eslintignore new file mode 100644 index 0000000000..b512c09d47 --- /dev/null +++ b/ui/.eslintignore @@ -0,0 +1 @@ +node_modules \ No newline at end of file diff --git a/ui/.eslintrc.json b/ui/.eslintrc.json new file mode 100644 index 0000000000..8b326a0cc5 --- /dev/null +++ b/ui/.eslintrc.json @@ -0,0 +1,7 @@ +{ + "extends": [ + "next", + "plugin:storybook/recommended", + "prettier" + ] +} diff --git a/ui/.gitignore b/ui/.gitignore new file mode 100644 index 0000000000..cda618479f --- /dev/null +++ b/ui/.gitignore @@ -0,0 +1,38 @@ +# See https://help.github.com/articles/ignoring-files/ for more about ignoring files. + +# dependencies +/node_modules +/.pnp +.pnp.js + +# testing +/coverage + +# next.js +/.next/ +/out/ + +# production +/build + +# misc +.DS_Store +*.pem + +# debug +npm-debug.log* +yarn-debug.log* +yarn-error.log* + +# local env files +.env*.local + +# vercel +.vercel + +# typescript +*.tsbuildinfo +next-env.d.ts + +.vscode +.yarn \ No newline at end of file diff --git a/ui/.nvmrc b/ui/.nvmrc new file mode 100644 index 0000000000..7ec56198d3 --- /dev/null +++ b/ui/.nvmrc @@ -0,0 +1 @@ +18.17.1 \ No newline at end of file diff --git a/ui/.prettierignore b/ui/.prettierignore new file mode 100644 index 0000000000..1380c2e74b --- /dev/null +++ b/ui/.prettierignore @@ -0,0 +1,2 @@ +node_modules +.next \ No newline at end of file diff --git a/ui/.prettierrc.json b/ui/.prettierrc.json new file mode 100644 index 0000000000..377f844553 --- /dev/null +++ b/ui/.prettierrc.json @@ -0,0 +1,8 @@ +{ + "trailingComma": "es5", + "semi": true, + "tabWidth": 2, + "singleQuote": true, + "jsxSingleQuote": true, + "plugins": ["prettier-plugin-organize-imports"] +} diff --git a/ui/.storybook/decorators.tsx b/ui/.storybook/decorators.tsx new file mode 100644 index 0000000000..311112c6aa --- /dev/null +++ b/ui/.storybook/decorators.tsx @@ -0,0 +1,8 @@ +import { Decorator } from '@storybook/react'; +import { AppThemeProvider } from '../lib/AppTheme'; + +export const withTheme: Decorator = (Story) => ( + + + +); diff --git a/ui/.storybook/main.ts b/ui/.storybook/main.ts new file mode 100644 index 0000000000..ec0111c96d --- /dev/null +++ b/ui/.storybook/main.ts @@ -0,0 +1,57 @@ +import type { StorybookConfig } from '@storybook/nextjs'; +import CopyPlugin from 'copy-webpack-plugin'; +import path from 'path'; + +const MATERIAL_ICONS_FONT_NODE_MODULE_PATH = + 'node_modules/material-symbols/material-symbols-sharp.woff2'; + +const config: StorybookConfig = { + stories: ['../lib/**/*.mdx', '../lib/**/*.stories.@(js|jsx|mjs|ts|tsx)'], + addons: [ + '@storybook/addon-links', + '@storybook/addon-essentials', + '@storybook/addon-interactions', + { + name: '@storybook/addon-styling', + options: { + postCss: { + implementation: require.resolve('postcss'), + }, + }, + }, + ], + framework: { + name: '@storybook/nextjs', + options: {}, + }, + docs: { + autodocs: 'tag', + }, + staticDirs: ['./public'], + webpackFinal(config) { + const iconsPath = path.resolve( + __dirname, + '..', + MATERIAL_ICONS_FONT_NODE_MODULE_PATH + ); + + const toPath = path.resolve( + __dirname, + 'public', + MATERIAL_ICONS_FONT_NODE_MODULE_PATH + ); + + config.plugins?.push( + new CopyPlugin({ + patterns: [ + { + from: iconsPath, + to: toPath, + }, + ], + }) + ); + return config; + }, +}; +export default config; diff --git a/ui/.storybook/preview.ts b/ui/.storybook/preview.ts new file mode 100644 index 0000000000..cfa5ad0e34 --- /dev/null +++ b/ui/.storybook/preview.ts @@ -0,0 +1,17 @@ +import type { Preview } from '@storybook/react'; +import { withTheme } from './decorators'; + +const preview: Preview = { + parameters: { + actions: { argTypesRegex: '^on[A-Z].*' }, + controls: { + matchers: { + color: /(background|color)$/i, + date: /Date$/, + }, + }, + }, + decorators: [withTheme], +}; + +export default preview; diff --git a/ui/.storybook/public/.gitignore b/ui/.storybook/public/.gitignore new file mode 100644 index 0000000000..86d0cb2726 --- /dev/null +++ b/ui/.storybook/public/.gitignore @@ -0,0 +1,4 @@ +# Ignore everything in this directory +* +# Except this file +!.gitignore \ No newline at end of file diff --git a/ui/.yarnrc.yml b/ui/.yarnrc.yml new file mode 100644 index 0000000000..91b1101f52 --- /dev/null +++ b/ui/.yarnrc.yml @@ -0,0 +1,5 @@ +compressionLevel: mixed + +enableGlobalCache: false + +nodeLinker: node-modules diff --git a/ui/README.md b/ui/README.md new file mode 100644 index 0000000000..5352c77e22 --- /dev/null +++ b/ui/README.md @@ -0,0 +1,50 @@ +PeerDB Cloud Template + +## Prerequisites + +- [NodeJS](https://nodejs.org/en): `18.17.1` +- [yarn](https://yarnpkg.com/) package manager: `3.6.1` + +## Getting Started + +Install dependencies using [yarn](https://classic.yarnpkg.com/en/) + +```bash +yarn +``` + +### Start the example project + +Start the NextJS dev server that runs the example project. + +```bash +yarn dev +``` + +Open [http://localhost:3000](http://localhost:3000) with your browser to see the result. + +The example project files resides inside the `app` folder. + +### Start the Storybook + +Every UI component has a [Storybook](https://storybook.js.org/) story attached to it. To run and view the Storybook for the project run the following + +```bash +yarn storybook +``` + +Open [http://localhost:6000](http://localhost:6000) with your browser to see the result. + +The stories and their corresponding components resides inside the `lib` folder. + +## Storybook Github pages + +The Storybook in this repositories Github pages at [Storybook](https://peerdb-io.github.io/peerdb-cloud-template). + +To deploy a new version of Storybook to Github pages run the script + +```bash +yarn storybook:deploy +``` + +It will automatically run the storybook build, push the content to the branch `gh-pages` to automatically deply the newly built Storybook to Github pages. diff --git a/ui/app/connectors/create/page.tsx b/ui/app/connectors/create/page.tsx new file mode 100644 index 0000000000..f55ba52cb4 --- /dev/null +++ b/ui/app/connectors/create/page.tsx @@ -0,0 +1,41 @@ +import { Action } from '@/lib/Action'; +import { Button } from '@/lib/Button'; +import { ButtonGroup } from '@/lib/ButtonGroup'; +import { Icon } from '@/lib/Icon'; +import { Label } from '@/lib/Label'; +import { LayoutMain, RowWithSelect } from '@/lib/Layout'; +import { Panel } from '@/lib/Panel'; +import { Select } from '@/lib/Select'; + +export default function CreateConnector() { + return ( + + + + + }>Learn about connectors + + + + + Data source + + } + action={ + + + + + + + + + + + + + + + } + > + {peers.map((peer) => ( + + ))} + + ); +} + +function Loading() { + return

🌀 Loading...

; +} + +export default async function Connectors() { + return ( + + +
+ New connector + + } + > + Connectors +
+
+ + }> + + + +
+ ); +} diff --git a/ui/app/dashboard/layout.tsx b/ui/app/dashboard/layout.tsx new file mode 100644 index 0000000000..69a53b44ea --- /dev/null +++ b/ui/app/dashboard/layout.tsx @@ -0,0 +1,7 @@ +import SidebarComponent from '@/components/SidebarComponent'; +import { Layout } from '@/lib/Layout'; +import { PropsWithChildren } from 'react'; + +export default function PageLayout({ children }: PropsWithChildren) { + return }>{children}; +} diff --git a/ui/app/dashboard/page.tsx b/ui/app/dashboard/page.tsx new file mode 100644 index 0000000000..3bd2bfb2a6 --- /dev/null +++ b/ui/app/dashboard/page.tsx @@ -0,0 +1,195 @@ +import { Badge } from '@/lib/Badge'; +import { Button } from '@/lib/Button'; +import { Checkbox } from '@/lib/Checkbox'; +import { Color } from '@/lib/Color'; +import { Icon } from '@/lib/Icon'; +import { Label } from '@/lib/Label'; +import { LayoutMain, Row } from '@/lib/Layout'; +import { Panel } from '@/lib/Panel'; +import { SearchField } from '@/lib/SearchField'; +import { Select } from '@/lib/Select'; +import { Table, TableCell, TableRow } from '@/lib/Table'; + +const Badges = [ + + + Active + , + + + Paused + , + + + Broken + , + + + Incomplete + , +]; + +const ExampleTable = ({ title }: { title: string }) => ( + {title}} + toolbar={{ + left: ( + <> + + + + + + + ), + right: , + }} + header={ + + + + + + + + + + + + + + + + + + } + > + {Array(8) + .fill(null) + .map((_, index) => ( + + + + + + + + + + + + + + + + + + + + {Badges[index % Badges.length]} + + + + + ))} +
+); + +export default function Dashboard() { + return ( + + + + + + +
+ + + -180.1% + + {' from last month'} + + } + /> + + + -180.1% + + {' from last month'} + + } + /> + + + -180.1% + + {' from last month'} + + } + /> +
+
+ + + + + + +
+ ); +} diff --git a/ui/app/favicon.ico b/ui/app/favicon.ico new file mode 100644 index 0000000000000000000000000000000000000000..7953005e8d8b94e3c8a92691da65a01aace2d400 GIT binary patch literal 4663 zcmd^D`8(8K^ncGFvLqjhER7}mn(SqXiew8zmNBG|ZEO?Sh6Hj}VK^U_I2vN%5)J@7f`11DD0m_A*CgED+!Vln5no~fcHfKE7XhF? zgNN?L0RRI1S1(<3jDoDrAW?9*Xy3+VFx2u+so&6baY3p71O$ivefsXXhI4fJqetKG z>E^X_A-h_kZCS53($6oD9`ac9?#l~k%4wXE@M=7fdGa6w+JI@9nY@#*>*Bi}H$FMI z_w0qARV5_?I~&hJt^wqFhxJ(=z@0^7S{eZQxPZ>HP$1+90Eo-{|Ah5VH9VEUcPA6% zc-O`F&Ak8f!s-6`e8Ghc<9qT8LdkS48Ys}x9OL=m@Y8F1?v_YVzLG~wDhA8_o$mr5 zhdl4j-njtr8=j7{VzD|!1O$>9tdW5r9&=!QqorPmsFdHA5QB_2uo6<2l!YHxdC?|BRH!sc zV8~AZ;Qh(32(TTd)P@P|a z`y0(jhU6K491iHs5a$)V%!T_!G;BVe;5gk9uB#Gv=>6@)_@PT!b_C#9CHeK){rcHe zo)3cFQcrl3VlFEz<^jA}M4oWI#A>IjysM`bADF1qJ3eC=ElbqB4yLJ#3p(EwGvhvN%${+HbljiSjN%t!cL!jA%kl?38srVApT4Hz-V-P)O?u05i+R+XO=Ey z0(8o9n4aKHT~JSP6UL9)pHg6Vto%JZdIjig+u<;=&4qW--=)TJcBy0I#IE?rH9Qye zG?(;zpmaY!GCj{wf%+jOifn9%y^Lj*dSI#l+)!l3-q?eeNBfXQ47sv&V3Df^cPVZ= z-bUEgas~~(wUz$nImjaME zp%(RtZ9%9gw;wW5NC1#H`)2U2mwy(w4xFpI|0vE_vPZU?-NbxFv;mQga2forlhi4L zY8R;%daEn=;<#120M376MRP!BFW}qTdz8tQSQByWecTM0X#KpH%g_FL(fvvbBOs#c zTGh`AgC_$keCWD#Yw9v)&$4j5ni(?CSkzXfwhGG}J}vx-A9Ce$E@|m2{HHmV7hSgt zS-g4cbcylVoWgO5vNNd!0AXZ^!1M@{%Au9nlQ`sJP;IcikwW zWM!x}pca#aM=O-bLnm)&<)EQ1>FIz^I_8>B4un!jTIm&AaSh zQ?9{24wxsv8}qO@_FC#!{(>NAO;Fp3%^u66HeKlTKrK!0^Q%sj zQ0L!}?Rl(j_K$-Yq#*;YDpjJ?8HVt;$Rg38E;|rE(~#dY&ISTdbS`*8BKQs=n}%ysf`oKar*4DEwyQ7TuR(1yaLG7m#a);5thdvo5Q0#sc?1g}N@ObTbf4z=T> zdbf98I|3U*PzIiSx=~I?N7a{@mSKgKCl-Y|WLhr8r<@giwl{UE@(;oB)#MMOq~QKQ zx5U>wljxQ2!%cD|5ZSRO6?394LUHy}Coz84AlWomY&aU$dsC!wg$&J>=VDN7=Q|7g zNJtaQpj`gpOElz?M3eK$=+*-2WWtVlKI65wW<+4`cxt^t%sF0HUmXs{PHO_0Oy>84 zMIlB5RQto!Pr;~K6BD>($v8|vYSqaqtJ_fu)EUb8(qm}$gq^(tQ`i;xNRIEVl<*NJ zUtudq$vg?S5n7a!3{mba3UYAAy%98Po3~3bNWL}hQ!RA5>y7wZ+f?DeNYI0)LUc9l z+q3p6XqW@p-4d~7A(M<@|#)}&^8pQS#pfMor2?_4w6X&fCND>(1J*xTA* zKlrB{#%=-|H3^&2I68KPhP+xYPmI9-f`8cC`xYoXQvng_0hh*zw|3-rasl?a*(cCk0a!zLr@s2 zR}8ci|oGsmK(0qJsg zChDXcho3^`1ZHzW0-fyN4=$eXo5AcSjU>vrxS4Yjf=jEQPSQ55Z5tEK6k_Vo zgH5Lb4Ph(jU@QW?RQDx|zlvV#7(?#)%26{D zu4nG)YgO~<=t{uTIF34Ge};J2;Gd*9clv%I}oP#^(3{OTd^N<%b5M!VOEe?ZxbJp zK3i=tBXwZ#+7kArMvMC5AH#!X-;=G}i7z4r{2vz^+;}3V z!M@mirlc5r_4YcB1Pa+|4?85rJ9Q|s``8wp!={mMkN0YI~Mr0KQbSe<4rZVi0)3=B6U5`zDd_~sn$UM_OxorOaQ-o|1!gSU z87axh2Bf;R!7^Zt*hd6NKUM@|%!lu3U7+tz)y5w;uvG38f(n{rzxj>0a|JE(`AJ!J zL{vCY8aLNpm)h(_i1=(6qT*C^y-r*+>s}J@(&+XM#73slQP$hMxLxzD#NwpLC9`mDyS4kBk5S8;09O@Me7_%h>!*YTm@IH;Nf;IOs7#XX?RLD62l%~acHP3% zz-A@q<7**>HPb!pk!zHW<19t+`icn+V=Sski5EY-TirF2w29Y|&KtqU&-k%!YmtoxkQ(&5t(lKxl&r7XX!J&SPCn^7Sc*ZGPMK=bKBjgI! zUPuTm8{DYUht}|p&Wl&^j=TDIr3*MP-C%tGr)Ibzf$a#Iy$FuOjpu<=D&|#KZvrW7 z@Gl=XJ0=+wz|%AiU5vx0uMt~XbuyKte9ld+0pFy_n@HIvl7nh9 zp}(8@00$ICEiEOg55hGV`41L+2X|FTX)rl(y_MB`su*nO2&U;f>j3Kij0qi>ON?dsF9YPyBlpc6Po6w7luzAseN`oq3p6o4Kn1Ex`~RXwJlM@mB5}j zpv;vyOaIaG%ZPr$P{I!OzJzd{>Gt}=3lb(26w&C}L`LeweLT>vE(U*W6df>6$f4DU zLoP=j7;V)qZ6Z_pZkXv+u*!hb`HrQm0woB>=@-@E||<@@b!{FKwR$&70b|`gFgjsmSjf=rPPcWMAX} zC>TTcLCB51`k-u(GN0eu(h-k0pVjwhT6I<1T;9eS4)&=s70JhV{nQdRJ5J--JoURa zXQRFHV9dx2^GMm{LEDZE(+I8`$~c4WY;c^$%}?ogdkhG=etRf_1`GGHE!K0#FRhc? zjg5uTbA4GGE8PY+K7;jUBex0O^QCm*W=P2+jcWlxViZh&QjOu>vRyIT{sB*yx!W9}w`qx&BXIw3qD(!FaTNzWCOjdvGXE2nmd))c{W@m=W3Ho%g~1 z=dzy5ehG2{vca9OQvmA;wpRuV3o@V3+Jla;2)DT(Nzk63QrK9@AQZe?+}>;aes;=~ zhBU^W04Sgt-|A`YxHz|K4;4Y`f*LUTHYaZa5qB0>G9>{zRC07tj2>2C1&W{uhRjy~-3 z*-d_>jbK(p1vPfTfrwH%>Hyy$QvXlr+j%JyWeVm#`HEO99LUcDhqn>FvNvNQBs3?> zbV%SRKCS8P(FC<@kXTpPO6_hnS8TG#5G%}Y%>{qR0Yp2dtyZj`uN4h@Gv^i`P)q69 zuLKNOF|=`F2^TH1US~Z%dX)qgg$n=(MX)e-`ecKVbdS;8#j8d*;HRIfrNKl#bzeIxB~Tfg@MS#?|yn>4V4xHpyV0mWleNo%iEEuyQa$ z&oo03Z1MWv4uHff + + + {children} + + + + ); +} diff --git a/ui/app/mirrors/create/page.tsx b/ui/app/mirrors/create/page.tsx new file mode 100644 index 0000000000..75341d4910 --- /dev/null +++ b/ui/app/mirrors/create/page.tsx @@ -0,0 +1,119 @@ +import { Button } from '@/lib/Button'; +import { ButtonGroup } from '@/lib/ButtonGroup'; +import { Label } from '@/lib/Label'; +import { LayoutMain, RowWithSelect, RowWithTextField } from '@/lib/Layout'; +import { Panel } from '@/lib/Panel'; +import { Select } from '@/lib/Select'; +import { TextField } from '@/lib/TextField'; + +export default function CreateMirrors() { + return ( + + + + + + + + + Mirror type + + } + action={} + /> + + Destination + + } + action={ + + + + + + + + + + + + + + + + + + + Apple + Banana + Blueberry + Grapes + Pineapple + + ), + description: , + }, +} satisfies Meta; + +export const Default: StoryObj = {}; diff --git a/ui/lib/Layout/RowWithSwitch.stories.tsx b/ui/lib/Layout/RowWithSwitch.stories.tsx new file mode 100644 index 0000000000..ffd537aeb9 --- /dev/null +++ b/ui/lib/Layout/RowWithSwitch.stories.tsx @@ -0,0 +1,18 @@ +'use client'; + +import { Meta, StoryObj } from '@storybook/react'; +import { Label } from '../Label'; +import { Switch } from '../Switch'; +import { RowWithSwitch } from './Layout'; + +export default { + title: 'Components / Layout / RowWithSwitch', + component: RowWithSwitch, + args: { + label: , + action: , + description: , + }, +} satisfies Meta; + +export const Default: StoryObj = {}; diff --git a/ui/lib/Layout/RowWithTextField.stories.tsx b/ui/lib/Layout/RowWithTextField.stories.tsx new file mode 100644 index 0000000000..f51040b76d --- /dev/null +++ b/ui/lib/Layout/RowWithTextField.stories.tsx @@ -0,0 +1,24 @@ +'use client'; + +import { Meta, StoryObj } from '@storybook/react'; +import { Label } from '../Label'; +import { TextField } from '../TextField'; +import { RowWithTextField } from './Layout'; + +export default { + title: 'Components / Layout / RowWithTextField', + component: RowWithTextField, + args: { + label: , + action: , + description: , + }, +} satisfies Meta; + +export const Default: StoryObj = {}; + +export const WithInstruction: StoryObj = { + args: { + instruction: , + }, +}; diff --git a/ui/lib/Layout/RowWithToggleGroup.stories.tsx b/ui/lib/Layout/RowWithToggleGroup.stories.tsx new file mode 100644 index 0000000000..8848a000d7 --- /dev/null +++ b/ui/lib/Layout/RowWithToggleGroup.stories.tsx @@ -0,0 +1,30 @@ +'use client'; + +import { Meta, StoryObj } from '@storybook/react'; +import { Icon } from '../Icon'; +import { Label } from '../Label'; +import { ToggleGroup, ToggleGroupItem } from '../Toggle'; +import { RowWithToggleGroup } from './Layout'; + +export default { + title: 'Components / Layout / RowWithToggleGroup', + component: RowWithToggleGroup, + args: { + label: , + action: ( + + + + + + + + + + + + ), + }, +} satisfies Meta; + +export const Default: StoryObj = {}; diff --git a/ui/lib/Layout/checker.png b/ui/lib/Layout/checker.png new file mode 100644 index 0000000000000000000000000000000000000000..63aa6a4d3df52007b2cb4cded0502477256b58fa GIT binary patch literal 284 zcmeAS@N?(olHy`uVBq!ia0vp^4j|0I1|(Ny7TyC=k3C%+Ln`9l-ge|`ao}(X-0^>5 zHQz_|#D~S}RXFyqax71~*YbLA$?rFL9d?r+dpB@hU|QK(%%8--5!5)T@lI391fEHb zMipF&b`AU*tS?Mv)E?;ex3E9&-nNEiMT6PpOH3QtlNq1QRAOsLSmVfZpdndbKAwT2 zNr9uNDO1IbO}RChy-YhtKLH3QC{)}%Y-z`qz`&N^5P3F*BUV~dfW;AncqA7v; + +type Story = StoryObj; +export const AspectRatio1x1: Story = { + args: { + ratio: '1 / 1', + }, +}; + +export const AspectRatio2x1: Story = { + args: { + ratio: '2 / 1', + }, +}; + +export const AspectRatio2x3: Story = { + args: { + ratio: '2 / 3', + }, +}; + +export const AspectRatio3x2: Story = { + args: { + ratio: '3 / 2', + }, +}; + +export const AspectRatio3x4: Story = { + args: { + ratio: '3 / 4', + }, +}; + +export const AspectRatio4x3: Story = { + args: { + ratio: '4 / 3', + }, +}; + +export const AspectRatio4x5: Story = { + args: { + ratio: '4 / 5', + }, +}; + +export const AspectRatio5x4: Story = { + args: { + ratio: '5 / 4', + }, +}; + +export const AspectRatio9x16: Story = { + args: { + ratio: '9 / 16', + }, +}; + +export const AspectRatio16x9: Story = { + args: { + ratio: '16 / 9', + }, +}; + +export const AspectRatio21x9: Story = { + args: { + ratio: '21 / 9', + }, +}; + +export const AspectRatioGolden: Story = { + args: { + ratio: 'golden', + }, +}; diff --git a/ui/lib/Media/Media.styles.ts b/ui/lib/Media/Media.styles.ts new file mode 100644 index 0000000000..632ebca0f0 --- /dev/null +++ b/ui/lib/Media/Media.styles.ts @@ -0,0 +1,38 @@ +import styled from 'styled-components'; + +export type AspectRatio = + | '1 / 1' + | '2 / 1' + | '2 / 3' + | '3 / 2' + | '3 / 4' + | '4 / 3' + | '4 / 5' + | '5 / 4' + | '9 / 16' + | '16 / 9' + | '21 / 9' + | 'golden'; + +type BaseImageProps = { + $ratio: AspectRatio; +}; + +const GOLDEN_RATIO = 1.618033987; + +export const BaseImage = styled.img` + display: flex; + justify-content: center; + align-items: center; + width: 64px; + + border: 0; + border-radius: ${({ theme }) => theme.radius.xSmall}; + background-color: ${({ theme }) => theme.colors.base.background.normal}; + + object-fit: cover; + object-position: center; + + aspect-ratio: ${({ $ratio }) => + $ratio === 'golden' ? GOLDEN_RATIO : $ratio}; +`; diff --git a/ui/lib/Media/Media.tsx b/ui/lib/Media/Media.tsx new file mode 100644 index 0000000000..653f67fdaa --- /dev/null +++ b/ui/lib/Media/Media.tsx @@ -0,0 +1,16 @@ +'use client'; +import { ComponentProps } from 'react'; +import { AspectRatio, BaseImage } from './Media.styles'; + +type MediaProps = ComponentProps<'img'> & { + ratio: AspectRatio; + className?: string; +}; +/** + * Media component + * + * [Figma spec](https://www.figma.com/file/DBMDh1LNNvp9H99N9lZgJ7/PeerDB?type=design&node-id=1-1872&mode=dev) + */ +export function Media({ ratio, ...imageProps }: MediaProps) { + return ; +} diff --git a/ui/lib/Media/checker.png b/ui/lib/Media/checker.png new file mode 100644 index 0000000000000000000000000000000000000000..63aa6a4d3df52007b2cb4cded0502477256b58fa GIT binary patch literal 284 zcmeAS@N?(olHy`uVBq!ia0vp^4j|0I1|(Ny7TyC=k3C%+Ln`9l-ge|`ao}(X-0^>5 zHQz_|#D~S}RXFyqax71~*YbLA$?rFL9d?r+dpB@hU|QK(%%8--5!5)T@lI391fEHb zMipF&b`AU*tS?Mv)E?;ex3E9&-nNEiMT6PpOH3QtlNq1QRAOsLSmVfZpdndbKAwT2 zNr9uNDO1IbO}RChy-YhtKLH3QC{)}%Y-z`qz`&N^5P3F*BUV~dfW;AncqA7v theme.spacing.medium}; + display: flex; + flex-flow: column; +`; +Panel.displayName = 'Panel'; diff --git a/ui/lib/Panel/index.ts b/ui/lib/Panel/index.ts new file mode 100644 index 0000000000..8960d84f60 --- /dev/null +++ b/ui/lib/Panel/index.ts @@ -0,0 +1 @@ +export * from './Panel'; diff --git a/ui/lib/ProgressBar/ProgressBar.stories.tsx b/ui/lib/ProgressBar/ProgressBar.stories.tsx new file mode 100644 index 0000000000..bc3c059eb0 --- /dev/null +++ b/ui/lib/ProgressBar/ProgressBar.stories.tsx @@ -0,0 +1,14 @@ +import { Meta, StoryObj } from '@storybook/react'; +import { ProgressBar } from './ProgressBar'; + +export default { + title: 'Components / ProgressBar', + component: ProgressBar, + tags: ['autodocs'], + args: { + progress: 50, + }, +} satisfies Meta; + +type Story = StoryObj; +export const Default: Story = {}; diff --git a/ui/lib/ProgressBar/ProgressBar.styles.ts b/ui/lib/ProgressBar/ProgressBar.styles.ts new file mode 100644 index 0000000000..651361c2a9 --- /dev/null +++ b/ui/lib/ProgressBar/ProgressBar.styles.ts @@ -0,0 +1,31 @@ +import * as RadixProgress from '@radix-ui/react-progress'; +import styled from 'styled-components'; + +export const ProgressWrapper = styled.div` + display: flex; + align-items: center; + margin: ${({ theme }) => `${theme.spacing.xSmall} ${theme.spacing.medium}`}; + height: ${({ theme }) => theme.spacing.xxLarge}; +`; + +export const ProgressRoot = styled(RadixProgress.Root)` + position: relative; + overflow: hidden; + background-color: ${({ theme }) => theme.colors.base.border.subtle}; + border-radius: ${({ theme }) => theme.radius.xSmall}; + width: 100%; + height: ${({ theme }) => theme.spacing.xSmall}; +`; + +type ProgressIndicatorProps = { + $progress: number; +}; +export const ProgressIndicator = styled( + RadixProgress.Indicator +)` + background-color: ${({ theme }) => theme.colors.accent.fill.normal}; + width: 100%; + height: 100%; + translate: -${({ $progress }) => 100 - $progress}% 0%; + transition: translate 660ms cubic-bezier(0.65, 0, 0.35, 1); +`; diff --git a/ui/lib/ProgressBar/ProgressBar.tsx b/ui/lib/ProgressBar/ProgressBar.tsx new file mode 100644 index 0000000000..88f3045bd3 --- /dev/null +++ b/ui/lib/ProgressBar/ProgressBar.tsx @@ -0,0 +1,30 @@ +'use client'; +import { + ProgressIndicator, + ProgressRoot, + ProgressWrapper, +} from './ProgressBar.styles'; + +type ProgressBarProps = { + /** Progress value: 0 - 100 */ + progress: number; + + className?: string; +}; + +/** + * Progress bar + * + * [Figma spec](https://www.figma.com/file/DBMDh1LNNvp9H99N9lZgJ7/PeerDB?type=design&node-id=1-1406) + */ +export function ProgressBar({ progress, ...wrapperProps }: ProgressBarProps) { + const clampedProgress = Math.min(100, Math.max(0, progress)); + + return ( + + + + + + ); +} diff --git a/ui/lib/ProgressBar/index.ts b/ui/lib/ProgressBar/index.ts new file mode 100644 index 0000000000..51a1fa968a --- /dev/null +++ b/ui/lib/ProgressBar/index.ts @@ -0,0 +1 @@ +export * from './ProgressBar'; diff --git a/ui/lib/ProgressCircle/ProgressCircle.stories.tsx b/ui/lib/ProgressCircle/ProgressCircle.stories.tsx new file mode 100644 index 0000000000..895f29a6ad --- /dev/null +++ b/ui/lib/ProgressCircle/ProgressCircle.stories.tsx @@ -0,0 +1,21 @@ +import { Meta, StoryObj } from '@storybook/react'; +import { ProgressCircle } from './ProgressCircle'; + +export default { + title: 'Components / ProgressCircle', + component: ProgressCircle, + tags: ['autodocs'], +} satisfies Meta; + +type Story = StoryObj; +export const DeterminateProgressCircle: Story = { + args: { + variant: 'determinate_progress_circle', + }, +}; + +export const IndeterminateProgressCircle: Story = { + args: { + variant: 'intermediate_progress_circle', + }, +}; diff --git a/ui/lib/ProgressCircle/ProgressCircle.styles.ts b/ui/lib/ProgressCircle/ProgressCircle.styles.ts new file mode 100644 index 0000000000..61733dc0ac --- /dev/null +++ b/ui/lib/ProgressCircle/ProgressCircle.styles.ts @@ -0,0 +1,44 @@ +import styled, { RuleSet, css, keyframes } from 'styled-components'; +import { Icon, IconProps } from '../Icon'; + +const spin = keyframes` + from { + rotate: 0deg; + } + to { + rotate: 360deg; + } +`; + +const spin45degIncrements = css` + animation-name: ${spin}; + animation-duration: 1s; + animation-direction: normal; + animation-timing-function: steps(8, end); + animation-iteration-count: infinite; +`; + +const spinLinear = css` + animation-name: ${spin}; + animation-duration: 1s; + animation-direction: normal; + animation-timing-function: linear; + animation-iteration-count: infinite; +`; + +export type ProgressCircleVariant = Extract< + IconProps['name'], + 'determinate_progress_circle' | 'intermediate_progress_circle' +>; + +const variants = { + determinate_progress_circle: spinLinear, + intermediate_progress_circle: spin45degIncrements, +} satisfies Record; + +type BaseIconProps = { + $variant: ProgressCircleVariant; +}; +export const BaseIcon = styled(Icon)` + ${({ $variant }) => variants[$variant]} +`; diff --git a/ui/lib/ProgressCircle/ProgressCircle.tsx b/ui/lib/ProgressCircle/ProgressCircle.tsx new file mode 100644 index 0000000000..d09a0dcfed --- /dev/null +++ b/ui/lib/ProgressCircle/ProgressCircle.tsx @@ -0,0 +1,11 @@ +'use client'; +import { BaseIcon, ProgressCircleVariant } from './ProgressCircle.styles'; + +type ProgressCircleProps = { + variant: ProgressCircleVariant; + className?: string; +}; + +export function ProgressCircle({ variant, ...iconProps }: ProgressCircleProps) { + return ; +} diff --git a/ui/lib/ProgressCircle/index.ts b/ui/lib/ProgressCircle/index.ts new file mode 100644 index 0000000000..882ee3c574 --- /dev/null +++ b/ui/lib/ProgressCircle/index.ts @@ -0,0 +1 @@ +export * from './ProgressCircle'; diff --git a/ui/lib/RadioButtonGroup/RadioButtonGroup.stories.tsx b/ui/lib/RadioButtonGroup/RadioButtonGroup.stories.tsx new file mode 100644 index 0000000000..66576a34c7 --- /dev/null +++ b/ui/lib/RadioButtonGroup/RadioButtonGroup.stories.tsx @@ -0,0 +1,21 @@ +import { Meta, StoryObj } from '@storybook/react'; +import { RadioButton, RadioButtonGroup } from './RadioButtonGroup'; + +export default { + title: 'Components / Input / RadioButtonGroup', + component: RadioButtonGroup, + tags: ['autodocs'], + args: { + children: , + disabled: false, + }, +} satisfies Meta; + +type Story = StoryObj; +export const Default: Story = {}; + +export const Disabled: Story = { + args: { + disabled: true, + }, +}; diff --git a/ui/lib/RadioButtonGroup/RadioButtonGroup.styles.ts b/ui/lib/RadioButtonGroup/RadioButtonGroup.styles.ts new file mode 100644 index 0000000000..32689e0865 --- /dev/null +++ b/ui/lib/RadioButtonGroup/RadioButtonGroup.styles.ts @@ -0,0 +1,53 @@ +import * as RadixRadioGroup from '@radix-ui/react-radio-group'; +import styled from 'styled-components'; + +export const RadioButtonGroupRoot = styled(RadixRadioGroup.Root)``; + +export const RadioButtonItem = styled(RadixRadioGroup.Item)` + --border-color: ${({ theme }) => theme.colors.base.border.normal}; + --background-color: transparent; + + display: flex; + justify-content: center; + align-items: center; + margin: ${({ theme }) => + `calc(${theme.spacing.xxSmall} + ${theme.spacing.xSmall}) calc(${theme.spacing.xxSmall} + ${theme.spacing.medium})`}; + + border-radius: ${({ theme }) => theme.radius.xxLarge}; + border: 1px solid var(--border-color); + background-color: var(--background-color); + + width: 20px; + height: 20px; + flex: 0 0 20px; + + &:hover { + --border-color: ${({ theme }) => theme.colors.base.border.hovered}; + } + + &:focus { + outline: none; + } + + &:focus-visible { + outline: 2px solid ${({ theme }) => theme.colors.accent.border.normal}; + outline-offset: -2px; + } + + &[data-disabled] { + opacity: 0.3; + } + + &[data-state='checked'] { + --background-color: ${({ theme }) => theme.colors.accent.fill.normal}; + --border-color: ${({ theme }) => theme.colors.accent.fill.normal}; + } +`; + +export const RadioButtonIndicator = styled(RadixRadioGroup.Indicator)` + width: 8px; + height: 8px; + flex: 0 0 8px; + background-color: ${({ theme }) => theme.colors.special.fixed.white}; + border-radius: 50%; +`; diff --git a/ui/lib/RadioButtonGroup/RadioButtonGroup.tsx b/ui/lib/RadioButtonGroup/RadioButtonGroup.tsx new file mode 100644 index 0000000000..33769e5d85 --- /dev/null +++ b/ui/lib/RadioButtonGroup/RadioButtonGroup.tsx @@ -0,0 +1,35 @@ +'use client'; +import { ComponentProps } from 'react'; +import { + RadioButtonGroupRoot, + RadioButtonIndicator, + RadioButtonItem, +} from './RadioButtonGroup.styles'; + +type RadioButtonProps = ComponentProps; + +export function RadioButton({ ...radioButtonItemProps }: RadioButtonProps) { + return ( + + + + ); +} + +type RadioButtonGroupProps = ComponentProps; + +/** + * Radio button group inputfield + * + * [Figma spec](https://www.figma.com/file/DBMDh1LNNvp9H99N9lZgJ7/PeerDB?type=design&node-id=1-1147&mode=design&t=b6qwokbWgkPCcgVS-4) + */ +export function RadioButtonGroup({ + children, + ...radioButtonGroupProps +}: RadioButtonGroupProps) { + return ( + + {children} + + ); +} diff --git a/ui/lib/RadioButtonGroup/index.ts b/ui/lib/RadioButtonGroup/index.ts new file mode 100644 index 0000000000..3aa372d4ba --- /dev/null +++ b/ui/lib/RadioButtonGroup/index.ts @@ -0,0 +1 @@ +export * from './RadioButtonGroup'; diff --git a/ui/lib/SearchField/SearchField.stories.tsx b/ui/lib/SearchField/SearchField.stories.tsx new file mode 100644 index 0000000000..f717d61506 --- /dev/null +++ b/ui/lib/SearchField/SearchField.stories.tsx @@ -0,0 +1,20 @@ +import { Meta, StoryObj } from '@storybook/react'; +import { SearchField } from './SearchField'; + +export default { + title: 'Components / SearchField', + component: SearchField, + tags: ['autodocs'], + args: { + placeholder: 'Placeholder', + }, +} as Meta; + +type Story = StoryObj; +export const Default: Story = {}; + +export const Disabled: Story = { + args: { + disabled: true, + }, +}; diff --git a/ui/lib/SearchField/SearchField.styles.ts b/ui/lib/SearchField/SearchField.styles.ts new file mode 100644 index 0000000000..ae950f073c --- /dev/null +++ b/ui/lib/SearchField/SearchField.styles.ts @@ -0,0 +1,43 @@ +import styled, { css } from 'styled-components'; + +export const StyledWrapper = styled.div` + --border-color: ${({ theme }) => theme.colors.base.border.normal}; + + display: flex; + align-items: flex-start; + margin: ${({ theme }) => theme.spacing.xxSmall} 0px; + padding: ${({ theme }) => `${theme.spacing.xxSmall} ${theme.spacing.medium}`}; + column-gap: ${({ theme }) => theme.spacing.medium}; + border-radius: ${({ theme }) => theme.radius.medium}; + border: 1px solid var(--border-color); + + color: ${({ theme }) => theme.colors.base.text.highContrast}; + ${({ theme }) => css(theme.text.regular.body)}; + + &:focus-within { + outline: 2px solid ${({ theme }) => theme.colors.accent.border.normal}; + outline-offset: -2px; + } + + &:hover { + --border-color: ${({ theme }) => theme.colors.base.border.hovered}; + } + + &[data-disabled] { + opacity: 0.3; + } +`; + +export const BaseInput = styled.input` + all: unset; + + flex: 1 1 auto; + + &::placeholder { + color: ${({ theme }) => theme.colors.base.text.lowContrast}; + } + + &:focus { + outline: none; + } +`; diff --git a/ui/lib/SearchField/SearchField.tsx b/ui/lib/SearchField/SearchField.tsx new file mode 100644 index 0000000000..b010be769e --- /dev/null +++ b/ui/lib/SearchField/SearchField.tsx @@ -0,0 +1,18 @@ +'use client'; +import { ComponentProps } from 'react'; +import { Icon } from '../Icon'; +import { BaseInput, StyledWrapper } from './SearchField.styles'; + +type SearchFieldProps = {} & ComponentProps; + +export function SearchField({ className, ...inputProps }: SearchFieldProps) { + return ( + + + + + ); +} diff --git a/ui/lib/SearchField/index.ts b/ui/lib/SearchField/index.ts new file mode 100644 index 0000000000..9f5ef33472 --- /dev/null +++ b/ui/lib/SearchField/index.ts @@ -0,0 +1 @@ +export * from './SearchField'; diff --git a/ui/lib/Select/Select.stories.tsx b/ui/lib/Select/Select.stories.tsx new file mode 100644 index 0000000000..9a30caf43b --- /dev/null +++ b/ui/lib/Select/Select.stories.tsx @@ -0,0 +1,36 @@ +import { Meta, StoryObj } from '@storybook/react'; +import { Separator } from '../Separator'; +import { Select } from './Select'; +import { SelectItem } from './SelectItem'; + +export default { + title: 'Components / Input / Select', + component: Select, + tags: ['autodocs'], +} satisfies Meta; + +type Story = StoryObj; +export const Default: Story = { + render: () => ( + + ), +}; + +export const Disabled: Story = { + render: () => ( + + ), +}; diff --git a/ui/lib/Select/Select.styles.ts b/ui/lib/Select/Select.styles.ts new file mode 100644 index 0000000000..7f7b60c22a --- /dev/null +++ b/ui/lib/Select/Select.styles.ts @@ -0,0 +1,50 @@ +import * as RadixSelect from '@radix-ui/react-select'; +import styled, { css } from 'styled-components'; + +export const StyledSelectIcon = styled(RadixSelect.Icon)` + color: ${({ theme }) => theme.colors.base.text.lowContrast}; +`; + +export const StyledSelectTrigger = styled(RadixSelect.Trigger)` + all: unset; + + min-width: 128px; + width: 100%; + margin: ${({ theme }) => theme.spacing.xxSmall} 0px; + padding: ${({ theme }) => `${theme.spacing.xxSmall} ${theme.spacing.medium}`}; + + display: flex; + flex-flow: row nowrap; + + border-radius: ${({ theme }) => theme.radius.medium}; + color: ${({ theme }) => theme.colors.base.text.highContrast}; + ${({ theme }) => css(theme.text.regular.body)} + + &[data-state='open'], + &:focus-visible, + &:hover:not([data-disabled]) { + justify-content: space-between; + + > i { + color: ${({ theme }) => theme.colors.base.text.highContrast}; + } + } + + &[data-state='open'], + &:hover:not([data-disabled]) { + background-color: ${({ theme }) => theme.colors.base.surface.hovered}; + } + + &:focus { + outline: none; + } + + &:focus-visible { + outline: 2px solid ${({ theme }) => theme.colors.accent.border.normal}; + outline-offset: -2px; + } + + &[data-disabled] { + opacity: 0.3; + } +`; diff --git a/ui/lib/Select/Select.tsx b/ui/lib/Select/Select.tsx new file mode 100644 index 0000000000..13f733e3c5 --- /dev/null +++ b/ui/lib/Select/Select.tsx @@ -0,0 +1,34 @@ +'use client'; +// your-select.jsx +import * as RadixSelect from '@radix-ui/react-select'; +import { PropsWithChildren } from 'react'; +import { Icon } from '../Icon'; +import { StyledSelectIcon, StyledSelectTrigger } from './Select.styles'; +import { SelectMenu } from './SelectMenu'; + +type SelectProps = PropsWithChildren & { + placeholder?: string; + id?: string; + className?: string; +}; +export function Select({ + placeholder, + children, + id, + className, + ...selectRootProps +}: SelectProps) { + return ( + + + + + + + + + {children} + + + ); +} diff --git a/ui/lib/Select/SelectItem.stories.tsx b/ui/lib/Select/SelectItem.stories.tsx new file mode 100644 index 0000000000..e9e01e7603 --- /dev/null +++ b/ui/lib/Select/SelectItem.stories.tsx @@ -0,0 +1,36 @@ +import * as RadixSelect from '@radix-ui/react-select'; +import { Meta, StoryObj } from '@storybook/react'; +import { SelectItem } from './SelectItem'; + +export default { + title: 'Components / Input / Select / SelectItem', + component: SelectItem, + tags: ['autodocs'], + args: { + value: 'label', + }, +} satisfies Meta; + +type Story = StoryObj; +export const Default: Story = { + render: (props) => ( + + + Label + + + ), +}; + +export const Disabled: Story = { + args: { + disabled: true, + }, + render: (props) => ( + + + Label + + + ), +}; diff --git a/ui/lib/Select/SelectItem.styles.ts b/ui/lib/Select/SelectItem.styles.ts new file mode 100644 index 0000000000..32c489ad6d --- /dev/null +++ b/ui/lib/Select/SelectItem.styles.ts @@ -0,0 +1,32 @@ +import * as RadixSelect from '@radix-ui/react-select'; +import styled, { css } from 'styled-components'; + +export const StyledItem = styled(RadixSelect.Item)` + position: relative; + display: flex; + justify-content: flex-start; + + padding: 0; + padding-left: ${({ theme }) => theme.spacing['4xLarge']}; + padding-right: ${({ theme }) => theme.spacing.medium}; + + border-radius: ${({ theme }) => theme.radius.small}; + + color: ${({ theme }) => theme.colors.base.text.highContrast}; + ${({ theme }) => css(theme.text.regular.body)} + + &[data-state="checked"], + &[data-highlighted] { + background-color: ${({ theme }) => theme.colors.base.surface.hovered}; + outline: none; + } + + &[data-disabled] { + opacity: 0.3; + } +`; + +export const StyledIndicator = styled(RadixSelect.ItemIndicator)` + position: absolute; + left: ${({ theme }) => theme.spacing.medium}; +`; diff --git a/ui/lib/Select/SelectItem.tsx b/ui/lib/Select/SelectItem.tsx new file mode 100644 index 0000000000..b0ff603e52 --- /dev/null +++ b/ui/lib/Select/SelectItem.tsx @@ -0,0 +1,17 @@ +'use client'; +import * as RadixSelect from '@radix-ui/react-select'; +import type { PropsWithChildren } from 'react'; +import { Icon } from '../Icon'; +import { StyledIndicator, StyledItem } from './SelectItem.styles'; + +type SelectItemProps = PropsWithChildren; +export function SelectItem({ children, ...selectItemProps }: SelectItemProps) { + return ( + + + + + {children} + + ); +} diff --git a/ui/lib/Select/SelectMenu.stories.tsx b/ui/lib/Select/SelectMenu.stories.tsx new file mode 100644 index 0000000000..36cb6f0727 --- /dev/null +++ b/ui/lib/Select/SelectMenu.stories.tsx @@ -0,0 +1,28 @@ +import * as RadixSelect from '@radix-ui/react-select'; +import { Meta, StoryObj } from '@storybook/react'; +import { SelectItem } from './SelectItem'; +import { SelectMenu } from './SelectMenu'; + +export default { + title: 'Components / Input / Select / SelectMenu', + component: SelectMenu, + tags: ['autodocs'], +} satisfies Meta; + +type Story = StoryObj; +export const Default: Story = { + render: (props) => ( +
+ + + Item one + Item two + Item three + Item four + Item five + Item six + + +
+ ), +}; diff --git a/ui/lib/Select/SelectMenu.styles.ts b/ui/lib/Select/SelectMenu.styles.ts new file mode 100644 index 0000000000..82c22b3017 --- /dev/null +++ b/ui/lib/Select/SelectMenu.styles.ts @@ -0,0 +1,12 @@ +import * as RadixSelect from '@radix-ui/react-select'; +import styled, { css } from 'styled-components'; + +export const SelectContent = styled(RadixSelect.SelectContent)` + padding: ${({ theme }) => theme.spacing.medium}; + border: 1px solid ${({ theme }) => theme.colors.base.border.subtle}; + border-radius: ${({ theme }) => theme.radius.medium}; + background-color: ${({ theme }) => theme.colors.base.background.normal}; + ${({ theme }) => css(theme.dropShadow.large)}; + + width: var(--radix-popper-anchor-width, 'auto'); +`; diff --git a/ui/lib/Select/SelectMenu.tsx b/ui/lib/Select/SelectMenu.tsx new file mode 100644 index 0000000000..34aa128a37 --- /dev/null +++ b/ui/lib/Select/SelectMenu.tsx @@ -0,0 +1,13 @@ +'use client'; +import * as RadixSelect from '@radix-ui/react-select'; +import { PropsWithChildren } from 'react'; +import { SelectContent } from './SelectMenu.styles'; + +type SelectMenuProps = PropsWithChildren; +export function SelectMenu({ children, ...contentProps }: SelectMenuProps) { + return ( + + {children} + + ); +} diff --git a/ui/lib/Select/index.ts b/ui/lib/Select/index.ts new file mode 100644 index 0000000000..7669724a89 --- /dev/null +++ b/ui/lib/Select/index.ts @@ -0,0 +1,2 @@ +export * from './Select'; +export * from './SelectItem'; diff --git a/ui/lib/Separator/Separator.stories.tsx b/ui/lib/Separator/Separator.stories.tsx new file mode 100644 index 0000000000..5cf441785e --- /dev/null +++ b/ui/lib/Separator/Separator.stories.tsx @@ -0,0 +1,48 @@ +import { Meta, StoryObj } from '@storybook/react'; +import { Separator } from './Separator'; + +export default { + title: 'Components / Separator', + component: Separator, + tags: ['autodocs'], + args: { + height: 'tall', + variant: 'normal', + }, +} as Meta; + +type Story = StoryObj; +export const Normal: Story = { + args: { + variant: 'normal', + height: 'tall', + }, +}; + +export const Indent: Story = { + args: { + variant: 'indent', + height: 'tall', + }, +}; + +export const Thick: Story = { + args: { + variant: 'thick', + height: 'tall', + }, +}; + +export const Centered: Story = { + args: { + variant: 'centered', + height: 'tall', + }, +}; + +export const Empty: Story = { + args: { + variant: 'empty', + height: 'tall', + }, +}; diff --git a/ui/lib/Separator/Separator.styles.ts b/ui/lib/Separator/Separator.styles.ts new file mode 100644 index 0000000000..a220b1df48 --- /dev/null +++ b/ui/lib/Separator/Separator.styles.ts @@ -0,0 +1,70 @@ +import styled, { css } from 'styled-components'; + +const heights = { + thin: css` + --height: ${({ theme }) => theme.spacing.medium}; + `, + tall: css` + --height: ${({ theme }) => theme.spacing.xLarge}; + `, +}; + +const variants = { + normal: css` + --line-height: 1px; + --line-width: 100%; + `, + thick: css` + --line-height: 2px; + --line-width: 100%; + `, + indent: css` + --line-height: 1px; + --line-width: calc(100% - ${({ theme }) => theme.spacing.medium}); + &::before { + left: ${({ theme }) => theme.spacing.medium}; + } + `, + centered: css` + --line-height: 1px; + --line-width: calc(100% - ${({ theme }) => theme.spacing.medium} * 2); + &::before { + left: ${({ theme }) => theme.spacing.medium}; + } + `, + empty: css` + &::before { + content: none; + } + `, +}; + +export type SeparatorHeight = keyof typeof heights; +export type SeparatorVariant = keyof typeof variants; + +type BaseSeparatorProps = { + $height: SeparatorHeight; + $variant: SeparatorVariant; +}; + +export const BaseSeparator = styled.div` + width: 100%; + height: var(--height); + position: relative; + + &::before { + content: ''; + display: block; + position: absolute; + top: 50%; + translate: 0 -50%; + left: 0; + right: 0; + width: var(--line-width); + height: var(--line-height); + background-color: ${({ theme }) => theme.colors.base.border.subtle}; + } + + ${({ $height }) => heights[$height]} + ${({ $variant }) => variants[$variant]} +`; diff --git a/ui/lib/Separator/Separator.tsx b/ui/lib/Separator/Separator.tsx new file mode 100644 index 0000000000..a4219e2940 --- /dev/null +++ b/ui/lib/Separator/Separator.tsx @@ -0,0 +1,22 @@ +'use client'; +import { PolymorphicComponentProps } from '../types'; +import { + BaseSeparator, + SeparatorHeight, + SeparatorVariant, +} from './Separator.styles'; + +type SeparatorProps = { + height?: SeparatorHeight; + variant?: SeparatorVariant; +}; + +export function Separator({ + height = 'tall', + variant = 'centered', + ...separatorProps +}: PolymorphicComponentProps) { + return ( + + ); +} diff --git a/ui/lib/Separator/index.ts b/ui/lib/Separator/index.ts new file mode 100644 index 0000000000..6e97479974 --- /dev/null +++ b/ui/lib/Separator/index.ts @@ -0,0 +1 @@ +export * from './Separator'; diff --git a/ui/lib/Sidebar/Sidebar.stories.tsx b/ui/lib/Sidebar/Sidebar.stories.tsx new file mode 100644 index 0000000000..94cc07fe99 --- /dev/null +++ b/ui/lib/Sidebar/Sidebar.stories.tsx @@ -0,0 +1,41 @@ +import { Meta, StoryObj } from '@storybook/react'; +import { Color } from '../Color'; +import { Icon } from '../Icon'; +import { Label } from '../Label'; +import { Separator } from '../Separator'; +import { Sidebar } from './Sidebar'; +import { SidebarItem } from './SidebarItem'; + +export default { + title: 'Components / Sidebar', + component: Sidebar, + tags: ['autodocs'], + args: {}, +} as Meta; + +type Story = StoryObj; +export const Default: Story = { + render: (props) => ( + + + + + }>Recents + } selected> + Desktop + + }>Documents + }> + Downloads + + + + + + }> + Macintosh HD + + }>Network + + ), +}; diff --git a/ui/lib/Sidebar/Sidebar.styles.ts b/ui/lib/Sidebar/Sidebar.styles.ts new file mode 100644 index 0000000000..19397fda06 --- /dev/null +++ b/ui/lib/Sidebar/Sidebar.styles.ts @@ -0,0 +1,25 @@ +import styled from 'styled-components'; + +export const StyledWrapper = styled.aside` + padding: ${({ theme }) => theme.spacing.medium}; + display: flex; + flex-direction: column; + justify-content: space-between; + align-items: flex-start; + background-color: ${({ theme }) => theme.colors.base.background.subtle}; + border-right: 1px solid ${({ theme }) => theme.colors.base.border.subtle}; + height: 100vh; +`; + +export const StyledItemWrapper = styled.div` + flex: 1 1 auto; + width: 100%; + display: flex; + flex-flow: column; +`; + +export const BottomRowWrapper = styled.div` + width: 100%; + display: flex; + flex-direction: column; +`; diff --git a/ui/lib/Sidebar/Sidebar.tsx b/ui/lib/Sidebar/Sidebar.tsx new file mode 100644 index 0000000000..6911473197 --- /dev/null +++ b/ui/lib/Sidebar/Sidebar.tsx @@ -0,0 +1,66 @@ +'use client'; + +import { PropsWithChildren } from 'react'; +import { Separator } from '../Separator'; +import { PolymorphicComponentProps, RenderObject } from '../types'; +import { isDefined } from '../utils/isDefined'; +import { + BottomRowWrapper, + StyledItemWrapper, + StyledWrapper, +} from './Sidebar.styles'; + +type SidebarProps = PropsWithChildren<{ + topTitle?: RenderObject; + avatar?: RenderObject; + selectButton?: RenderObject; + bottomRow?: RenderObject; + bottomLabel?: RenderObject; + className?: string; +}>; + +export function Sidebar({ + avatar, + bottomRow, + bottomLabel, + selectButton, + children, + topTitle, + ...wrapperProps +}: PolymorphicComponentProps<'div', SidebarProps>) { + const TopTitle = isDefined(topTitle) && ( + <> + {topTitle} + + + ); + + const Avatar = isDefined(avatar) && avatar; + + const BottomRow = isDefined(bottomRow) && ( + <> + {bottomRow} + + + ); + + const BottomLabel = isDefined(bottomLabel) && bottomLabel; + + const SelectButton = isDefined(selectButton) && ( + <> + {selectButton} + + + ); + + return ( + + {TopTitle} + {Avatar} + {SelectButton} + {children} + {BottomRow} + {BottomLabel} + + ); +} diff --git a/ui/lib/Sidebar/SidebarItem.stories.tsx b/ui/lib/Sidebar/SidebarItem.stories.tsx new file mode 100644 index 0000000000..046646e78d --- /dev/null +++ b/ui/lib/Sidebar/SidebarItem.stories.tsx @@ -0,0 +1,35 @@ +import { Meta, StoryObj } from '@storybook/react'; +import { Icon } from '../Icon'; +import { SidebarItem } from './SidebarItem'; + +export default { + title: 'Components / Sidebar / SidebarItem', + component: SidebarItem, + tags: ['autodocs'], + args: { + selected: false, + disabled: false, + leadingIcon: , + trailingIcon: , + suffix: 'Suffix', + }, +} as Meta; + +type Story = StoryObj; +export const Default: Story = { + render: (props) => Label, +}; + +export const Selected: Story = { + args: { + selected: true, + }, + render: (props) => Label, +}; + +export const Disabled: Story = { + args: { + disabled: true, + }, + render: (props) => Label, +}; diff --git a/ui/lib/Sidebar/SidebarItem.styles.ts b/ui/lib/Sidebar/SidebarItem.styles.ts new file mode 100644 index 0000000000..c9f38194dc --- /dev/null +++ b/ui/lib/Sidebar/SidebarItem.styles.ts @@ -0,0 +1,73 @@ +import styled from 'styled-components'; +import { Label } from '../Label'; + +export const StyledLabel = styled(Label)` + padding: 0; + flex: 1 1 auto; + color: var( + --text-color, + ${({ theme }) => theme.colors.base.text.highContrast} + ); +`; + +export const StyledSuffix = styled(Label)` + padding: 0; + color: var( + --text-color-suffix, + ${({ theme }) => theme.colors.base.text.lowContrast} + ); +`; + +export const BaseItem = styled.button` + all: unset; + + display: flex; + flex-flow: row nowrap; + padding: ${({ theme }) => `${theme.spacing.xxSmall} ${theme.spacing.medium}`}; + margin: ${({ theme }) => theme.spacing.xxSmall} 0px; + column-gap: ${({ theme }) => theme.spacing.medium}; + align-items: center; + border-radius: ${({ theme }) => theme.radius.medium}; + + cursor: pointer; + + .sidebar-item-icon--leading { + color: var( + --text-color, + ${({ theme }) => theme.colors.base.text.lowContrast} + ); + } + .sidebar-item-icon--trailing { + color: var( + --text-color, + ${({ theme }) => theme.colors.base.text.lowContrast} + ); + } + + &[data-selected] { + --text-color: ${({ theme }) => theme.colors.special.fixed.white}; + --text-color-suffix: ${({ theme }) => theme.colors.special.fixed.white}; + background-color: ${({ theme }) => theme.colors.accent.fill.normal}; + } + + &:hover:not([data-selected]) { + background-color: ${({ theme }) => theme.colors.base.surface.hovered}; + } + + &[data-disabled] { + opacity: 0.3; + pointer-events: none; + } + + &:focus-visible:not([data-disabled]) { + outline: 2px solid ${({ theme }) => theme.colors.accent.border.normal}; + outline-offset: -2px; + .sidebar-item-icon--trailing { + --text-color: ${({ theme }) => theme.colors.base.text.highContrast}; + } + } + + &:focus { + outline: none; + } +`; diff --git a/ui/lib/Sidebar/SidebarItem.tsx b/ui/lib/Sidebar/SidebarItem.tsx new file mode 100644 index 0000000000..a476f14553 --- /dev/null +++ b/ui/lib/Sidebar/SidebarItem.tsx @@ -0,0 +1,49 @@ +'use client'; +import { PropsWithChildren } from 'react'; +import { PolymorphicComponentProps, RenderObject } from '../types'; +import { isDefined } from '../utils/isDefined'; +import { renderObjectWith } from '../utils/renderObjectWith'; +import { BaseItem, StyledLabel, StyledSuffix } from './SidebarItem.styles'; + +type SidebarItemProps = PropsWithChildren<{ + selected?: boolean; + disabled?: boolean; + leadingIcon?: RenderObject; + suffix?: string; + trailingIcon?: RenderObject; +}>; + +export function SidebarItem({ + children, + leadingIcon, + suffix, + trailingIcon, + selected, + disabled, + ...baseItemProps +}: PolymorphicComponentProps) { + const LeadingIcon = renderObjectWith(leadingIcon, { + className: 'sidebar-item-icon--leading', + }); + + const TrailingIcon = renderObjectWith(trailingIcon, { + className: 'sidebar-item-icon--trailing', + }); + + const Suffix = isDefined(suffix) && ( + {suffix} + ); + + return ( + + {LeadingIcon} + {children} + {Suffix} + {TrailingIcon} + + ); +} diff --git a/ui/lib/Sidebar/index.ts b/ui/lib/Sidebar/index.ts new file mode 100644 index 0000000000..e74c06d3ee --- /dev/null +++ b/ui/lib/Sidebar/index.ts @@ -0,0 +1,2 @@ +export * from './Sidebar'; +export * from './SidebarItem'; diff --git a/ui/lib/Slot/Slot.stories.tsx b/ui/lib/Slot/Slot.stories.tsx new file mode 100644 index 0000000000..0b93e695ca --- /dev/null +++ b/ui/lib/Slot/Slot.stories.tsx @@ -0,0 +1,15 @@ +import { Meta, StoryObj } from '@storybook/react'; +import { Label } from '../Label'; +import { Slot } from './Slot'; + +export default { + title: 'Components / Slot', + component: Slot, + tags: ['autodocs'], + args: { + children: , + }, +} as Meta; + +type Story = StoryObj; +export const Default: Story = {}; diff --git a/ui/lib/Slot/Slot.styles.ts b/ui/lib/Slot/Slot.styles.ts new file mode 100644 index 0000000000..4ec0bd8d9d --- /dev/null +++ b/ui/lib/Slot/Slot.styles.ts @@ -0,0 +1,7 @@ +import styled from 'styled-components'; + +export const SlotWrapper = styled.div` + padding: ${({ theme }) => `${theme.spacing.xSmall} ${theme.spacing.medium}`}; + background-color: ${({ theme }) => theme.colors.base.background.subtle}; + border: 1px solid ${({ theme }) => theme.colors.base.background.subtle}; +`; diff --git a/ui/lib/Slot/Slot.tsx b/ui/lib/Slot/Slot.tsx new file mode 100644 index 0000000000..fa895ae1be --- /dev/null +++ b/ui/lib/Slot/Slot.tsx @@ -0,0 +1,16 @@ +'use client'; +import { PropsWithChildren } from 'react'; +import { SlotWrapper } from './Slot.styles'; + +type SlotProps = PropsWithChildren<{ + className?: string; +}>; + +/** + * Slot component + * + * [Figma spec](https://www.figma.com/file/DBMDh1LNNvp9H99N9lZgJ7/PeerDB?type=design&node-id=9-1715&mode=design&t=RoHQa2CnPu35HWVa-4) + */ +export function Slot({ children, ...wrapperProps }: SlotProps) { + return {children}; +} diff --git a/ui/lib/Slot/index.ts b/ui/lib/Slot/index.ts new file mode 100644 index 0000000000..41835581cd --- /dev/null +++ b/ui/lib/Slot/index.ts @@ -0,0 +1 @@ +export * from './Slot'; diff --git a/ui/lib/Switch/Switch.stories.tsx b/ui/lib/Switch/Switch.stories.tsx new file mode 100644 index 0000000000..e6ba7c4112 --- /dev/null +++ b/ui/lib/Switch/Switch.stories.tsx @@ -0,0 +1,26 @@ +import { Meta, StoryObj } from '@storybook/react'; +import { Switch } from './Switch'; + +export default { + title: 'Components / Input / Switch', + component: Switch, + tags: ['autodocs'], + args: { + name: 'story', + }, +} satisfies Meta; + +type Story = StoryObj; +export const Default: Story = {}; + +export const Disabled: Story = { + args: { + disabled: true, + }, +}; + +export const Checked: Story = { + args: { + checked: true, + }, +}; diff --git a/ui/lib/Switch/Switch.styles.ts b/ui/lib/Switch/Switch.styles.ts new file mode 100644 index 0000000000..968378d55b --- /dev/null +++ b/ui/lib/Switch/Switch.styles.ts @@ -0,0 +1,56 @@ +import * as RadixSwitch from '@radix-ui/react-switch'; +import styled from 'styled-components'; + +export const SwitchRoot = styled(RadixSwitch.Root)` + --background-color: ${({ theme }) => theme.colors.base.fill.normal}; + + all: unset; + + width: 40px; + height: 24px; + + position: relative; + background-color: var(--background-color); + border-radius: ${({ theme }) => theme.radius.xxLarge}; + margin: ${({ theme }) => `${theme.spacing.xSmall} ${theme.spacing.medium}`}; + + &:hover { + --background-color: ${({ theme }) => theme.colors.base.fill.hovered}; + } + + &:focus { + outline: none; + } + + &:focus-visible { + outline: 2px solid ${({ theme }) => theme.colors.accent.border.normal}; + } + + &[data-disabled] { + opacity: 0.3; + } + + &[data-state='checked'] { + --background-color: ${({ theme }) => theme.colors.positive.fill.normal}; + } + + &[data-state='checked']:hover { + --background-color: ${({ theme }) => theme.colors.positive.fill.hovered}; + } +`; + +export const SwitchThumb = styled(RadixSwitch.Thumb)` + display: block; + width: 20px; + height: 20px; + border-radius: 50%; + + background-color: ${({ theme }) => theme.colors.base.surface.normal}; + + transition: translate 100ms; + translate: 2px 0; + + &[data-state='checked'] { + translate: 18px; + } +`; diff --git a/ui/lib/Switch/Switch.tsx b/ui/lib/Switch/Switch.tsx new file mode 100644 index 0000000000..225aa222ca --- /dev/null +++ b/ui/lib/Switch/Switch.tsx @@ -0,0 +1,17 @@ +'use client'; +import { ComponentProps } from 'react'; +import { SwitchRoot, SwitchThumb } from './Switch.styles'; + +type SwitchProps = ComponentProps; +/** + * Switch inputfield + * + * [Figma spec](https://www.figma.com/file/DBMDh1LNNvp9H99N9lZgJ7/PeerDB?type=design&node-id=1-1328&mode=design&t=b6qwokbWgkPCcgVS-4) + */ +export function Switch({ ...switchRootProps }: SwitchProps) { + return ( + + + + ); +} diff --git a/ui/lib/Switch/index.ts b/ui/lib/Switch/index.ts new file mode 100644 index 0000000000..1b19c1d39c --- /dev/null +++ b/ui/lib/Switch/index.ts @@ -0,0 +1 @@ +export * from './Switch'; diff --git a/ui/lib/Table/Table.stories.tsx b/ui/lib/Table/Table.stories.tsx new file mode 100644 index 0000000000..27c41ec3db --- /dev/null +++ b/ui/lib/Table/Table.stories.tsx @@ -0,0 +1,173 @@ +import { Meta, StoryObj } from '@storybook/react'; +import { Badge } from '../Badge'; +import { Button } from '../Button'; +import { Checkbox } from '../Checkbox'; +import { Icon } from '../Icon'; +import { Label } from '../Label'; +import { SearchField } from '../SearchField'; +import { Select } from '../Select'; +import { Table } from './Table'; +import { TableCell } from './TableCell'; +import { TableRow } from './TableRow'; + +export default { + title: 'Components / Table', + component: Table, + tags: ['autodocs'], + render: (props) => ( + Table title} + toolbar={{ + left: ( + <> + + + + + + + ), + right: ( + <> + + + + ), + }} + header={ + + + + + + + + + + + + + + + + + + } + > + + + + + + + + + + + + + + + + + + + + + Active + + + + + + + + + + + + + + + + + + + + + + + + + + Active + + + + + + + + + + + + + + + + + + + + + + + + + + Active + + + + + +
+ ), +} satisfies Meta; + +type Story = StoryObj; +export const Default: Story = {}; diff --git a/ui/lib/Table/Table.styles.ts b/ui/lib/Table/Table.styles.ts new file mode 100644 index 0000000000..8b9375656f --- /dev/null +++ b/ui/lib/Table/Table.styles.ts @@ -0,0 +1,34 @@ +import styled from 'styled-components'; + +export const StyledWrapper = styled.div` + display: flex; + flex-direction: column; + width: 100%; +`; + +export const TableWrapper = styled.div` + width: 100%; + overflow-x: auto; +`; + +export const StyledTable = styled.table` + width: 100%; + border-spacing: 0; + border-collapse: collapse; +`; + +export const StyledTableBody = styled.tbody``; + +export const StyledTableHeader = styled.thead``; + +export const ToolbarWrapper = styled.div` + display: flex; + flex-flow: row wrap; + justify-content: space-between; +`; + +export const ToolbarSlot = styled.div` + display: flex; + flex-flow: row nowrap; + column-gap: ${({ theme }) => theme.spacing.medium}; +`; diff --git a/ui/lib/Table/Table.tsx b/ui/lib/Table/Table.tsx new file mode 100644 index 0000000000..c23faffaa4 --- /dev/null +++ b/ui/lib/Table/Table.tsx @@ -0,0 +1,86 @@ +'use client'; + +import { Children, PropsWithChildren } from 'react'; +import { Separator } from '../Separator'; +import { RenderObject } from '../types'; +import { isDefined } from '../utils/isDefined'; +import { renderObjectWith } from '../utils/renderObjectWith'; +import { + StyledTable, + StyledTableBody, + StyledTableHeader, + StyledWrapper, + TableWrapper, + ToolbarSlot, + ToolbarWrapper, +} from './Table.styles'; + +type TableProps = PropsWithChildren<{ + title?: RenderObject; + toolbar?: { + left?: RenderObject; + right?: RenderObject; + }; + header?: RenderObject; +}>; + +export function Table({ title, toolbar, header, children }: TableProps) { + const arrayChildren = Children.toArray(children); + + const Title = isDefined(title) && ( + <> + {renderObjectWith(title, { variant: 'headline' })} + + + ); + + const ToolbarLeft = isDefined(toolbar?.left) && ( + {renderObjectWith(toolbar?.left)} + ); + const ToolbarRight = isDefined(toolbar?.right) && ( + {renderObjectWith(toolbar?.right)} + ); + + const Toolbar = isDefined(toolbar) && ( + <> + + {ToolbarLeft} + {ToolbarRight} + + + + ); + + const Header = isDefined(header) && ( + {renderObjectWith(header)} + ); + + return ( + + {Title} + {Toolbar} + + + {Header} + + {isDefined(header) && ( + + )} + {Children.map(arrayChildren, (child, index) => { + const isLast = index === arrayChildren.length - 1; + + return ( + <> + {child} + {!isLast && ( + + )} + + ); + })} + + + + + ); +} diff --git a/ui/lib/Table/TableCell.stories.tsx b/ui/lib/Table/TableCell.stories.tsx new file mode 100644 index 0000000000..3acbde9466 --- /dev/null +++ b/ui/lib/Table/TableCell.stories.tsx @@ -0,0 +1,44 @@ +import { Meta, StoryObj } from '@storybook/react'; +import { Checkbox } from '../Checkbox'; +import { Label } from '../Label'; +import { Select } from '../Select'; +import { TableCell } from './TableCell'; + +export default { + title: 'Components / Table / TableCell', + component: TableCell, +} satisfies Meta; + +type Story = StoryObj; +export const ButtonCell: Story = { + args: { + variant: 'button', + }, + render: (props) => ( + + + + ), +}; + +export const NormalCell: Story = { + args: { + variant: 'normal', + }, + render: (props) => ( + + + + + Active + + + + + + + + ), +} satisfies Meta; + +type Story = StoryObj; +export const Default: Story = {}; diff --git a/ui/lib/Table/TableRow.tsx b/ui/lib/Table/TableRow.tsx new file mode 100644 index 0000000000..494f870451 --- /dev/null +++ b/ui/lib/Table/TableRow.tsx @@ -0,0 +1,4 @@ +'use client'; +import styled from 'styled-components'; + +export const TableRow = styled.tr``; diff --git a/ui/lib/Table/index.ts b/ui/lib/Table/index.ts new file mode 100644 index 0000000000..f3ddb7c8f5 --- /dev/null +++ b/ui/lib/Table/index.ts @@ -0,0 +1,3 @@ +export * from './Table'; +export * from './TableCell'; +export * from './TableRow'; diff --git a/ui/lib/TextField/TextField.stories.tsx b/ui/lib/TextField/TextField.stories.tsx new file mode 100644 index 0000000000..4f0d4335bc --- /dev/null +++ b/ui/lib/TextField/TextField.stories.tsx @@ -0,0 +1,26 @@ +import { Meta, StoryObj } from '@storybook/react'; +import { TextField } from './TextField'; + +export default { + title: 'Components / Input / TextField', + component: TextField, + tags: ['autodocs'], + args: { + placeholder: 'Placeholder', + disabled: false, + }, +} satisfies Meta; + +type Story = StoryObj; + +export const SimpleField: Story = { + args: { + variant: 'simple', + }, +}; + +export const TextAreaField: Story = { + args: { + variant: 'text-area', + }, +}; diff --git a/ui/lib/TextField/TextField.styles.ts b/ui/lib/TextField/TextField.styles.ts new file mode 100644 index 0000000000..a287761761 --- /dev/null +++ b/ui/lib/TextField/TextField.styles.ts @@ -0,0 +1,48 @@ +import styled, { css } from 'styled-components'; + +export const baseStyles = css` + --border-color: ${({ theme }) => theme.colors.base.border.normal}; + + display: flex; + width: 100%; + padding: ${({ theme }) => `${theme.spacing.xxSmall} ${theme.spacing.medium}`}; + margin: ${({ theme }) => `${theme.spacing.xxSmall} 0px`}; + border-radius: ${({ theme }) => theme.radius.xSmall}; + border: 1px solid var(--border-color); + background-color: ${({ theme }) => theme.colors.base.background.normal}; + + color: ${({ theme }) => theme.colors.base.text.highContrast}; + ${({ theme }) => theme.text.regular.body}; + + &::placeholder { + color: ${({ theme }) => theme.colors.base.text.lowContrast}; + } + + &:focus { + outline: none; + } + + &:hover { + --border-color: ${({ theme }) => theme.colors.base.border.hovered}; + } + + &:focus-visible { + outline: 2px solid ${({ theme }) => theme.colors.accent.border.normal}; + outline-offset: -2px; + } + + &:disabled { + opacity: 0.3; + } +`; + +export const BaseInputField = styled.input` + height: 28px; + ${baseStyles} +`; + +export const BaseTextArea = styled.textarea` + resize: none; + min-height: ${({ theme }) => theme.size.xxSmall}; + ${baseStyles} +`; diff --git a/ui/lib/TextField/TextField.tsx b/ui/lib/TextField/TextField.tsx new file mode 100644 index 0000000000..7a95191392 --- /dev/null +++ b/ui/lib/TextField/TextField.tsx @@ -0,0 +1,29 @@ +'use client'; +import { ComponentProps } from 'react'; +import { BaseInputField, BaseTextArea } from './TextField.styles'; + +type TextFieldProps = + | ({ + variant: 'simple'; + } & ComponentProps) + | ({ + variant: 'text-area'; + } & ComponentProps); + +/** + * TextField inputfield + * + * [Figma spec](https://www.figma.com/file/DBMDh1LNNvp9H99N9lZgJ7/PeerDB?type=design&node-id=1-246&mode=design&t=b6qwokbWgkPCcgVS-4) + */ +export function TextField({ variant, ...inputProps }: TextFieldProps) { + if (variant === 'text-area') { + return ( + )} /> + ); + } + return ( + )} + /> + ); +} diff --git a/ui/lib/TextField/index.ts b/ui/lib/TextField/index.ts new file mode 100644 index 0000000000..665fa3cb54 --- /dev/null +++ b/ui/lib/TextField/index.ts @@ -0,0 +1 @@ +export * from './TextField'; diff --git a/ui/lib/Thumbnail/Thumbnail.stories.tsx b/ui/lib/Thumbnail/Thumbnail.stories.tsx new file mode 100644 index 0000000000..1e77b8e248 --- /dev/null +++ b/ui/lib/Thumbnail/Thumbnail.stories.tsx @@ -0,0 +1,37 @@ +import { Meta, StoryObj } from '@storybook/react'; +import { Thumbnail } from '.'; +import checkerImage from './checker.png'; + +export default { + title: 'Components / Thumbnail', + component: Thumbnail, + tags: ['autodocs'], + args: { + src: checkerImage.src, + }, +} as Meta; + +type Story = StoryObj; +export const Small: Story = { + args: { + size: 'small', + }, +}; + +export const Medium: Story = { + args: { + size: 'medium', + }, +}; + +export const Large: Story = { + args: { + size: 'large', + }, +}; + +export const XLarge: Story = { + args: { + size: 'xLarge', + }, +}; diff --git a/ui/lib/Thumbnail/Thumbnail.styles.ts b/ui/lib/Thumbnail/Thumbnail.styles.ts new file mode 100644 index 0000000000..4fb2f91969 --- /dev/null +++ b/ui/lib/Thumbnail/Thumbnail.styles.ts @@ -0,0 +1,40 @@ +import styled, { css } from 'styled-components'; + +const sizes = { + small: css` + width: ${({ theme }) => theme.spacing.xxLarge}; + height: ${({ theme }) => theme.spacing.xxLarge}; + `, + medium: css` + width: ${({ theme }) => theme.spacing['3xLarge']}; + height: ${({ theme }) => theme.spacing['3xLarge']}; + `, + large: css` + width: ${({ theme }) => theme.spacing['5xLarge']}; + height: ${({ theme }) => theme.spacing['5xLarge']}; + `, + xLarge: css` + width: ${({ theme }) => theme.spacing['6xLarge']}; + height: ${({ theme }) => theme.spacing['6xLarge']}; + `, +}; + +export type ThumbnailSize = keyof typeof sizes; + +type BaseThumbnailProps = { + $size: ThumbnailSize; +}; + +export const BaseThumbnail = styled.img` + display: flex; + justify-content: center; + align-items: center; + border: 0; + border-radius: ${({ theme }) => theme.radius.xSmall}; + background-color: ${({ theme }) => theme.colors.base.background.normal}; + + object-fit: cover; + object-position: center; + + ${({ $size }) => sizes[$size]} +`; diff --git a/ui/lib/Thumbnail/Thumbnail.tsx b/ui/lib/Thumbnail/Thumbnail.tsx new file mode 100644 index 0000000000..9436f0eb15 --- /dev/null +++ b/ui/lib/Thumbnail/Thumbnail.tsx @@ -0,0 +1,17 @@ +'use client'; +import { ComponentProps } from 'react'; +import { BaseThumbnail, ThumbnailSize } from './Thumbnail.styles'; + +type ThumbnailProps = { + size: ThumbnailSize; + className?: string; +} & ComponentProps<'img'>; + +/** + * Thumbnail component + * + * [Figma spec](https://www.figma.com/file/DBMDh1LNNvp9H99N9lZgJ7/PeerDB?type=design&node-id=1-1532&mode=dev) + */ +export function Thumbnail({ size, ...imageProps }: ThumbnailProps) { + return ; +} diff --git a/ui/lib/Thumbnail/checker.png b/ui/lib/Thumbnail/checker.png new file mode 100644 index 0000000000000000000000000000000000000000..63aa6a4d3df52007b2cb4cded0502477256b58fa GIT binary patch literal 284 zcmeAS@N?(olHy`uVBq!ia0vp^4j|0I1|(Ny7TyC=k3C%+Ln`9l-ge|`ao}(X-0^>5 zHQz_|#D~S}RXFyqax71~*YbLA$?rFL9d?r+dpB@hU|QK(%%8--5!5)T@lI391fEHb zMipF&b`AU*tS?Mv)E?;ex3E9&-nNEiMT6PpOH3QtlNq1QRAOsLSmVfZpdndbKAwT2 zNr9uNDO1IbO}RChy-YhtKLH3QC{)}%Y-z`qz`&N^5P3F*BUV~dfW;AncqA7v, + open: true, + }, +} satisfies Meta; + +type Story = StoryObj; +export const DefaultInteractive: Story = {}; + +export const DefaultNonInteractive: Story = { + args: { + actionText: undefined, + }, +}; + +export const DefaultTimed: Story = { + args: { + icon: , + }, +}; diff --git a/ui/lib/Toast/Toast.styles.ts b/ui/lib/Toast/Toast.styles.ts new file mode 100644 index 0000000000..b9e7a89614 --- /dev/null +++ b/ui/lib/Toast/Toast.styles.ts @@ -0,0 +1,103 @@ +import * as RadixToast from '@radix-ui/react-toast'; +import { keyframes, styled } from 'styled-components'; + +export const ToastAction = styled(RadixToast.Action)` + all: unset; + cursor: pointer; + + display: flex; + justify-content: center; + align-items: center; + padding: ${({ theme }) => `${theme.spacing.medium} ${theme.spacing.large}`}; + border-left: 1px solid ${({ theme }) => theme.colors.base.border.normal}; + ${({ theme }) => theme.text.medium.body}; + + &:hover { + background-color: ${({ theme }) => theme.colors.base.surface.hovered}; + } + + &:focus { + outline: none; + } + + &:focus-visible { + outline: 2px solid ${({ theme }) => theme.colors.accent.border.normal}; + outline-offset: -2px; + } +`; + +export const ToastTitle = styled(RadixToast.Title)` + ${({ theme }) => theme.text.regular.body}; +`; + +const VIEWPORT_PADDING = 25; + +const hide = keyframes({ + '0%': { opacity: 1 }, + '100%': { opacity: 0 }, +}); + +const slideIn = keyframes({ + from: { transform: `translateX(calc(100% + ${VIEWPORT_PADDING}px))` }, + to: { transform: 'translateX(0)' }, +}); + +const swipeOut = keyframes({ + from: { transform: 'translateX(var(--radix-toast-swipe-end-x))' }, + to: { transform: `translateX(calc(100% + ${VIEWPORT_PADDING}px))` }, +}); + +export const ToastRoot = styled(RadixToast.Root)` + background-color: ${({ theme }) => theme.colors.base.background.normal}; + border-radius: ${({ theme }) => theme.radius.medium}; + border: 1px solid ${({ theme }) => theme.colors.base.border.normal}; + ${({ theme }) => theme.dropShadow.large}; + + display: inline-flex; + flex-flow: row nowrap; + + &[data-state='open'] { + animation: ${slideIn} 150ms cubic-bezier(0.16, 1, 0.3, 1); + } + + &[data-state='closed'] { + animation: ${hide} 100ms ease-in; + } + + &[data-swipe='move'] { + transform: translateX(var(--radix-toast-swipe-move-x)); + } + + &[data-swipe='cancel'] { + transform: translateX(0); + transition: transform 200ms ease-out; + } + + &[data-swipe='end'] { + animation: ${swipeOut} 100ms ease-out; + } +`; + +export const ToastContent = styled.div` + display: flex; + flex-flow: row nowrap; + align-items: center; + column-gap: ${({ theme }) => theme.spacing.medium}; + padding: ${({ theme }) => `${theme.spacing.medium} ${theme.spacing.large}`}; +`; + +export const ToastViewport = styled(RadixToast.Viewport)` + position: fixed; + top: 0; + right: 0; + display: flex; + flex-direction: column; + padding: ${VIEWPORT_PADDING}; + gap: 10; + width: 390; + max-width: 100vw; + margin: 0; + list-style: none; + z-index: 2147483647; + outline: none; +`; diff --git a/ui/lib/Toast/Toast.tsx b/ui/lib/Toast/Toast.tsx new file mode 100644 index 0000000000..2a4618088c --- /dev/null +++ b/ui/lib/Toast/Toast.tsx @@ -0,0 +1,54 @@ +'use client'; +import * as RadixToast from '@radix-ui/react-toast'; +import { RenderObject } from '../types'; +import { isDefined } from '../utils/isDefined'; +import { renderObjectWith } from '../utils/renderObjectWith'; +import { + ToastAction, + ToastContent, + ToastRoot, + ToastTitle, + ToastViewport, +} from './Toast.styles'; + +type ToastProps = RadixToast.ToastProps & { + message?: string; + icon?: RenderObject; + actionText?: string; + className?: string; +}; + +/** + * Toast component + * + * [Figma spec](https://www.figma.com/file/DBMDh1LNNvp9H99N9lZgJ7/PeerDB?type=design&node-id=1-1949&mode=design&t=GFIJ9vbM0Q2bR3Ml-4) + * + * Based on the [Radix Toast](https://www.radix-ui.com/primitives/docs/components/toast) component + */ +export function Toast({ + message, + icon, + actionText: action, + children, + ...rootProps +}: ToastProps) { + const Icon = renderObjectWith(icon); + const Action = isDefined(action) && ( + {action} + ); + const Message = isDefined(message) && {message}; + + return ( + + + + {Icon} + {Message} + {children} + + {Action} + + + + ); +} diff --git a/ui/lib/Toast/index.ts b/ui/lib/Toast/index.ts new file mode 100644 index 0000000000..1b794ee786 --- /dev/null +++ b/ui/lib/Toast/index.ts @@ -0,0 +1 @@ +export * from './Toast'; diff --git a/ui/lib/Toggle/ToggleButton/ToggleButton.stories.tsx b/ui/lib/Toggle/ToggleButton/ToggleButton.stories.tsx new file mode 100644 index 0000000000..98cce3d948 --- /dev/null +++ b/ui/lib/Toggle/ToggleButton/ToggleButton.stories.tsx @@ -0,0 +1,54 @@ +import { Meta, StoryObj } from '@storybook/react'; +import { Icon } from '../../Icon'; +import { ToggleButton } from './ToggleButton'; + +export default { + title: 'Components / Toggle / ToggleButton', + component: ToggleButton, + tags: ['autodocs'], + args: { + disabled: false, + pressed: false, + }, +} satisfies Meta; + +type Story = StoryObj; +export const DefaultIcon: Story = { + args: { + children: , + }, +}; + +export const DefaultLabel: Story = { + args: { + children: 'Toggle', + }, +}; + +export const SelectedIcon: Story = { + args: { + pressed: true, + children: , + }, +}; + +export const SelectedLabel: Story = { + args: { + pressed: true, + children: 'Toggle', + }, +}; + +export const DisabledIcon: Story = { + args: { + disabled: true, + children: , + }, +}; + +export const DisabledLabel: Story = { + args: { + disabled: true, + children: 'Toggle', + }, +}; diff --git a/ui/lib/Toggle/ToggleButton/ToggleButton.styles.ts b/ui/lib/Toggle/ToggleButton/ToggleButton.styles.ts new file mode 100644 index 0000000000..ae91ea6529 --- /dev/null +++ b/ui/lib/Toggle/ToggleButton/ToggleButton.styles.ts @@ -0,0 +1,7 @@ +import * as RadixToggle from '@radix-ui/react-toggle'; +import styled from 'styled-components'; +import { baseButtonStyle } from '../styles'; + +export const BaseToggleRoot = styled(RadixToggle.Root)` + ${baseButtonStyle} +`; diff --git a/ui/lib/Toggle/ToggleButton/ToggleButton.tsx b/ui/lib/Toggle/ToggleButton/ToggleButton.tsx new file mode 100644 index 0000000000..8fb3719902 --- /dev/null +++ b/ui/lib/Toggle/ToggleButton/ToggleButton.tsx @@ -0,0 +1,18 @@ +'use client'; +import * as RadixToggle from '@radix-ui/react-toggle'; +import { PropsWithChildren } from 'react'; +import { BaseToggleRoot } from './ToggleButton.styles'; + +type ToggleButtonProps = PropsWithChildren<{ className?: string }> & + RadixToggle.ToggleProps; + +/** + * Toggle button component + * + * [Figma spec](https://www.figma.com/file/DBMDh1LNNvp9H99N9lZgJ7/PeerDB?type=design&node-id=1-816&mode=design&t=fsqGSfRrNseMunYH-4) + * + * Based on the [Radix Toggle](https://www.radix-ui.com/primitives/docs/components/toggle) component + */ +export function ToggleButton({ children, ...rootProps }: ToggleButtonProps) { + return {children}; +} diff --git a/ui/lib/Toggle/ToggleButton/index.ts b/ui/lib/Toggle/ToggleButton/index.ts new file mode 100644 index 0000000000..650fcfda4c --- /dev/null +++ b/ui/lib/Toggle/ToggleButton/index.ts @@ -0,0 +1 @@ +export * from './ToggleButton'; diff --git a/ui/lib/Toggle/ToggleGroup/ToggleGroup.stories.tsx b/ui/lib/Toggle/ToggleGroup/ToggleGroup.stories.tsx new file mode 100644 index 0000000000..28ae43e410 --- /dev/null +++ b/ui/lib/Toggle/ToggleGroup/ToggleGroup.stories.tsx @@ -0,0 +1,35 @@ +import { Meta, StoryObj } from '@storybook/react'; +import { Icon } from '../../Icon'; +import { ToggleGroup } from './ToggleGroup'; +import { ToggleGroupItem } from './ToggleGroup.styles'; + +export default { + title: 'Components / Toggle / ToggleGroup', + component: ToggleGroup, + tags: ['autodocs'], + args: { + disabled: false, + }, + render: (props) => ( + + + + + + + + + + + + ), +} as Meta; + +type Story = StoryObj; +export const Default: Story = {}; + +export const Disabled: Story = { + args: { + disabled: true, + }, +}; diff --git a/ui/lib/Toggle/ToggleGroup/ToggleGroup.styles.ts b/ui/lib/Toggle/ToggleGroup/ToggleGroup.styles.ts new file mode 100644 index 0000000000..599716f67c --- /dev/null +++ b/ui/lib/Toggle/ToggleGroup/ToggleGroup.styles.ts @@ -0,0 +1,15 @@ +'use client'; + +import * as RadixToggleGroup from '@radix-ui/react-toggle-group'; +import styled from 'styled-components'; +import { baseButtonStyle } from '../styles'; + +export const BaseToggleGroupRoot = styled(RadixToggleGroup.Root)` + display: inline-flex; + flex-flow: row nowrap; +`; + +export const ToggleGroupItem = styled(RadixToggleGroup.Item)` + ${baseButtonStyle} +`; +ToggleGroupItem.displayName = 'ToggleGroupItem'; diff --git a/ui/lib/Toggle/ToggleGroup/ToggleGroup.tsx b/ui/lib/Toggle/ToggleGroup/ToggleGroup.tsx new file mode 100644 index 0000000000..d8e8fee72b --- /dev/null +++ b/ui/lib/Toggle/ToggleGroup/ToggleGroup.tsx @@ -0,0 +1,24 @@ +'use client'; +import * as RadixToggleGroup from '@radix-ui/react-toggle-group'; +import { PropsWithChildren } from 'react'; +import { BaseToggleGroupRoot } from './ToggleGroup.styles'; + +type ToggleGroupProps = PropsWithChildren<{ + className?: string; +}> & + Omit; + +/** + * Toggle group component + * + * [Figma spec](https://www.figma.com/file/DBMDh1LNNvp9H99N9lZgJ7/PeerDB?type=design&node-id=1-1221&mode=design&t=fsqGSfRrNseMunYH-4) + * + * Based on the [Radix Toggle Group](https://www.radix-ui.com/primitives/docs/components/toggle-group) component + */ +export function ToggleGroup({ children, ...rootProps }: ToggleGroupProps) { + return ( + + {children} + + ); +} diff --git a/ui/lib/Toggle/ToggleGroup/index.ts b/ui/lib/Toggle/ToggleGroup/index.ts new file mode 100644 index 0000000000..5885b8ffc2 --- /dev/null +++ b/ui/lib/Toggle/ToggleGroup/index.ts @@ -0,0 +1,2 @@ +export * from './ToggleGroup'; +export { ToggleGroupItem } from './ToggleGroup.styles'; diff --git a/ui/lib/Toggle/index.ts b/ui/lib/Toggle/index.ts new file mode 100644 index 0000000000..6f4aaf01f5 --- /dev/null +++ b/ui/lib/Toggle/index.ts @@ -0,0 +1,2 @@ +export * from './ToggleButton'; +export * from './ToggleGroup'; diff --git a/ui/lib/Toggle/styles.ts b/ui/lib/Toggle/styles.ts new file mode 100644 index 0000000000..f0ad08eb60 --- /dev/null +++ b/ui/lib/Toggle/styles.ts @@ -0,0 +1,38 @@ +import { css } from 'styled-components'; + +export const baseButtonStyle = css` + all: unset; + display: inline-flex; + align-items: center; + margin: ${({ theme }) => theme.spacing.xxSmall}; + border-radius: ${({ theme }) => theme.radius.small}; + padding: ${({ theme }) => `${theme.radius.xxSmall} ${theme.radius.small}`}; + color: ${({ theme }) => theme.colors.base.text.lowContrast}; + + ${({ theme }) => theme.text.medium.body} + + &:hover:not([data-disabled]) { + background-color: ${({ theme }) => theme.colors.base.surface.hovered}; + color: ${({ theme }) => theme.colors.base.text.highContrast}; + } + + &:focus { + outline: 0; + } + + &:focus-visible { + outline: 2px solid ${({ theme }) => theme.colors.accent.border.normal}; + outline-offset: -2px; + } + + &[data-state='on'], + &[data-state='on']:hover:not([data-disabled]) { + background-color: ${({ theme }) => theme.colors.base.surface.selected}; + color: ${({ theme }) => theme.colors.base.text.highContrast}; + } + + &[data-disabled] { + background-color: transparent; + opacity: 0.3; + } +`; diff --git a/ui/lib/Tooltip/Tooltip.stories.tsx b/ui/lib/Tooltip/Tooltip.stories.tsx new file mode 100644 index 0000000000..98a4826e35 --- /dev/null +++ b/ui/lib/Tooltip/Tooltip.stories.tsx @@ -0,0 +1,18 @@ +import { Meta, StoryObj } from '@storybook/react'; +import { Label } from '../Label'; +import { Tooltip } from './Tooltip'; + +export default { + title: 'Components / Tooltip', + component: Tooltip, + tags: ['autodocs'], +} satisfies Meta; + +type Story = StoryObj; +export const Default: Story = { + render: () => ( + + + + ), +}; diff --git a/ui/lib/Tooltip/Tooltip.styles.ts b/ui/lib/Tooltip/Tooltip.styles.ts new file mode 100644 index 0000000000..60fad54931 --- /dev/null +++ b/ui/lib/Tooltip/Tooltip.styles.ts @@ -0,0 +1,25 @@ +import * as RadixTooltip from '@radix-ui/react-tooltip'; +import styled, { css } from 'styled-components'; + +export const TooltipContent = styled(RadixTooltip.Content)` + display: inline-flex; + padding: ${({ theme }) => `${theme.spacing.xSmall} ${theme.spacing.medium}`}; + flex-direction: column; + justify-content: center; + align-items: center; + border-radius: ${({ theme }) => theme.radius.xxSmall}; + background: ${({ theme }) => theme.colors.special.inverted.black}; + + ${({ theme }) => css(theme.dropShadow.large)} + user-select: none; + + color: ${({ theme }) => theme.colors.special.inverted.white}; + ${({ theme }) => css(theme.text.regular.footnote)}; + + width: var(--radix-tooltip-trigger-width); + max-height: var(--radix-tooltip-content-available-height); +`; + +export const TriggerContent = styled.div` + display: inline-block; +`; diff --git a/ui/lib/Tooltip/Tooltip.tsx b/ui/lib/Tooltip/Tooltip.tsx new file mode 100644 index 0000000000..95473df2af --- /dev/null +++ b/ui/lib/Tooltip/Tooltip.tsx @@ -0,0 +1,37 @@ +'use client'; +import * as RadixTooltip from '@radix-ui/react-tooltip'; +import { PropsWithChildren } from 'react'; +import { TooltipContent, TriggerContent } from './Tooltip.styles'; + +type TooltipProps = PropsWithChildren<{ + content: string; +}> & + RadixTooltip.TooltipProps & + RadixTooltip.TooltipContentProps; + +export function Tooltip({ + content, + children, + open, + defaultOpen, + onOpenChange, + ...contentProps +}: TooltipProps) { + return ( + + + + + {children} + + + {content} + + + ); +} diff --git a/ui/lib/Tooltip/index.ts b/ui/lib/Tooltip/index.ts new file mode 100644 index 0000000000..7594a8f06c --- /dev/null +++ b/ui/lib/Tooltip/index.ts @@ -0,0 +1 @@ +export * from './Tooltip'; diff --git a/ui/lib/types.ts b/ui/lib/types.ts new file mode 100644 index 0000000000..3bea8fb651 --- /dev/null +++ b/ui/lib/types.ts @@ -0,0 +1,18 @@ +import type { ReactElement } from 'react'; + +export type RenderObject = ReactElement | string; + +type PropsOf< + E extends keyof JSX.IntrinsicElements | React.JSXElementConstructor, +> = JSX.LibraryManagedAttributes>; + +export interface AsProps { + as?: E; + className?: string; +} + +export type AsComponentProps = AsProps & + Omit, keyof AsProps>; + +export type PolymorphicComponentProps = P & + AsComponentProps; diff --git a/ui/lib/utils/isDefined.ts b/ui/lib/utils/isDefined.ts new file mode 100644 index 0000000000..0a8b019627 --- /dev/null +++ b/ui/lib/utils/isDefined.ts @@ -0,0 +1,3 @@ +export const isDefined = ( + value: TValue | undefined | null +): value is TValue => value !== null && value !== undefined; diff --git a/ui/lib/utils/renderObjectWith.ts b/ui/lib/utils/renderObjectWith.ts new file mode 100644 index 0000000000..a4b1e1860a --- /dev/null +++ b/ui/lib/utils/renderObjectWith.ts @@ -0,0 +1,33 @@ +import cn from 'classnames'; +import { Attributes, cloneElement, isValidElement } from 'react'; +import { RenderObject } from '../types'; +import { isDefined } from './isDefined'; + +export const renderObjectWith = < + TProps extends Partial & Attributes & { className?: string }, +>( + renderObject?: RenderObject, + injectedProps?: TProps +) => { + if (!isDefined(renderObject)) return null; + + const element = renderObject; + if (!isValidElement(element)) return null; + + const elementClassName = + 'className' in element.props + ? typeof element.props.className === 'string' + ? element.props.className + : undefined + : undefined; + const injectedClassName = + injectedProps && 'className' in injectedProps + ? typeof injectedProps.className === 'string' + ? injectedProps.className + : undefined + : undefined; + + const className = cn(elementClassName, injectedClassName); + + return cloneElement(element, { ...injectedProps, className } as TProps); +}; diff --git a/ui/next.config.js b/ui/next.config.js new file mode 100644 index 0000000000..a42a10c727 --- /dev/null +++ b/ui/next.config.js @@ -0,0 +1,11 @@ +/** @type {import('next').NextConfig} */ +const nextConfig = { + compiler: { + styledComponents: true, + }, + reactStrictMode: true, + swcMinify: true, + output: 'standalone', +}; + +module.exports = nextConfig; diff --git a/ui/package.json b/ui/package.json new file mode 100644 index 0000000000..9b747bdc89 --- /dev/null +++ b/ui/package.json @@ -0,0 +1,70 @@ +{ + "name": "peerdb-cloud-template", + "version": "0.1.0", + "private": true, + "scripts": { + "dev": "next dev", + "build": "next build", + "start": "next start", + "lint": "next lint", + "format": "prettier --write .", + "storybook": "storybook dev -p 6006", + "storybook:clean": "rm -rf ./storybook-static", + "storybook:build": "storybook build", + "storybook:deploy": "yarn storybook:build && touch storybook-static/.nojekyll && rm storybook-static/.gitignore && yarn gh-pages -t -d storybook-static && yarn storybook:clean" + }, + "dependencies": { + "@grpc/grpc-js": "^1.9.2", + "@radix-ui/react-checkbox": "^1.0.4", + "@radix-ui/react-dialog": "^1.0.4", + "@radix-ui/react-form": "^0.0.3", + "@radix-ui/react-progress": "^1.0.3", + "@radix-ui/react-radio-group": "^1.1.3", + "@radix-ui/react-select": "^1.2.2", + "@radix-ui/react-switch": "^1.0.3", + "@radix-ui/react-toast": "^1.1.4", + "@radix-ui/react-toggle": "^1.0.3", + "@radix-ui/react-toggle-group": "^1.0.4", + "@radix-ui/react-tooltip": "^1.0.6", + "@tremor/react": "^3.6.6", + "@types/node": "20.5.0", + "@types/react": "18.2.20", + "@types/react-dom": "18.2.7", + "classnames": "^2.3.2", + "long": "^5.2.3", + "material-symbols": "0.11.0", + "next": "13.4.16", + "prop-types": "^15.8.1", + "protobufjs": "^7.2.5", + "react": "18.2.0", + "react-dom": "18.2.0", + "styled-components": "^6.0.7" + }, + "devDependencies": { + "@storybook/addon-essentials": "^7.3.0", + "@storybook/addon-interactions": "^7.3.0", + "@storybook/addon-links": "^7.3.0", + "@storybook/addon-styling": "^1.3.7", + "@storybook/blocks": "^7.3.0", + "@storybook/nextjs": "^7.3.0", + "@storybook/react": "^7.3.0", + "@storybook/testing-library": "^0.2.0", + "autoprefixer": "^10.4.15", + "copy-webpack-plugin": "^11.0.0", + "eslint": "^8.48.0", + "eslint-config-next": "13.4.19", + "eslint-config-prettier": "^9.0.0", + "eslint-plugin-storybook": "^0.6.13", + "gh-pages": "^6.0.0", + "less": "^4.1.2", + "postcss": "^8.4.28", + "prettier": "^3.0.1", + "prettier-plugin-organize-imports": "^3.2.3", + "storybook": "^7.3.0", + "string-width": "^6.1.0", + "tailwindcss": "^3.3.3", + "typescript": "^5.2.2", + "webpack": "^5.76.0" + }, + "packageManager": "yarn@3.6.3" +} diff --git a/ui/postcss.config.js b/ui/postcss.config.js new file mode 100644 index 0000000000..12a703d900 --- /dev/null +++ b/ui/postcss.config.js @@ -0,0 +1,6 @@ +module.exports = { + plugins: { + tailwindcss: {}, + autoprefixer: {}, + }, +}; diff --git a/ui/public/images/peerdb-combinedMark.svg b/ui/public/images/peerdb-combinedMark.svg new file mode 100644 index 0000000000..48602828a2 --- /dev/null +++ b/ui/public/images/peerdb-combinedMark.svg @@ -0,0 +1,18 @@ + + + + + + + + + + + + + + + + + + diff --git a/ui/rpc/promisify.ts b/ui/rpc/promisify.ts new file mode 100644 index 0000000000..b561f7c011 --- /dev/null +++ b/ui/rpc/promisify.ts @@ -0,0 +1,53 @@ +import type { + CallOptions, + Client, + ClientUnaryCall, + Metadata, + ServiceError, +} from '@grpc/grpc-js'; + +type OriginalCall = ( + request: T, + metadata: Metadata, + options: Partial, + callback: (err: ServiceError | null, res?: U) => void +) => ClientUnaryCall; + +type PromisifiedCall = ( + request: T, + metadata?: Metadata, + options?: Partial +) => Promise; + +export type PromisifiedClient = { $: C } & { + [prop in Exclude]: C[prop] extends OriginalCall< + infer T, + infer U + > + ? PromisifiedCall + : never; +}; + +export function promisifyClient(client: C) { + return new Proxy(client, { + get: (target, descriptor) => { + const key = descriptor as keyof PromisifiedClient; + + if (key === '$') return target; + + const func = target[key]; + if (typeof func === 'function') + return (...args: unknown[]) => + new Promise((resolve, reject) => + func.call( + target, + ...[ + ...args, + (err: unknown, res: unknown) => + err ? reject(err) : resolve(res), + ] + ) + ); + }, + }) as unknown as PromisifiedClient; +} diff --git a/ui/rpc/rpc.ts b/ui/rpc/rpc.ts new file mode 100644 index 0000000000..1c344ae81a --- /dev/null +++ b/ui/rpc/rpc.ts @@ -0,0 +1,12 @@ +import 'server-only'; + +import { FlowServiceClient } from '@/grpc_generated/route'; +import { credentials } from '@grpc/grpc-js'; +import { promisifyClient } from './promisify'; + +export function GetFlowServiceClient(address: string) { + console.log(`Connecting to Flow server at ${address}`); + return promisifyClient( + new FlowServiceClient(address, credentials.createInsecure()) + ); +} diff --git a/ui/tailwind.config.ts b/ui/tailwind.config.ts new file mode 100644 index 0000000000..08d37404ce --- /dev/null +++ b/ui/tailwind.config.ts @@ -0,0 +1,105 @@ +import type { Config } from 'tailwindcss'; +import { appThemeColors } from './lib/AppTheme/appThemeColors'; +import { appThemeDropShadow } from './lib/AppTheme/appThemeDropShadow'; +import { appThemeRadius } from './lib/AppTheme/appThemeRadius'; + +module.exports = { + darkMode: 'class', + content: [ + './app/**/*.{js,ts,jsx,tsx,mdx}', + './components/**/*.{js,ts,jsx,tsx,mdx}', + './lib/**/*.{js,ts,jsx,tsx,mdx}', + './node_modules/@tremor/**/*.{js,ts,jsx,tsx}', + ], + theme: { + transparent: 'transparent', + current: 'currentColor', + extend: { + colors: { + blue: { + 500: appThemeColors.accent.fill.normal, + }, + gray: { + 500: appThemeColors.base.border.subtle, + }, + tremor: { + brand: { + faint: '#eff6ff', // blue-50 + muted: '#bfdbfe', // blue-200 + subtle: '#60a5fa', // blue-400 + DEFAULT: appThemeColors.accent.fill.normal, // blue-500 + emphasis: '#1d4ed8', // blue-700 + inverted: '#ffffff', // white + }, + background: { + muted: '#f9fafb', // gray-50 + subtle: '#f3f4f6', // gray-100 + DEFAULT: appThemeColors.base.background.normal, // white + emphasis: '#374151', // gray-700 + }, + border: { + DEFAULT: appThemeColors.base.border.normal, // gray-200 + }, + ring: { + DEFAULT: '#e5e7eb', // gray-200 + }, + content: { + subtle: '#9ca3af', // gray-400 + DEFAULT: appThemeColors.base.text.lowContrast, // gray-500 + emphasis: appThemeColors.base.text.highContrast, // gray-700 + strong: '#111827', // gray-900 + inverted: '#ffffff', // white + }, + }, + }, + boxShadow: { + // light + 'tremor-input': '0 1px 2px 0 rgb(0 0 0 / 0.05)', + 'tremor-card': + '0 1px 3px 0 rgb(0 0 0 / 0.1), 0 1px 2px -1px rgb(0 0 0 / 0.1)', + 'tremor-dropdown': appThemeDropShadow.large.boxShadow, + }, + borderRadius: { + 'tremor-small': '0.375rem', + 'tremor-default': appThemeRadius.medium, + 'tremor-full': '9999px', + }, + fontSize: { + 'tremor-label': ['12px', { lineHeight: '16px', fontWeight: 500 }], + 'tremor-default': ['16px', { lineHeight: '24px' }], + 'tremor-title': ['1.125rem', { lineHeight: '1.75rem' }], + 'tremor-metric': ['1.875rem', { lineHeight: '2.25rem' }], + }, + }, + }, + safelist: [ + { + pattern: + /^(bg-(?:slate|gray|zinc|neutral|stone|red|orange|amber|yellow|lime|green|emerald|teal|cyan|sky|blue|indigo|violet|purple|fuchsia|pink|rose)-(?:50|100|200|300|400|500|600|700|800|900|950))$/, + variants: ['hover', 'ui-selected'], + }, + { + pattern: + /^(text-(?:slate|gray|zinc|neutral|stone|red|orange|amber|yellow|lime|green|emerald|teal|cyan|sky|blue|indigo|violet|purple|fuchsia|pink|rose)-(?:50|100|200|300|400|500|600|700|800|900|950))$/, + variants: ['hover', 'ui-selected'], + }, + { + pattern: + /^(border-(?:slate|gray|zinc|neutral|stone|red|orange|amber|yellow|lime|green|emerald|teal|cyan|sky|blue|indigo|violet|purple|fuchsia|pink|rose)-(?:50|100|200|300|400|500|600|700|800|900|950))$/, + variants: ['hover', 'ui-selected'], + }, + { + pattern: + /^(ring-(?:slate|gray|zinc|neutral|stone|red|orange|amber|yellow|lime|green|emerald|teal|cyan|sky|blue|indigo|violet|purple|fuchsia|pink|rose)-(?:50|100|200|300|400|500|600|700|800|900|950))$/, + }, + { + pattern: + /^(stroke-(?:slate|gray|zinc|neutral|stone|red|orange|amber|yellow|lime|green|emerald|teal|cyan|sky|blue|indigo|violet|purple|fuchsia|pink|rose)-(?:50|100|200|300|400|500|600|700|800|900|950))$/, + }, + { + pattern: + /^(fill-(?:slate|gray|zinc|neutral|stone|red|orange|amber|yellow|lime|green|emerald|teal|cyan|sky|blue|indigo|violet|purple|fuchsia|pink|rose)-(?:50|100|200|300|400|500|600|700|800|900|950))$/, + }, + ], + plugins: [require('@headlessui/tailwindcss')], +} satisfies Config; diff --git a/ui/tsconfig.json b/ui/tsconfig.json new file mode 100644 index 0000000000..23ba4fd549 --- /dev/null +++ b/ui/tsconfig.json @@ -0,0 +1,28 @@ +{ + "compilerOptions": { + "target": "es5", + "lib": ["dom", "dom.iterable", "esnext"], + "allowJs": true, + "skipLibCheck": true, + "strict": true, + "forceConsistentCasingInFileNames": true, + "noEmit": true, + "esModuleInterop": true, + "module": "esnext", + "moduleResolution": "bundler", + "resolveJsonModule": true, + "isolatedModules": true, + "jsx": "preserve", + "incremental": true, + "plugins": [ + { + "name": "next" + } + ], + "paths": { + "@/*": ["./*"] + } + }, + "include": ["next-env.d.ts", "**/*.ts", "**/*.tsx", ".next/types/**/*.ts"], + "exclude": ["node_modules"] +} diff --git a/ui/yarn.lock b/ui/yarn.lock new file mode 100644 index 0000000000..3f01e702bb --- /dev/null +++ b/ui/yarn.lock @@ -0,0 +1,10768 @@ +# THIS IS AN AUTOGENERATED FILE. DO NOT EDIT THIS FILE DIRECTLY. +# yarn lockfile v1 + + +"@aashutoshrathi/word-wrap@^1.2.3": + version "1.2.6" + resolved "https://registry.yarnpkg.com/@aashutoshrathi/word-wrap/-/word-wrap-1.2.6.tgz#bd9154aec9983f77b3a034ecaa015c2e4201f6cf" + integrity sha512-1Yjs2SvM8TflER/OD3cOjhWWOZb58A2t7wpE2S9XfBYTiIl+XFhQG2bjy4Pu1I+EAlCNUzRDYDdFwFYUKvXcIA== + +"@alloc/quick-lru@^5.2.0": + version "5.2.0" + resolved "https://registry.yarnpkg.com/@alloc/quick-lru/-/quick-lru-5.2.0.tgz#7bf68b20c0a350f936915fcae06f58e32007ce30" + integrity sha512-UrcABB+4bUrFABwbluTIBErXwvbsU/V7TZWfmbgJfbkwiBuziS9gxdODUyuiecfdGQ85jglMW6juS3+z5TsKLw== + +"@ampproject/remapping@^2.2.0": + version "2.2.1" + resolved "https://registry.yarnpkg.com/@ampproject/remapping/-/remapping-2.2.1.tgz#99e8e11851128b8702cd57c33684f1d0f260b630" + integrity sha512-lFMjJTrFL3j7L9yBxwYfCq2k6qqwHyzuUl/XBnif78PWTJYyL/dfowQHWE3sp6U6ZzqWiiIZnpTMO96zhkjwtg== + dependencies: + "@jridgewell/gen-mapping" "^0.3.0" + "@jridgewell/trace-mapping" "^0.3.9" + +"@aw-web-design/x-default-browser@1.4.126": + version "1.4.126" + resolved "https://registry.yarnpkg.com/@aw-web-design/x-default-browser/-/x-default-browser-1.4.126.tgz#43e4bd8f0314ed907a8718d7e862a203af79bc16" + integrity sha512-Xk1sIhyNC/esHGGVjL/niHLowM0csl/kFO5uawBy4IrWwy0o1G8LGt3jP6nmWGz+USxeeqbihAmp/oVZju6wug== + dependencies: + default-browser-id "3.0.0" + +"@babel/cli@^7.21.0": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/cli/-/cli-7.22.15.tgz#22ed82d76745a43caa60a89917bedb7c9b5bd145" + integrity sha512-prtg5f6zCERIaECeTZzd2fMtVjlfjhUcO+fBLQ6DXXdq5FljN+excVitJ2nogsusdf31LeqkjAfXZ7Xq+HmN8g== + dependencies: + "@jridgewell/trace-mapping" "^0.3.17" + commander "^4.0.1" + convert-source-map "^1.1.0" + fs-readdir-recursive "^1.1.0" + glob "^7.2.0" + make-dir "^2.1.0" + slash "^2.0.0" + optionalDependencies: + "@nicolo-ribaudo/chokidar-2" "2.1.8-no-fsevents.3" + chokidar "^3.4.0" + +"@babel/code-frame@^7.0.0", "@babel/code-frame@^7.10.4", "@babel/code-frame@^7.16.7", "@babel/code-frame@^7.22.13": + version "7.22.13" + resolved "https://registry.yarnpkg.com/@babel/code-frame/-/code-frame-7.22.13.tgz#e3c1c099402598483b7a8c46a721d1038803755e" + integrity sha512-XktuhWlJ5g+3TJXc5upd9Ks1HutSArik6jf2eAjYFyIOf4ej3RN+184cZbzDvbPnuTJIUhPKKJE3cIsYTiAT3w== + dependencies: + "@babel/highlight" "^7.22.13" + chalk "^2.4.2" + +"@babel/compat-data@^7.20.5", "@babel/compat-data@^7.22.6", "@babel/compat-data@^7.22.9": + version "7.22.9" + resolved "https://registry.yarnpkg.com/@babel/compat-data/-/compat-data-7.22.9.tgz#71cdb00a1ce3a329ce4cbec3a44f9fef35669730" + integrity sha512-5UamI7xkUcJ3i9qVDS+KFDEK8/7oJ55/sJMB1Ge7IEapr7KfdfV/HErR+koZwOfd+SgtFKOKRhRakdg++DcJpQ== + +"@babel/core@^7.11.6", "@babel/core@^7.12.3", "@babel/core@^7.13.16", "@babel/core@^7.21.0", "@babel/core@^7.22.0", "@babel/core@^7.22.9", "@babel/core@^7.7.5": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/core/-/core-7.22.15.tgz#15d4fd03f478a459015a4b94cfbb3bd42c48d2f4" + integrity sha512-PtZqMmgRrvj8ruoEOIwVA3yoF91O+Hgw9o7DAUTNBA6Mo2jpu31clx9a7Nz/9JznqetTR6zwfC4L3LAjKQXUwA== + dependencies: + "@ampproject/remapping" "^2.2.0" + "@babel/code-frame" "^7.22.13" + "@babel/generator" "^7.22.15" + "@babel/helper-compilation-targets" "^7.22.15" + "@babel/helper-module-transforms" "^7.22.15" + "@babel/helpers" "^7.22.15" + "@babel/parser" "^7.22.15" + "@babel/template" "^7.22.15" + "@babel/traverse" "^7.22.15" + "@babel/types" "^7.22.15" + convert-source-map "^1.7.0" + debug "^4.1.0" + gensync "^1.0.0-beta.2" + json5 "^2.2.3" + semver "^6.3.1" + +"@babel/generator@^7.12.11", "@babel/generator@^7.22.15", "@babel/generator@^7.22.9": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/generator/-/generator-7.22.15.tgz#1564189c7ec94cb8f77b5e8a90c4d200d21b2339" + integrity sha512-Zu9oWARBqeVOW0dZOjXc3JObrzuqothQ3y/n1kUtrjCoCPLkXUwMvOo/F/TCfoHMbWIFlWwpZtkZVb9ga4U2pA== + dependencies: + "@babel/types" "^7.22.15" + "@jridgewell/gen-mapping" "^0.3.2" + "@jridgewell/trace-mapping" "^0.3.17" + jsesc "^2.5.1" + +"@babel/helper-annotate-as-pure@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/helper-annotate-as-pure/-/helper-annotate-as-pure-7.22.5.tgz#e7f06737b197d580a01edf75d97e2c8be99d3882" + integrity sha512-LvBTxu8bQSQkcyKOU+a1btnNFQ1dMAd0R6PyW3arXes06F6QLWLIrd681bxRPIXlrMGR3XYnW9JyML7dP3qgxg== + dependencies: + "@babel/types" "^7.22.5" + +"@babel/helper-builder-binary-assignment-operator-visitor@^7.22.5": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/helper-builder-binary-assignment-operator-visitor/-/helper-builder-binary-assignment-operator-visitor-7.22.15.tgz#5426b109cf3ad47b91120f8328d8ab1be8b0b956" + integrity sha512-QkBXwGgaoC2GtGZRoma6kv7Szfv06khvhFav67ZExau2RaXzy8MpHSMO2PNoP2XtmQphJQRHFfg77Bq731Yizw== + dependencies: + "@babel/types" "^7.22.15" + +"@babel/helper-compilation-targets@^7.20.7", "@babel/helper-compilation-targets@^7.22.15", "@babel/helper-compilation-targets@^7.22.5", "@babel/helper-compilation-targets@^7.22.6": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/helper-compilation-targets/-/helper-compilation-targets-7.22.15.tgz#0698fc44551a26cf29f18d4662d5bf545a6cfc52" + integrity sha512-y6EEzULok0Qvz8yyLkCvVX+02ic+By2UdOhylwUOvOn9dvYc9mKICJuuU1n1XBI02YWsNsnrY1kc6DVbjcXbtw== + dependencies: + "@babel/compat-data" "^7.22.9" + "@babel/helper-validator-option" "^7.22.15" + browserslist "^4.21.9" + lru-cache "^5.1.1" + semver "^6.3.1" + +"@babel/helper-create-class-features-plugin@^7.18.6", "@babel/helper-create-class-features-plugin@^7.22.11", "@babel/helper-create-class-features-plugin@^7.22.15", "@babel/helper-create-class-features-plugin@^7.22.5": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/helper-create-class-features-plugin/-/helper-create-class-features-plugin-7.22.15.tgz#97a61b385e57fe458496fad19f8e63b63c867de4" + integrity sha512-jKkwA59IXcvSaiK2UN45kKwSC9o+KuoXsBDvHvU/7BecYIp8GQ2UwrVvFgJASUT+hBnwJx6MhvMCuMzwZZ7jlg== + dependencies: + "@babel/helper-annotate-as-pure" "^7.22.5" + "@babel/helper-environment-visitor" "^7.22.5" + "@babel/helper-function-name" "^7.22.5" + "@babel/helper-member-expression-to-functions" "^7.22.15" + "@babel/helper-optimise-call-expression" "^7.22.5" + "@babel/helper-replace-supers" "^7.22.9" + "@babel/helper-skip-transparent-expression-wrappers" "^7.22.5" + "@babel/helper-split-export-declaration" "^7.22.6" + semver "^6.3.1" + +"@babel/helper-create-regexp-features-plugin@^7.18.6", "@babel/helper-create-regexp-features-plugin@^7.22.5": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/helper-create-regexp-features-plugin/-/helper-create-regexp-features-plugin-7.22.15.tgz#5ee90093914ea09639b01c711db0d6775e558be1" + integrity sha512-29FkPLFjn4TPEa3RE7GpW+qbE8tlsu3jntNYNfcGsc49LphF1PQIiD+vMZ1z1xVOKt+93khA9tc2JBs3kBjA7w== + dependencies: + "@babel/helper-annotate-as-pure" "^7.22.5" + regexpu-core "^5.3.1" + semver "^6.3.1" + +"@babel/helper-define-polyfill-provider@^0.4.2": + version "0.4.2" + resolved "https://registry.yarnpkg.com/@babel/helper-define-polyfill-provider/-/helper-define-polyfill-provider-0.4.2.tgz#82c825cadeeeee7aad237618ebbe8fa1710015d7" + integrity sha512-k0qnnOqHn5dK9pZpfD5XXZ9SojAITdCKRn2Lp6rnDGzIbaP0rHyMPk/4wsSxVBVz4RfN0q6VpXWP2pDGIoQ7hw== + dependencies: + "@babel/helper-compilation-targets" "^7.22.6" + "@babel/helper-plugin-utils" "^7.22.5" + debug "^4.1.1" + lodash.debounce "^4.0.8" + resolve "^1.14.2" + +"@babel/helper-environment-visitor@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/helper-environment-visitor/-/helper-environment-visitor-7.22.5.tgz#f06dd41b7c1f44e1f8da6c4055b41ab3a09a7e98" + integrity sha512-XGmhECfVA/5sAt+H+xpSg0mfrHq6FzNr9Oxh7PSEBBRUb/mL7Kz3NICXb194rCqAEdxkhPT1a88teizAFyvk8Q== + +"@babel/helper-function-name@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/helper-function-name/-/helper-function-name-7.22.5.tgz#ede300828905bb15e582c037162f99d5183af1be" + integrity sha512-wtHSq6jMRE3uF2otvfuD3DIvVhOsSNshQl0Qrd7qC9oQJzHvOL4qQXlQn2916+CXGywIjpGuIkoyZRRxHPiNQQ== + dependencies: + "@babel/template" "^7.22.5" + "@babel/types" "^7.22.5" + +"@babel/helper-hoist-variables@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/helper-hoist-variables/-/helper-hoist-variables-7.22.5.tgz#c01a007dac05c085914e8fb652b339db50d823bb" + integrity sha512-wGjk9QZVzvknA6yKIUURb8zY3grXCcOZt+/7Wcy8O2uctxhplmUPkOdlgoNhmdVee2c92JXbf1xpMtVNbfoxRw== + dependencies: + "@babel/types" "^7.22.5" + +"@babel/helper-member-expression-to-functions@^7.22.15", "@babel/helper-member-expression-to-functions@^7.22.5": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/helper-member-expression-to-functions/-/helper-member-expression-to-functions-7.22.15.tgz#b95a144896f6d491ca7863576f820f3628818621" + integrity sha512-qLNsZbgrNh0fDQBCPocSL8guki1hcPvltGDv/NxvUoABwFq7GkKSu1nRXeJkVZc+wJvne2E0RKQz+2SQrz6eAA== + dependencies: + "@babel/types" "^7.22.15" + +"@babel/helper-module-imports@^7.18.6", "@babel/helper-module-imports@^7.22.15", "@babel/helper-module-imports@^7.22.5": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/helper-module-imports/-/helper-module-imports-7.22.15.tgz#16146307acdc40cc00c3b2c647713076464bdbf0" + integrity sha512-0pYVBnDKZO2fnSPCrgM/6WMc7eS20Fbok+0r88fp+YtWVLZrp4CkafFGIp+W0VKw4a22sgebPT99y+FDNMdP4w== + dependencies: + "@babel/types" "^7.22.15" + +"@babel/helper-module-transforms@^7.22.15", "@babel/helper-module-transforms@^7.22.5", "@babel/helper-module-transforms@^7.22.9": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/helper-module-transforms/-/helper-module-transforms-7.22.15.tgz#40ad2f6950f143900e9c1c72363c0b431a606082" + integrity sha512-l1UiX4UyHSFsYt17iQ3Se5pQQZZHa22zyIXURmvkmLCD4t/aU+dvNWHatKac/D9Vm9UES7nvIqHs4jZqKviUmQ== + dependencies: + "@babel/helper-environment-visitor" "^7.22.5" + "@babel/helper-module-imports" "^7.22.15" + "@babel/helper-simple-access" "^7.22.5" + "@babel/helper-split-export-declaration" "^7.22.6" + "@babel/helper-validator-identifier" "^7.22.15" + +"@babel/helper-optimise-call-expression@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/helper-optimise-call-expression/-/helper-optimise-call-expression-7.22.5.tgz#f21531a9ccbff644fdd156b4077c16ff0c3f609e" + integrity sha512-HBwaojN0xFRx4yIvpwGqxiV2tUfl7401jlok564NgB9EHS1y6QT17FmKWm4ztqjeVdXLuC4fSvHc5ePpQjoTbw== + dependencies: + "@babel/types" "^7.22.5" + +"@babel/helper-plugin-utils@^7.0.0", "@babel/helper-plugin-utils@^7.10.4", "@babel/helper-plugin-utils@^7.12.13", "@babel/helper-plugin-utils@^7.14.5", "@babel/helper-plugin-utils@^7.18.6", "@babel/helper-plugin-utils@^7.18.9", "@babel/helper-plugin-utils@^7.20.2", "@babel/helper-plugin-utils@^7.22.5", "@babel/helper-plugin-utils@^7.8.0", "@babel/helper-plugin-utils@^7.8.3": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/helper-plugin-utils/-/helper-plugin-utils-7.22.5.tgz#dd7ee3735e8a313b9f7b05a773d892e88e6d7295" + integrity sha512-uLls06UVKgFG9QD4OeFYLEGteMIAa5kpTPcFL28yuCIIzsf6ZyKZMllKVOCZFhiZ5ptnwX4mtKdWCBE/uT4amg== + +"@babel/helper-remap-async-to-generator@^7.22.5", "@babel/helper-remap-async-to-generator@^7.22.9": + version "7.22.9" + resolved "https://registry.yarnpkg.com/@babel/helper-remap-async-to-generator/-/helper-remap-async-to-generator-7.22.9.tgz#53a25b7484e722d7efb9c350c75c032d4628de82" + integrity sha512-8WWC4oR4Px+tr+Fp0X3RHDVfINGpF3ad1HIbrc8A77epiR6eMMc6jsgozkzT2uDiOOdoS9cLIQ+XD2XvI2WSmQ== + dependencies: + "@babel/helper-annotate-as-pure" "^7.22.5" + "@babel/helper-environment-visitor" "^7.22.5" + "@babel/helper-wrap-function" "^7.22.9" + +"@babel/helper-replace-supers@^7.22.5", "@babel/helper-replace-supers@^7.22.9": + version "7.22.9" + resolved "https://registry.yarnpkg.com/@babel/helper-replace-supers/-/helper-replace-supers-7.22.9.tgz#cbdc27d6d8d18cd22c81ae4293765a5d9afd0779" + integrity sha512-LJIKvvpgPOPUThdYqcX6IXRuIcTkcAub0IaDRGCZH0p5GPUp7PhRU9QVgFcDDd51BaPkk77ZjqFwh6DZTAEmGg== + dependencies: + "@babel/helper-environment-visitor" "^7.22.5" + "@babel/helper-member-expression-to-functions" "^7.22.5" + "@babel/helper-optimise-call-expression" "^7.22.5" + +"@babel/helper-simple-access@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/helper-simple-access/-/helper-simple-access-7.22.5.tgz#4938357dc7d782b80ed6dbb03a0fba3d22b1d5de" + integrity sha512-n0H99E/K+Bika3++WNL17POvo4rKWZ7lZEp1Q+fStVbUi8nxPQEBOlTmCOxW/0JsS56SKKQ+ojAe2pHKJHN35w== + dependencies: + "@babel/types" "^7.22.5" + +"@babel/helper-skip-transparent-expression-wrappers@^7.20.0", "@babel/helper-skip-transparent-expression-wrappers@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/helper-skip-transparent-expression-wrappers/-/helper-skip-transparent-expression-wrappers-7.22.5.tgz#007f15240b5751c537c40e77abb4e89eeaaa8847" + integrity sha512-tK14r66JZKiC43p8Ki33yLBVJKlQDFoA8GYN67lWCDCqoL6EMMSuM9b+Iff2jHaM/RRFYl7K+iiru7hbRqNx8Q== + dependencies: + "@babel/types" "^7.22.5" + +"@babel/helper-split-export-declaration@^7.22.6": + version "7.22.6" + resolved "https://registry.yarnpkg.com/@babel/helper-split-export-declaration/-/helper-split-export-declaration-7.22.6.tgz#322c61b7310c0997fe4c323955667f18fcefb91c" + integrity sha512-AsUnxuLhRYsisFiaJwvp1QF+I3KjD5FOxut14q/GzovUe6orHLesW2C7d754kRm53h5gqrz6sFl6sxc4BVtE/g== + dependencies: + "@babel/types" "^7.22.5" + +"@babel/helper-string-parser@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/helper-string-parser/-/helper-string-parser-7.22.5.tgz#533f36457a25814cf1df6488523ad547d784a99f" + integrity sha512-mM4COjgZox8U+JcXQwPijIZLElkgEpO5rsERVDJTc2qfCDfERyob6k5WegS14SX18IIjv+XD+GrqNumY5JRCDw== + +"@babel/helper-validator-identifier@^7.22.15", "@babel/helper-validator-identifier@^7.22.5": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/helper-validator-identifier/-/helper-validator-identifier-7.22.15.tgz#601fa28e4cc06786c18912dca138cec73b882044" + integrity sha512-4E/F9IIEi8WR94324mbDUMo074YTheJmd7eZF5vITTeYchqAi6sYXRLHUVsmkdmY4QjfKTcB2jB7dVP3NaBElQ== + +"@babel/helper-validator-option@^7.22.15": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/helper-validator-option/-/helper-validator-option-7.22.15.tgz#694c30dfa1d09a6534cdfcafbe56789d36aba040" + integrity sha512-bMn7RmyFjY/mdECUbgn9eoSY4vqvacUnS9i9vGAGttgFWesO6B4CYWA7XlpbWgBt71iv/hfbPlynohStqnu5hA== + +"@babel/helper-wrap-function@^7.22.9": + version "7.22.10" + resolved "https://registry.yarnpkg.com/@babel/helper-wrap-function/-/helper-wrap-function-7.22.10.tgz#d845e043880ed0b8c18bd194a12005cb16d2f614" + integrity sha512-OnMhjWjuGYtdoO3FmsEFWvBStBAe2QOgwOLsLNDjN+aaiMD8InJk1/O3HSD8lkqTjCgg5YI34Tz15KNNA3p+nQ== + dependencies: + "@babel/helper-function-name" "^7.22.5" + "@babel/template" "^7.22.5" + "@babel/types" "^7.22.10" + +"@babel/helpers@^7.22.15": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/helpers/-/helpers-7.22.15.tgz#f09c3df31e86e3ea0b7ff7556d85cdebd47ea6f1" + integrity sha512-7pAjK0aSdxOwR+CcYAqgWOGy5dcfvzsTIfFTb2odQqW47MDfv14UaJDY6eng8ylM2EaeKXdxaSWESbkmaQHTmw== + dependencies: + "@babel/template" "^7.22.15" + "@babel/traverse" "^7.22.15" + "@babel/types" "^7.22.15" + +"@babel/highlight@^7.22.13": + version "7.22.13" + resolved "https://registry.yarnpkg.com/@babel/highlight/-/highlight-7.22.13.tgz#9cda839e5d3be9ca9e8c26b6dd69e7548f0cbf16" + integrity sha512-C/BaXcnnvBCmHTpz/VGZ8jgtE2aYlW4hxDhseJAWZb7gqGM/qtCK6iZUb0TyKFf7BOUsBH7Q7fkRsDRhg1XklQ== + dependencies: + "@babel/helper-validator-identifier" "^7.22.5" + chalk "^2.4.2" + js-tokens "^4.0.0" + +"@babel/parser@^7.1.0", "@babel/parser@^7.13.16", "@babel/parser@^7.14.7", "@babel/parser@^7.20.7", "@babel/parser@^7.22.15", "@babel/parser@^7.22.7": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/parser/-/parser-7.22.15.tgz#d34592bfe288a32e741aa0663dbc4829fcd55160" + integrity sha512-RWmQ/sklUN9BvGGpCDgSubhHWfAx24XDTDObup4ffvxaYsptOg2P3KG0j+1eWKLxpkX0j0uHxmpq2Z1SP/VhxA== + +"@babel/plugin-bugfix-safari-id-destructuring-collision-in-function-expression@^7.22.15": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/plugin-bugfix-safari-id-destructuring-collision-in-function-expression/-/plugin-bugfix-safari-id-destructuring-collision-in-function-expression-7.22.15.tgz#02dc8a03f613ed5fdc29fb2f728397c78146c962" + integrity sha512-FB9iYlz7rURmRJyXRKEnalYPPdn87H5no108cyuQQyMwlpJ2SJtpIUBI27kdTin956pz+LPypkPVPUTlxOmrsg== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-bugfix-v8-spread-parameters-in-optional-chaining@^7.22.15": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/plugin-bugfix-v8-spread-parameters-in-optional-chaining/-/plugin-bugfix-v8-spread-parameters-in-optional-chaining-7.22.15.tgz#2aeb91d337d4e1a1e7ce85b76a37f5301781200f" + integrity sha512-Hyph9LseGvAeeXzikV88bczhsrLrIZqDPxO+sSmAunMPaGrBGhfMWzCPYTtiW9t+HzSE2wtV8e5cc5P6r1xMDQ== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + "@babel/helper-skip-transparent-expression-wrappers" "^7.22.5" + "@babel/plugin-transform-optional-chaining" "^7.22.15" + +"@babel/plugin-external-helpers@^7.18.6": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-external-helpers/-/plugin-external-helpers-7.22.5.tgz#92b0705b74756123f289388320e0e12c407fdf9a" + integrity sha512-ngnNEWxmykPk82mH4ajZT0qTztr3Je6hrMuKAslZVM8G1YZTENJSYwrIGtt6KOtznug3exmAtF4so/nPqJuA4A== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-proposal-class-properties@^7.13.0", "@babel/plugin-proposal-class-properties@^7.18.6": + version "7.18.6" + resolved "https://registry.yarnpkg.com/@babel/plugin-proposal-class-properties/-/plugin-proposal-class-properties-7.18.6.tgz#b110f59741895f7ec21a6fff696ec46265c446a3" + integrity sha512-cumfXOF0+nzZrrN8Rf0t7M+tF6sZc7vhQwYQck9q1/5w2OExlD+b4v4RpMJFaV1Z7WcDRgO6FqvxqxGlwo+RHQ== + dependencies: + "@babel/helper-create-class-features-plugin" "^7.18.6" + "@babel/helper-plugin-utils" "^7.18.6" + +"@babel/plugin-proposal-export-namespace-from@^7.18.9": + version "7.18.9" + resolved "https://registry.yarnpkg.com/@babel/plugin-proposal-export-namespace-from/-/plugin-proposal-export-namespace-from-7.18.9.tgz#5f7313ab348cdb19d590145f9247540e94761203" + integrity sha512-k1NtHyOMvlDDFeb9G5PhUXuGj8m/wiwojgQVEhJ/fsVsMCpLyOP4h0uGEjYJKrRI+EVPlb5Jk+Gt9P97lOGwtA== + dependencies: + "@babel/helper-plugin-utils" "^7.18.9" + "@babel/plugin-syntax-export-namespace-from" "^7.8.3" + +"@babel/plugin-proposal-nullish-coalescing-operator@^7.13.8": + version "7.18.6" + resolved "https://registry.yarnpkg.com/@babel/plugin-proposal-nullish-coalescing-operator/-/plugin-proposal-nullish-coalescing-operator-7.18.6.tgz#fdd940a99a740e577d6c753ab6fbb43fdb9467e1" + integrity sha512-wQxQzxYeJqHcfppzBDnm1yAY0jSRkUXR2z8RePZYrKwMKgMlE8+Z6LUno+bd6LvbGh8Gltvy74+9pIYkr+XkKA== + dependencies: + "@babel/helper-plugin-utils" "^7.18.6" + "@babel/plugin-syntax-nullish-coalescing-operator" "^7.8.3" + +"@babel/plugin-proposal-numeric-separator@^7.18.6": + version "7.18.6" + resolved "https://registry.yarnpkg.com/@babel/plugin-proposal-numeric-separator/-/plugin-proposal-numeric-separator-7.18.6.tgz#899b14fbafe87f053d2c5ff05b36029c62e13c75" + integrity sha512-ozlZFogPqoLm8WBr5Z8UckIoE4YQ5KESVcNudyXOR8uqIkliTEgJ3RoketfG6pmzLdeZF0H/wjE9/cCEitBl7Q== + dependencies: + "@babel/helper-plugin-utils" "^7.18.6" + "@babel/plugin-syntax-numeric-separator" "^7.10.4" + +"@babel/plugin-proposal-object-rest-spread@^7.20.7": + version "7.20.7" + resolved "https://registry.yarnpkg.com/@babel/plugin-proposal-object-rest-spread/-/plugin-proposal-object-rest-spread-7.20.7.tgz#aa662940ef425779c75534a5c41e9d936edc390a" + integrity sha512-d2S98yCiLxDVmBmE8UjGcfPvNEUbA1U5q5WxaWFUGRzJSVAZqm5W6MbPct0jxnegUZ0niLeNX+IOzEs7wYg9Dg== + dependencies: + "@babel/compat-data" "^7.20.5" + "@babel/helper-compilation-targets" "^7.20.7" + "@babel/helper-plugin-utils" "^7.20.2" + "@babel/plugin-syntax-object-rest-spread" "^7.8.3" + "@babel/plugin-transform-parameters" "^7.20.7" + +"@babel/plugin-proposal-optional-chaining@^7.13.12": + version "7.21.0" + resolved "https://registry.yarnpkg.com/@babel/plugin-proposal-optional-chaining/-/plugin-proposal-optional-chaining-7.21.0.tgz#886f5c8978deb7d30f678b2e24346b287234d3ea" + integrity sha512-p4zeefM72gpmEe2fkUr/OnOXpWEf8nAgk7ZYVqqfFiyIG7oFfVZcCrU64hWn5xp4tQ9LkV4bTIa5rD0KANpKNA== + dependencies: + "@babel/helper-plugin-utils" "^7.20.2" + "@babel/helper-skip-transparent-expression-wrappers" "^7.20.0" + "@babel/plugin-syntax-optional-chaining" "^7.8.3" + +"@babel/plugin-proposal-private-property-in-object@7.21.0-placeholder-for-preset-env.2": + version "7.21.0-placeholder-for-preset-env.2" + resolved "https://registry.yarnpkg.com/@babel/plugin-proposal-private-property-in-object/-/plugin-proposal-private-property-in-object-7.21.0-placeholder-for-preset-env.2.tgz#7844f9289546efa9febac2de4cfe358a050bd703" + integrity sha512-SOSkfJDddaM7mak6cPEpswyTRnuRltl429hMraQEglW+OkovnCzsiszTmsrlY//qLFjCpQDFRvjdm2wA5pPm9w== + +"@babel/plugin-syntax-async-generators@^7.8.4": + version "7.8.4" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-async-generators/-/plugin-syntax-async-generators-7.8.4.tgz#a983fb1aeb2ec3f6ed042a210f640e90e786fe0d" + integrity sha512-tycmZxkGfZaxhMRbXlPXuVFpdWlXpir2W4AMhSJgRKzk/eDlIXOhb2LHWoLpDF7TEHylV5zNhykX6KAgHJmTNw== + dependencies: + "@babel/helper-plugin-utils" "^7.8.0" + +"@babel/plugin-syntax-bigint@^7.8.3": + version "7.8.3" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-bigint/-/plugin-syntax-bigint-7.8.3.tgz#4c9a6f669f5d0cdf1b90a1671e9a146be5300cea" + integrity sha512-wnTnFlG+YxQm3vDxpGE57Pj0srRU4sHE/mDkt1qv2YJJSeUAec2ma4WLUnUPeKjyrfntVwe/N6dCXpU+zL3Npg== + dependencies: + "@babel/helper-plugin-utils" "^7.8.0" + +"@babel/plugin-syntax-class-properties@^7.12.13": + version "7.12.13" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-class-properties/-/plugin-syntax-class-properties-7.12.13.tgz#b5c987274c4a3a82b89714796931a6b53544ae10" + integrity sha512-fm4idjKla0YahUNgFNLCB0qySdsoPiZP3iQE3rky0mBUtMZ23yDJ9SJdg6dXTSDnulOVqiF3Hgr9nbXvXTQZYA== + dependencies: + "@babel/helper-plugin-utils" "^7.12.13" + +"@babel/plugin-syntax-class-static-block@^7.14.5": + version "7.14.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-class-static-block/-/plugin-syntax-class-static-block-7.14.5.tgz#195df89b146b4b78b3bf897fd7a257c84659d406" + integrity sha512-b+YyPmr6ldyNnM6sqYeMWE+bgJcJpO6yS4QD7ymxgH34GBPNDM/THBh8iunyvKIZztiwLH4CJZ0RxTk9emgpjw== + dependencies: + "@babel/helper-plugin-utils" "^7.14.5" + +"@babel/plugin-syntax-dynamic-import@^7.8.3": + version "7.8.3" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-dynamic-import/-/plugin-syntax-dynamic-import-7.8.3.tgz#62bf98b2da3cd21d626154fc96ee5b3cb68eacb3" + integrity sha512-5gdGbFon+PszYzqs83S3E5mpi7/y/8M9eC90MRTZfduQOYW76ig6SOSPNe41IG5LoP3FGBn2N0RjVDSQiS94kQ== + dependencies: + "@babel/helper-plugin-utils" "^7.8.0" + +"@babel/plugin-syntax-export-namespace-from@^7.8.3": + version "7.8.3" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-export-namespace-from/-/plugin-syntax-export-namespace-from-7.8.3.tgz#028964a9ba80dbc094c915c487ad7c4e7a66465a" + integrity sha512-MXf5laXo6c1IbEbegDmzGPwGNTsHZmEy6QGznu5Sh2UCWvueywb2ee+CCE4zQiZstxU9BMoQO9i6zUFSY0Kj0Q== + dependencies: + "@babel/helper-plugin-utils" "^7.8.3" + +"@babel/plugin-syntax-flow@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-flow/-/plugin-syntax-flow-7.22.5.tgz#163b820b9e7696ce134df3ee716d9c0c98035859" + integrity sha512-9RdCl0i+q0QExayk2nOS7853w08yLucnnPML6EN9S8fgMPVtdLDCdx/cOQ/i44Lb9UeQX9A35yaqBBOMMZxPxQ== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-syntax-import-assertions@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-import-assertions/-/plugin-syntax-import-assertions-7.22.5.tgz#07d252e2aa0bc6125567f742cd58619cb14dce98" + integrity sha512-rdV97N7KqsRzeNGoWUOK6yUsWarLjE5Su/Snk9IYPU9CwkWHs4t+rTGOvffTR8XGkJMTAdLfO0xVnXm8wugIJg== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-syntax-import-attributes@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-import-attributes/-/plugin-syntax-import-attributes-7.22.5.tgz#ab840248d834410b829f569f5262b9e517555ecb" + integrity sha512-KwvoWDeNKPETmozyFE0P2rOLqh39EoQHNjqizrI5B8Vt0ZNS7M56s7dAiAqbYfiAYOuIzIh96z3iR2ktgu3tEg== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-syntax-import-meta@^7.10.4": + version "7.10.4" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-import-meta/-/plugin-syntax-import-meta-7.10.4.tgz#ee601348c370fa334d2207be158777496521fd51" + integrity sha512-Yqfm+XDx0+Prh3VSeEQCPU81yC+JWZ2pDPFSS4ZdpfZhp4MkFMaDC1UqseovEKwSUpnIL7+vK+Clp7bfh0iD7g== + dependencies: + "@babel/helper-plugin-utils" "^7.10.4" + +"@babel/plugin-syntax-json-strings@^7.8.3": + version "7.8.3" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-json-strings/-/plugin-syntax-json-strings-7.8.3.tgz#01ca21b668cd8218c9e640cb6dd88c5412b2c96a" + integrity sha512-lY6kdGpWHvjoe2vk4WrAapEuBR69EMxZl+RoGRhrFGNYVK8mOPAW8VfbT/ZgrFbXlDNiiaxQnAtgVCZ6jv30EA== + dependencies: + "@babel/helper-plugin-utils" "^7.8.0" + +"@babel/plugin-syntax-jsx@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-jsx/-/plugin-syntax-jsx-7.22.5.tgz#a6b68e84fb76e759fc3b93e901876ffabbe1d918" + integrity sha512-gvyP4hZrgrs/wWMaocvxZ44Hw0b3W8Pe+cMxc8V1ULQ07oh8VNbIRaoD1LRZVTvD+0nieDKjfgKg89sD7rrKrg== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-syntax-logical-assignment-operators@^7.10.4": + version "7.10.4" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-logical-assignment-operators/-/plugin-syntax-logical-assignment-operators-7.10.4.tgz#ca91ef46303530448b906652bac2e9fe9941f699" + integrity sha512-d8waShlpFDinQ5MtvGU9xDAOzKH47+FFoney2baFIoMr952hKOLp1HR7VszoZvOsV/4+RRszNY7D17ba0te0ig== + dependencies: + "@babel/helper-plugin-utils" "^7.10.4" + +"@babel/plugin-syntax-nullish-coalescing-operator@^7.8.3": + version "7.8.3" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-nullish-coalescing-operator/-/plugin-syntax-nullish-coalescing-operator-7.8.3.tgz#167ed70368886081f74b5c36c65a88c03b66d1a9" + integrity sha512-aSff4zPII1u2QD7y+F8oDsz19ew4IGEJg9SVW+bqwpwtfFleiQDMdzA/R+UlWDzfnHFCxxleFT0PMIrR36XLNQ== + dependencies: + "@babel/helper-plugin-utils" "^7.8.0" + +"@babel/plugin-syntax-numeric-separator@^7.10.4": + version "7.10.4" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-numeric-separator/-/plugin-syntax-numeric-separator-7.10.4.tgz#b9b070b3e33570cd9fd07ba7fa91c0dd37b9af97" + integrity sha512-9H6YdfkcK/uOnY/K7/aA2xpzaAgkQn37yzWUMRK7OaPOqOpGS1+n0H5hxT9AUw9EsSjPW8SVyMJwYRtWs3X3ug== + dependencies: + "@babel/helper-plugin-utils" "^7.10.4" + +"@babel/plugin-syntax-object-rest-spread@^7.8.3": + version "7.8.3" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-object-rest-spread/-/plugin-syntax-object-rest-spread-7.8.3.tgz#60e225edcbd98a640332a2e72dd3e66f1af55871" + integrity sha512-XoqMijGZb9y3y2XskN+P1wUGiVwWZ5JmoDRwx5+3GmEplNyVM2s2Dg8ILFQm8rWM48orGy5YpI5Bl8U1y7ydlA== + dependencies: + "@babel/helper-plugin-utils" "^7.8.0" + +"@babel/plugin-syntax-optional-catch-binding@^7.8.3": + version "7.8.3" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-optional-catch-binding/-/plugin-syntax-optional-catch-binding-7.8.3.tgz#6111a265bcfb020eb9efd0fdfd7d26402b9ed6c1" + integrity sha512-6VPD0Pc1lpTqw0aKoeRTMiB+kWhAoT24PA+ksWSBrFtl5SIRVpZlwN3NNPQjehA2E/91FV3RjLWoVTglWcSV3Q== + dependencies: + "@babel/helper-plugin-utils" "^7.8.0" + +"@babel/plugin-syntax-optional-chaining@^7.8.3": + version "7.8.3" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-optional-chaining/-/plugin-syntax-optional-chaining-7.8.3.tgz#4f69c2ab95167e0180cd5336613f8c5788f7d48a" + integrity sha512-KoK9ErH1MBlCPxV0VANkXW2/dw4vlbGDrFgz8bmUsBGYkFRcbRwMh6cIJubdPrkxRwuGdtCk0v/wPTKbQgBjkg== + dependencies: + "@babel/helper-plugin-utils" "^7.8.0" + +"@babel/plugin-syntax-private-property-in-object@^7.14.5": + version "7.14.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-private-property-in-object/-/plugin-syntax-private-property-in-object-7.14.5.tgz#0dc6671ec0ea22b6e94a1114f857970cd39de1ad" + integrity sha512-0wVnp9dxJ72ZUJDV27ZfbSj6iHLoytYZmh3rFcxNnvsJF3ktkzLDZPy/mA17HGsaQT3/DQsWYX1f1QGWkCoVUg== + dependencies: + "@babel/helper-plugin-utils" "^7.14.5" + +"@babel/plugin-syntax-top-level-await@^7.14.5": + version "7.14.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-top-level-await/-/plugin-syntax-top-level-await-7.14.5.tgz#c1cfdadc35a646240001f06138247b741c34d94c" + integrity sha512-hx++upLv5U1rgYfwe1xBQUhRmU41NEvpUvrp8jkrSCdvGSnM5/qdRMtylJ6PG5OFkBaHkbTAKTnd3/YyESRHFw== + dependencies: + "@babel/helper-plugin-utils" "^7.14.5" + +"@babel/plugin-syntax-typescript@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-typescript/-/plugin-syntax-typescript-7.22.5.tgz#aac8d383b062c5072c647a31ef990c1d0af90272" + integrity sha512-1mS2o03i7t1c6VzH6fdQ3OA8tcEIxwG18zIPRp+UY1Ihv6W+XZzBCVxExF9upussPXJ0xE9XRHwMoNs1ep/nRQ== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-syntax-unicode-sets-regex@^7.18.6": + version "7.18.6" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-unicode-sets-regex/-/plugin-syntax-unicode-sets-regex-7.18.6.tgz#d49a3b3e6b52e5be6740022317580234a6a47357" + integrity sha512-727YkEAPwSIQTv5im8QHz3upqp92JTWhidIC81Tdx4VJYIte/VndKf1qKrfnnhPLiPghStWfvC/iFaMCQu7Nqg== + dependencies: + "@babel/helper-create-regexp-features-plugin" "^7.18.6" + "@babel/helper-plugin-utils" "^7.18.6" + +"@babel/plugin-transform-arrow-functions@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-arrow-functions/-/plugin-transform-arrow-functions-7.22.5.tgz#e5ba566d0c58a5b2ba2a8b795450641950b71958" + integrity sha512-26lTNXoVRdAnsaDXPpvCNUq+OVWEVC6bx7Vvz9rC53F2bagUWW4u4ii2+h8Fejfh7RYqPxn+libeFBBck9muEw== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-transform-async-generator-functions@^7.22.15": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-async-generator-functions/-/plugin-transform-async-generator-functions-7.22.15.tgz#3b153af4a6b779f340d5b80d3f634f55820aefa3" + integrity sha512-jBm1Es25Y+tVoTi5rfd5t1KLmL8ogLKpXszboWOTTtGFGz2RKnQe2yn7HbZ+kb/B8N0FVSGQo874NSlOU1T4+w== + dependencies: + "@babel/helper-environment-visitor" "^7.22.5" + "@babel/helper-plugin-utils" "^7.22.5" + "@babel/helper-remap-async-to-generator" "^7.22.9" + "@babel/plugin-syntax-async-generators" "^7.8.4" + +"@babel/plugin-transform-async-to-generator@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-async-to-generator/-/plugin-transform-async-to-generator-7.22.5.tgz#c7a85f44e46f8952f6d27fe57c2ed3cc084c3775" + integrity sha512-b1A8D8ZzE/VhNDoV1MSJTnpKkCG5bJo+19R4o4oy03zM7ws8yEMK755j61Dc3EyvdysbqH5BOOTquJ7ZX9C6vQ== + dependencies: + "@babel/helper-module-imports" "^7.22.5" + "@babel/helper-plugin-utils" "^7.22.5" + "@babel/helper-remap-async-to-generator" "^7.22.5" + +"@babel/plugin-transform-block-scoped-functions@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-block-scoped-functions/-/plugin-transform-block-scoped-functions-7.22.5.tgz#27978075bfaeb9fa586d3cb63a3d30c1de580024" + integrity sha512-tdXZ2UdknEKQWKJP1KMNmuF5Lx3MymtMN/pvA+p/VEkhK8jVcQ1fzSy8KM9qRYhAf2/lV33hoMPKI/xaI9sADA== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-transform-block-scoping@^7.22.15": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-block-scoping/-/plugin-transform-block-scoping-7.22.15.tgz#494eb82b87b5f8b1d8f6f28ea74078ec0a10a841" + integrity sha512-G1czpdJBZCtngoK1sJgloLiOHUnkb/bLZwqVZD8kXmq0ZnVfTTWUcs9OWtp0mBtYJ+4LQY1fllqBkOIPhXmFmw== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-transform-class-properties@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-class-properties/-/plugin-transform-class-properties-7.22.5.tgz#97a56e31ad8c9dc06a0b3710ce7803d5a48cca77" + integrity sha512-nDkQ0NfkOhPTq8YCLiWNxp1+f9fCobEjCb0n8WdbNUBc4IB5V7P1QnX9IjpSoquKrXF5SKojHleVNs2vGeHCHQ== + dependencies: + "@babel/helper-create-class-features-plugin" "^7.22.5" + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-transform-class-static-block@^7.22.11": + version "7.22.11" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-class-static-block/-/plugin-transform-class-static-block-7.22.11.tgz#dc8cc6e498f55692ac6b4b89e56d87cec766c974" + integrity sha512-GMM8gGmqI7guS/llMFk1bJDkKfn3v3C4KHK9Yg1ey5qcHcOlKb0QvcMrgzvxo+T03/4szNh5lghY+fEC98Kq9g== + dependencies: + "@babel/helper-create-class-features-plugin" "^7.22.11" + "@babel/helper-plugin-utils" "^7.22.5" + "@babel/plugin-syntax-class-static-block" "^7.14.5" + +"@babel/plugin-transform-classes@^7.22.15": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-classes/-/plugin-transform-classes-7.22.15.tgz#aaf4753aee262a232bbc95451b4bdf9599c65a0b" + integrity sha512-VbbC3PGjBdE0wAWDdHM9G8Gm977pnYI0XpqMd6LrKISj8/DJXEsWqgRuTYaNE9Bv0JGhTZUzHDlMk18IpOuoqw== + dependencies: + "@babel/helper-annotate-as-pure" "^7.22.5" + "@babel/helper-compilation-targets" "^7.22.15" + "@babel/helper-environment-visitor" "^7.22.5" + "@babel/helper-function-name" "^7.22.5" + "@babel/helper-optimise-call-expression" "^7.22.5" + "@babel/helper-plugin-utils" "^7.22.5" + "@babel/helper-replace-supers" "^7.22.9" + "@babel/helper-split-export-declaration" "^7.22.6" + globals "^11.1.0" + +"@babel/plugin-transform-computed-properties@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-computed-properties/-/plugin-transform-computed-properties-7.22.5.tgz#cd1e994bf9f316bd1c2dafcd02063ec261bb3869" + integrity sha512-4GHWBgRf0krxPX+AaPtgBAlTgTeZmqDynokHOX7aqqAB4tHs3U2Y02zH6ETFdLZGcg9UQSD1WCmkVrE9ErHeOg== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + "@babel/template" "^7.22.5" + +"@babel/plugin-transform-destructuring@^7.22.15": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-destructuring/-/plugin-transform-destructuring-7.22.15.tgz#e7404ea5bb3387073b9754be654eecb578324694" + integrity sha512-HzG8sFl1ZVGTme74Nw+X01XsUTqERVQ6/RLHo3XjGRzm7XD6QTtfS3NJotVgCGy8BzkDqRjRBD8dAyJn5TuvSQ== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-transform-dotall-regex@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-dotall-regex/-/plugin-transform-dotall-regex-7.22.5.tgz#dbb4f0e45766eb544e193fb00e65a1dd3b2a4165" + integrity sha512-5/Yk9QxCQCl+sOIB1WelKnVRxTJDSAIxtJLL2/pqL14ZVlbH0fUQUZa/T5/UnQtBNgghR7mfB8ERBKyKPCi7Vw== + dependencies: + "@babel/helper-create-regexp-features-plugin" "^7.22.5" + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-transform-duplicate-keys@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-duplicate-keys/-/plugin-transform-duplicate-keys-7.22.5.tgz#b6e6428d9416f5f0bba19c70d1e6e7e0b88ab285" + integrity sha512-dEnYD+9BBgld5VBXHnF/DbYGp3fqGMsyxKbtD1mDyIA7AkTSpKXFhCVuj/oQVOoALfBs77DudA0BE4d5mcpmqw== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-transform-dynamic-import@^7.22.11": + version "7.22.11" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-dynamic-import/-/plugin-transform-dynamic-import-7.22.11.tgz#2c7722d2a5c01839eaf31518c6ff96d408e447aa" + integrity sha512-g/21plo58sfteWjaO0ZNVb+uEOkJNjAaHhbejrnBmu011l/eNDScmkbjCC3l4FKb10ViaGU4aOkFznSu2zRHgA== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + "@babel/plugin-syntax-dynamic-import" "^7.8.3" + +"@babel/plugin-transform-exponentiation-operator@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-exponentiation-operator/-/plugin-transform-exponentiation-operator-7.22.5.tgz#402432ad544a1f9a480da865fda26be653e48f6a" + integrity sha512-vIpJFNM/FjZ4rh1myqIya9jXwrwwgFRHPjT3DkUA9ZLHuzox8jiXkOLvwm1H+PQIP3CqfC++WPKeuDi0Sjdj1g== + dependencies: + "@babel/helper-builder-binary-assignment-operator-visitor" "^7.22.5" + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-transform-export-namespace-from@^7.22.11": + version "7.22.11" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-export-namespace-from/-/plugin-transform-export-namespace-from-7.22.11.tgz#b3c84c8f19880b6c7440108f8929caf6056db26c" + integrity sha512-xa7aad7q7OiT8oNZ1mU7NrISjlSkVdMbNxn9IuLZyL9AJEhs1Apba3I+u5riX1dIkdptP5EKDG5XDPByWxtehw== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + "@babel/plugin-syntax-export-namespace-from" "^7.8.3" + +"@babel/plugin-transform-flow-strip-types@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-flow-strip-types/-/plugin-transform-flow-strip-types-7.22.5.tgz#0bb17110c7bf5b35a60754b2f00c58302381dee2" + integrity sha512-tujNbZdxdG0/54g/oua8ISToaXTFBf8EnSb5PgQSciIXWOWKX3S4+JR7ZE9ol8FZwf9kxitzkGQ+QWeov/mCiA== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + "@babel/plugin-syntax-flow" "^7.22.5" + +"@babel/plugin-transform-for-of@^7.22.15": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-for-of/-/plugin-transform-for-of-7.22.15.tgz#f64b4ccc3a4f131a996388fae7680b472b306b29" + integrity sha512-me6VGeHsx30+xh9fbDLLPi0J1HzmeIIyenoOQHuw2D4m2SAU3NrspX5XxJLBpqn5yrLzrlw2Iy3RA//Bx27iOA== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-transform-function-name@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-function-name/-/plugin-transform-function-name-7.22.5.tgz#935189af68b01898e0d6d99658db6b164205c143" + integrity sha512-UIzQNMS0p0HHiQm3oelztj+ECwFnj+ZRV4KnguvlsD2of1whUeM6o7wGNj6oLwcDoAXQ8gEqfgC24D+VdIcevg== + dependencies: + "@babel/helper-compilation-targets" "^7.22.5" + "@babel/helper-function-name" "^7.22.5" + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-transform-json-strings@^7.22.11": + version "7.22.11" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-json-strings/-/plugin-transform-json-strings-7.22.11.tgz#689a34e1eed1928a40954e37f74509f48af67835" + integrity sha512-CxT5tCqpA9/jXFlme9xIBCc5RPtdDq3JpkkhgHQqtDdiTnTI0jtZ0QzXhr5DILeYifDPp2wvY2ad+7+hLMW5Pw== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + "@babel/plugin-syntax-json-strings" "^7.8.3" + +"@babel/plugin-transform-literals@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-literals/-/plugin-transform-literals-7.22.5.tgz#e9341f4b5a167952576e23db8d435849b1dd7920" + integrity sha512-fTLj4D79M+mepcw3dgFBTIDYpbcB9Sm0bpm4ppXPaO+U+PKFFyV9MGRvS0gvGw62sd10kT5lRMKXAADb9pWy8g== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-transform-logical-assignment-operators@^7.22.11": + version "7.22.11" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-logical-assignment-operators/-/plugin-transform-logical-assignment-operators-7.22.11.tgz#24c522a61688bde045b7d9bc3c2597a4d948fc9c" + integrity sha512-qQwRTP4+6xFCDV5k7gZBF3C31K34ut0tbEcTKxlX/0KXxm9GLcO14p570aWxFvVzx6QAfPgq7gaeIHXJC8LswQ== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + "@babel/plugin-syntax-logical-assignment-operators" "^7.10.4" + +"@babel/plugin-transform-member-expression-literals@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-member-expression-literals/-/plugin-transform-member-expression-literals-7.22.5.tgz#4fcc9050eded981a468347dd374539ed3e058def" + integrity sha512-RZEdkNtzzYCFl9SE9ATaUMTj2hqMb4StarOJLrZRbqqU4HSBE7UlBw9WBWQiDzrJZJdUWiMTVDI6Gv/8DPvfew== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-transform-modules-amd@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-modules-amd/-/plugin-transform-modules-amd-7.22.5.tgz#4e045f55dcf98afd00f85691a68fc0780704f526" + integrity sha512-R+PTfLTcYEmb1+kK7FNkhQ1gP4KgjpSO6HfH9+f8/yfp2Nt3ggBjiVpRwmwTlfqZLafYKJACy36yDXlEmI9HjQ== + dependencies: + "@babel/helper-module-transforms" "^7.22.5" + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-transform-modules-commonjs@^7.13.8", "@babel/plugin-transform-modules-commonjs@^7.22.15": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-modules-commonjs/-/plugin-transform-modules-commonjs-7.22.15.tgz#b11810117ed4ee7691b29bd29fd9f3f98276034f" + integrity sha512-jWL4eh90w0HQOTKP2MoXXUpVxilxsB2Vl4ji69rSjS3EcZ/v4sBmn+A3NpepuJzBhOaEBbR7udonlHHn5DWidg== + dependencies: + "@babel/helper-module-transforms" "^7.22.15" + "@babel/helper-plugin-utils" "^7.22.5" + "@babel/helper-simple-access" "^7.22.5" + +"@babel/plugin-transform-modules-systemjs@^7.22.11": + version "7.22.11" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-modules-systemjs/-/plugin-transform-modules-systemjs-7.22.11.tgz#3386be5875d316493b517207e8f1931d93154bb1" + integrity sha512-rIqHmHoMEOhI3VkVf5jQ15l539KrwhzqcBO6wdCNWPWc/JWt9ILNYNUssbRpeq0qWns8svuw8LnMNCvWBIJ8wA== + dependencies: + "@babel/helper-hoist-variables" "^7.22.5" + "@babel/helper-module-transforms" "^7.22.9" + "@babel/helper-plugin-utils" "^7.22.5" + "@babel/helper-validator-identifier" "^7.22.5" + +"@babel/plugin-transform-modules-umd@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-modules-umd/-/plugin-transform-modules-umd-7.22.5.tgz#4694ae40a87b1745e3775b6a7fe96400315d4f98" + integrity sha512-+S6kzefN/E1vkSsKx8kmQuqeQsvCKCd1fraCM7zXm4SFoggI099Tr4G8U81+5gtMdUeMQ4ipdQffbKLX0/7dBQ== + dependencies: + "@babel/helper-module-transforms" "^7.22.5" + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-transform-named-capturing-groups-regex@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-named-capturing-groups-regex/-/plugin-transform-named-capturing-groups-regex-7.22.5.tgz#67fe18ee8ce02d57c855185e27e3dc959b2e991f" + integrity sha512-YgLLKmS3aUBhHaxp5hi1WJTgOUb/NCuDHzGT9z9WTt3YG+CPRhJs6nprbStx6DnWM4dh6gt7SU3sZodbZ08adQ== + dependencies: + "@babel/helper-create-regexp-features-plugin" "^7.22.5" + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-transform-new-target@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-new-target/-/plugin-transform-new-target-7.22.5.tgz#1b248acea54ce44ea06dfd37247ba089fcf9758d" + integrity sha512-AsF7K0Fx/cNKVyk3a+DW0JLo+Ua598/NxMRvxDnkpCIGFh43+h/v2xyhRUYf6oD8gE4QtL83C7zZVghMjHd+iw== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-transform-nullish-coalescing-operator@^7.22.11": + version "7.22.11" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-nullish-coalescing-operator/-/plugin-transform-nullish-coalescing-operator-7.22.11.tgz#debef6c8ba795f5ac67cd861a81b744c5d38d9fc" + integrity sha512-YZWOw4HxXrotb5xsjMJUDlLgcDXSfO9eCmdl1bgW4+/lAGdkjaEvOnQ4p5WKKdUgSzO39dgPl0pTnfxm0OAXcg== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + "@babel/plugin-syntax-nullish-coalescing-operator" "^7.8.3" + +"@babel/plugin-transform-numeric-separator@^7.22.11": + version "7.22.11" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-numeric-separator/-/plugin-transform-numeric-separator-7.22.11.tgz#498d77dc45a6c6db74bb829c02a01c1d719cbfbd" + integrity sha512-3dzU4QGPsILdJbASKhF/V2TVP+gJya1PsueQCxIPCEcerqF21oEcrob4mzjsp2Py/1nLfF5m+xYNMDpmA8vffg== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + "@babel/plugin-syntax-numeric-separator" "^7.10.4" + +"@babel/plugin-transform-object-rest-spread@^7.22.15": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-object-rest-spread/-/plugin-transform-object-rest-spread-7.22.15.tgz#21a95db166be59b91cde48775310c0df6e1da56f" + integrity sha512-fEB+I1+gAmfAyxZcX1+ZUwLeAuuf8VIg67CTznZE0MqVFumWkh8xWtn58I4dxdVf080wn7gzWoF8vndOViJe9Q== + dependencies: + "@babel/compat-data" "^7.22.9" + "@babel/helper-compilation-targets" "^7.22.15" + "@babel/helper-plugin-utils" "^7.22.5" + "@babel/plugin-syntax-object-rest-spread" "^7.8.3" + "@babel/plugin-transform-parameters" "^7.22.15" + +"@babel/plugin-transform-object-super@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-object-super/-/plugin-transform-object-super-7.22.5.tgz#794a8d2fcb5d0835af722173c1a9d704f44e218c" + integrity sha512-klXqyaT9trSjIUrcsYIfETAzmOEZL3cBYqOYLJxBHfMFFggmXOv+NYSX/Jbs9mzMVESw/WycLFPRx8ba/b2Ipw== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + "@babel/helper-replace-supers" "^7.22.5" + +"@babel/plugin-transform-optional-catch-binding@^7.22.11": + version "7.22.11" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-optional-catch-binding/-/plugin-transform-optional-catch-binding-7.22.11.tgz#461cc4f578a127bb055527b3e77404cad38c08e0" + integrity sha512-rli0WxesXUeCJnMYhzAglEjLWVDF6ahb45HuprcmQuLidBJFWjNnOzssk2kuc6e33FlLaiZhG/kUIzUMWdBKaQ== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + "@babel/plugin-syntax-optional-catch-binding" "^7.8.3" + +"@babel/plugin-transform-optional-chaining@^7.22.15": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-optional-chaining/-/plugin-transform-optional-chaining-7.22.15.tgz#d7a5996c2f7ca4ad2ad16dbb74444e5c4385b1ba" + integrity sha512-ngQ2tBhq5vvSJw2Q2Z9i7ealNkpDMU0rGWnHPKqRZO0tzZ5tlaoz4hDvhXioOoaE0X2vfNss1djwg0DXlfu30A== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + "@babel/helper-skip-transparent-expression-wrappers" "^7.22.5" + "@babel/plugin-syntax-optional-chaining" "^7.8.3" + +"@babel/plugin-transform-parameters@^7.20.7", "@babel/plugin-transform-parameters@^7.22.15": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-parameters/-/plugin-transform-parameters-7.22.15.tgz#719ca82a01d177af358df64a514d64c2e3edb114" + integrity sha512-hjk7qKIqhyzhhUvRT683TYQOFa/4cQKwQy7ALvTpODswN40MljzNDa0YldevS6tGbxwaEKVn502JmY0dP7qEtQ== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-transform-private-methods@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-private-methods/-/plugin-transform-private-methods-7.22.5.tgz#21c8af791f76674420a147ae62e9935d790f8722" + integrity sha512-PPjh4gyrQnGe97JTalgRGMuU4icsZFnWkzicB/fUtzlKUqvsWBKEpPPfr5a2JiyirZkHxnAqkQMO5Z5B2kK3fA== + dependencies: + "@babel/helper-create-class-features-plugin" "^7.22.5" + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-transform-private-property-in-object@^7.22.11": + version "7.22.11" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-private-property-in-object/-/plugin-transform-private-property-in-object-7.22.11.tgz#ad45c4fc440e9cb84c718ed0906d96cf40f9a4e1" + integrity sha512-sSCbqZDBKHetvjSwpyWzhuHkmW5RummxJBVbYLkGkaiTOWGxml7SXt0iWa03bzxFIx7wOj3g/ILRd0RcJKBeSQ== + dependencies: + "@babel/helper-annotate-as-pure" "^7.22.5" + "@babel/helper-create-class-features-plugin" "^7.22.11" + "@babel/helper-plugin-utils" "^7.22.5" + "@babel/plugin-syntax-private-property-in-object" "^7.14.5" + +"@babel/plugin-transform-property-literals@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-property-literals/-/plugin-transform-property-literals-7.22.5.tgz#b5ddabd73a4f7f26cd0e20f5db48290b88732766" + integrity sha512-TiOArgddK3mK/x1Qwf5hay2pxI6wCZnvQqrFSqbtg1GLl2JcNMitVH/YnqjP+M31pLUeTfzY1HAXFDnUBV30rQ== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-transform-react-display-name@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-react-display-name/-/plugin-transform-react-display-name-7.22.5.tgz#3c4326f9fce31c7968d6cb9debcaf32d9e279a2b" + integrity sha512-PVk3WPYudRF5z4GKMEYUrLjPl38fJSKNaEOkFuoprioowGuWN6w2RKznuFNSlJx7pzzXXStPUnNSOEO0jL5EVw== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-transform-react-jsx-development@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-react-jsx-development/-/plugin-transform-react-jsx-development-7.22.5.tgz#e716b6edbef972a92165cd69d92f1255f7e73e87" + integrity sha512-bDhuzwWMuInwCYeDeMzyi7TaBgRQei6DqxhbyniL7/VG4RSS7HtSL2QbY4eESy1KJqlWt8g3xeEBGPuo+XqC8A== + dependencies: + "@babel/plugin-transform-react-jsx" "^7.22.5" + +"@babel/plugin-transform-react-jsx@^7.22.15", "@babel/plugin-transform-react-jsx@^7.22.5": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-react-jsx/-/plugin-transform-react-jsx-7.22.15.tgz#7e6266d88705d7c49f11c98db8b9464531289cd6" + integrity sha512-oKckg2eZFa8771O/5vi7XeTvmM6+O9cxZu+kanTU7tD4sin5nO/G8jGJhq8Hvt2Z0kUoEDRayuZLaUlYl8QuGA== + dependencies: + "@babel/helper-annotate-as-pure" "^7.22.5" + "@babel/helper-module-imports" "^7.22.15" + "@babel/helper-plugin-utils" "^7.22.5" + "@babel/plugin-syntax-jsx" "^7.22.5" + "@babel/types" "^7.22.15" + +"@babel/plugin-transform-react-pure-annotations@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-react-pure-annotations/-/plugin-transform-react-pure-annotations-7.22.5.tgz#1f58363eef6626d6fa517b95ac66fe94685e32c0" + integrity sha512-gP4k85wx09q+brArVinTXhWiyzLl9UpmGva0+mWyKxk6JZequ05x3eUcIUE+FyttPKJFRRVtAvQaJ6YF9h1ZpA== + dependencies: + "@babel/helper-annotate-as-pure" "^7.22.5" + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-transform-regenerator@^7.22.10": + version "7.22.10" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-regenerator/-/plugin-transform-regenerator-7.22.10.tgz#8ceef3bd7375c4db7652878b0241b2be5d0c3cca" + integrity sha512-F28b1mDt8KcT5bUyJc/U9nwzw6cV+UmTeRlXYIl2TNqMMJif0Jeey9/RQ3C4NOd2zp0/TRsDns9ttj2L523rsw== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + regenerator-transform "^0.15.2" + +"@babel/plugin-transform-reserved-words@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-reserved-words/-/plugin-transform-reserved-words-7.22.5.tgz#832cd35b81c287c4bcd09ce03e22199641f964fb" + integrity sha512-DTtGKFRQUDm8svigJzZHzb/2xatPc6TzNvAIJ5GqOKDsGFYgAskjRulbR/vGsPKq3OPqtexnz327qYpP57RFyA== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-transform-runtime@^7.22.9": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-runtime/-/plugin-transform-runtime-7.22.15.tgz#3a625c4c05a39e932d7d34f5d4895cdd0172fdc9" + integrity sha512-tEVLhk8NRZSmwQ0DJtxxhTrCht1HVo8VaMzYT4w6lwyKBuHsgoioAUA7/6eT2fRfc5/23fuGdlwIxXhRVgWr4g== + dependencies: + "@babel/helper-module-imports" "^7.22.15" + "@babel/helper-plugin-utils" "^7.22.5" + babel-plugin-polyfill-corejs2 "^0.4.5" + babel-plugin-polyfill-corejs3 "^0.8.3" + babel-plugin-polyfill-regenerator "^0.5.2" + semver "^6.3.1" + +"@babel/plugin-transform-shorthand-properties@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-shorthand-properties/-/plugin-transform-shorthand-properties-7.22.5.tgz#6e277654be82b5559fc4b9f58088507c24f0c624" + integrity sha512-vM4fq9IXHscXVKzDv5itkO1X52SmdFBFcMIBZ2FRn2nqVYqw6dBexUgMvAjHW+KXpPPViD/Yo3GrDEBaRC0QYA== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-transform-spread@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-spread/-/plugin-transform-spread-7.22.5.tgz#6487fd29f229c95e284ba6c98d65eafb893fea6b" + integrity sha512-5ZzDQIGyvN4w8+dMmpohL6MBo+l2G7tfC/O2Dg7/hjpgeWvUx8FzfeOKxGog9IimPa4YekaQ9PlDqTLOljkcxg== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + "@babel/helper-skip-transparent-expression-wrappers" "^7.22.5" + +"@babel/plugin-transform-sticky-regex@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-sticky-regex/-/plugin-transform-sticky-regex-7.22.5.tgz#295aba1595bfc8197abd02eae5fc288c0deb26aa" + integrity sha512-zf7LuNpHG0iEeiyCNwX4j3gDg1jgt1k3ZdXBKbZSoA3BbGQGvMiSvfbZRR3Dr3aeJe3ooWFZxOOG3IRStYp2Bw== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-transform-template-literals@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-template-literals/-/plugin-transform-template-literals-7.22.5.tgz#8f38cf291e5f7a8e60e9f733193f0bcc10909bff" + integrity sha512-5ciOehRNf+EyUeewo8NkbQiUs4d6ZxiHo6BcBcnFlgiJfu16q0bQUw9Jvo0b0gBKFG1SMhDSjeKXSYuJLeFSMA== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-transform-typeof-symbol@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-typeof-symbol/-/plugin-transform-typeof-symbol-7.22.5.tgz#5e2ba478da4b603af8673ff7c54f75a97b716b34" + integrity sha512-bYkI5lMzL4kPii4HHEEChkD0rkc+nvnlR6+o/qdqR6zrm0Sv/nodmyLhlq2DO0YKLUNd2VePmPRjJXSBh9OIdA== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-transform-typescript@^7.22.15": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-typescript/-/plugin-transform-typescript-7.22.15.tgz#15adef906451d86349eb4b8764865c960eb54127" + integrity sha512-1uirS0TnijxvQLnlv5wQBwOX3E1wCFX7ITv+9pBV2wKEk4K+M5tqDaoNXnTH8tjEIYHLO98MwiTWO04Ggz4XuA== + dependencies: + "@babel/helper-annotate-as-pure" "^7.22.5" + "@babel/helper-create-class-features-plugin" "^7.22.15" + "@babel/helper-plugin-utils" "^7.22.5" + "@babel/plugin-syntax-typescript" "^7.22.5" + +"@babel/plugin-transform-unicode-escapes@^7.22.10": + version "7.22.10" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-unicode-escapes/-/plugin-transform-unicode-escapes-7.22.10.tgz#c723f380f40a2b2f57a62df24c9005834c8616d9" + integrity sha512-lRfaRKGZCBqDlRU3UIFovdp9c9mEvlylmpod0/OatICsSfuQ9YFthRo1tpTkGsklEefZdqlEFdY4A2dwTb6ohg== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-transform-unicode-property-regex@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-unicode-property-regex/-/plugin-transform-unicode-property-regex-7.22.5.tgz#098898f74d5c1e86660dc112057b2d11227f1c81" + integrity sha512-HCCIb+CbJIAE6sXn5CjFQXMwkCClcOfPCzTlilJ8cUatfzwHlWQkbtV0zD338u9dZskwvuOYTuuaMaA8J5EI5A== + dependencies: + "@babel/helper-create-regexp-features-plugin" "^7.22.5" + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-transform-unicode-regex@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-unicode-regex/-/plugin-transform-unicode-regex-7.22.5.tgz#ce7e7bb3ef208c4ff67e02a22816656256d7a183" + integrity sha512-028laaOKptN5vHJf9/Arr/HiJekMd41hOEZYvNsrsXqJ7YPYuX2bQxh31fkZzGmq3YqHRJzYFFAVYvKfMPKqyg== + dependencies: + "@babel/helper-create-regexp-features-plugin" "^7.22.5" + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-transform-unicode-sets-regex@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-transform-unicode-sets-regex/-/plugin-transform-unicode-sets-regex-7.22.5.tgz#77788060e511b708ffc7d42fdfbc5b37c3004e91" + integrity sha512-lhMfi4FC15j13eKrh3DnYHjpGj6UKQHtNKTbtc1igvAhRy4+kLhV07OpLcsN0VgDEw/MjAvJO4BdMJsHwMhzCg== + dependencies: + "@babel/helper-create-regexp-features-plugin" "^7.22.5" + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/preset-env@^7.20.2", "@babel/preset-env@^7.22.9": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/preset-env/-/preset-env-7.22.15.tgz#142716f8e00bc030dae5b2ac6a46fbd8b3e18ff8" + integrity sha512-tZFHr54GBkHk6hQuVA8w4Fmq+MSPsfvMG0vPnOYyTnJpyfMqybL8/MbNCPRT9zc2KBO2pe4tq15g6Uno4Jpoag== + dependencies: + "@babel/compat-data" "^7.22.9" + "@babel/helper-compilation-targets" "^7.22.15" + "@babel/helper-plugin-utils" "^7.22.5" + "@babel/helper-validator-option" "^7.22.15" + "@babel/plugin-bugfix-safari-id-destructuring-collision-in-function-expression" "^7.22.15" + "@babel/plugin-bugfix-v8-spread-parameters-in-optional-chaining" "^7.22.15" + "@babel/plugin-proposal-private-property-in-object" "7.21.0-placeholder-for-preset-env.2" + "@babel/plugin-syntax-async-generators" "^7.8.4" + "@babel/plugin-syntax-class-properties" "^7.12.13" + "@babel/plugin-syntax-class-static-block" "^7.14.5" + "@babel/plugin-syntax-dynamic-import" "^7.8.3" + "@babel/plugin-syntax-export-namespace-from" "^7.8.3" + "@babel/plugin-syntax-import-assertions" "^7.22.5" + "@babel/plugin-syntax-import-attributes" "^7.22.5" + "@babel/plugin-syntax-import-meta" "^7.10.4" + "@babel/plugin-syntax-json-strings" "^7.8.3" + "@babel/plugin-syntax-logical-assignment-operators" "^7.10.4" + "@babel/plugin-syntax-nullish-coalescing-operator" "^7.8.3" + "@babel/plugin-syntax-numeric-separator" "^7.10.4" + "@babel/plugin-syntax-object-rest-spread" "^7.8.3" + "@babel/plugin-syntax-optional-catch-binding" "^7.8.3" + "@babel/plugin-syntax-optional-chaining" "^7.8.3" + "@babel/plugin-syntax-private-property-in-object" "^7.14.5" + "@babel/plugin-syntax-top-level-await" "^7.14.5" + "@babel/plugin-syntax-unicode-sets-regex" "^7.18.6" + "@babel/plugin-transform-arrow-functions" "^7.22.5" + "@babel/plugin-transform-async-generator-functions" "^7.22.15" + "@babel/plugin-transform-async-to-generator" "^7.22.5" + "@babel/plugin-transform-block-scoped-functions" "^7.22.5" + "@babel/plugin-transform-block-scoping" "^7.22.15" + "@babel/plugin-transform-class-properties" "^7.22.5" + "@babel/plugin-transform-class-static-block" "^7.22.11" + "@babel/plugin-transform-classes" "^7.22.15" + "@babel/plugin-transform-computed-properties" "^7.22.5" + "@babel/plugin-transform-destructuring" "^7.22.15" + "@babel/plugin-transform-dotall-regex" "^7.22.5" + "@babel/plugin-transform-duplicate-keys" "^7.22.5" + "@babel/plugin-transform-dynamic-import" "^7.22.11" + "@babel/plugin-transform-exponentiation-operator" "^7.22.5" + "@babel/plugin-transform-export-namespace-from" "^7.22.11" + "@babel/plugin-transform-for-of" "^7.22.15" + "@babel/plugin-transform-function-name" "^7.22.5" + "@babel/plugin-transform-json-strings" "^7.22.11" + "@babel/plugin-transform-literals" "^7.22.5" + "@babel/plugin-transform-logical-assignment-operators" "^7.22.11" + "@babel/plugin-transform-member-expression-literals" "^7.22.5" + "@babel/plugin-transform-modules-amd" "^7.22.5" + "@babel/plugin-transform-modules-commonjs" "^7.22.15" + "@babel/plugin-transform-modules-systemjs" "^7.22.11" + "@babel/plugin-transform-modules-umd" "^7.22.5" + "@babel/plugin-transform-named-capturing-groups-regex" "^7.22.5" + "@babel/plugin-transform-new-target" "^7.22.5" + "@babel/plugin-transform-nullish-coalescing-operator" "^7.22.11" + "@babel/plugin-transform-numeric-separator" "^7.22.11" + "@babel/plugin-transform-object-rest-spread" "^7.22.15" + "@babel/plugin-transform-object-super" "^7.22.5" + "@babel/plugin-transform-optional-catch-binding" "^7.22.11" + "@babel/plugin-transform-optional-chaining" "^7.22.15" + "@babel/plugin-transform-parameters" "^7.22.15" + "@babel/plugin-transform-private-methods" "^7.22.5" + "@babel/plugin-transform-private-property-in-object" "^7.22.11" + "@babel/plugin-transform-property-literals" "^7.22.5" + "@babel/plugin-transform-regenerator" "^7.22.10" + "@babel/plugin-transform-reserved-words" "^7.22.5" + "@babel/plugin-transform-shorthand-properties" "^7.22.5" + "@babel/plugin-transform-spread" "^7.22.5" + "@babel/plugin-transform-sticky-regex" "^7.22.5" + "@babel/plugin-transform-template-literals" "^7.22.5" + "@babel/plugin-transform-typeof-symbol" "^7.22.5" + "@babel/plugin-transform-unicode-escapes" "^7.22.10" + "@babel/plugin-transform-unicode-property-regex" "^7.22.5" + "@babel/plugin-transform-unicode-regex" "^7.22.5" + "@babel/plugin-transform-unicode-sets-regex" "^7.22.5" + "@babel/preset-modules" "0.1.6-no-external-plugins" + "@babel/types" "^7.22.15" + babel-plugin-polyfill-corejs2 "^0.4.5" + babel-plugin-polyfill-corejs3 "^0.8.3" + babel-plugin-polyfill-regenerator "^0.5.2" + core-js-compat "^3.31.0" + semver "^6.3.1" + +"@babel/preset-flow@^7.13.13", "@babel/preset-flow@^7.22.5": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/preset-flow/-/preset-flow-7.22.15.tgz#30318deb9b3ebd9f5738e96da03a531e0cd3165d" + integrity sha512-dB5aIMqpkgbTfN5vDdTRPzjqtWiZcRESNR88QYnoPR+bmdYoluOzMX9tQerTv0XzSgZYctPfO1oc0N5zdog1ew== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + "@babel/helper-validator-option" "^7.22.15" + "@babel/plugin-transform-flow-strip-types" "^7.22.5" + +"@babel/preset-modules@0.1.6-no-external-plugins": + version "0.1.6-no-external-plugins" + resolved "https://registry.yarnpkg.com/@babel/preset-modules/-/preset-modules-0.1.6-no-external-plugins.tgz#ccb88a2c49c817236861fee7826080573b8a923a" + integrity sha512-HrcgcIESLm9aIR842yhJ5RWan/gebQUJ6E/E5+rf0y9o6oj7w0Br+sWuL6kEQ/o/AdfvR1Je9jG18/gnpwjEyA== + dependencies: + "@babel/helper-plugin-utils" "^7.0.0" + "@babel/types" "^7.4.4" + esutils "^2.0.2" + +"@babel/preset-react@^7.18.6", "@babel/preset-react@^7.22.5": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/preset-react/-/preset-react-7.22.15.tgz#9a776892b648e13cc8ca2edf5ed1264eea6b6afc" + integrity sha512-Csy1IJ2uEh/PecCBXXoZGAZBeCATTuePzCSB7dLYWS0vOEj6CNpjxIhW4duWwZodBNueH7QO14WbGn8YyeuN9w== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + "@babel/helper-validator-option" "^7.22.15" + "@babel/plugin-transform-react-display-name" "^7.22.5" + "@babel/plugin-transform-react-jsx" "^7.22.15" + "@babel/plugin-transform-react-jsx-development" "^7.22.5" + "@babel/plugin-transform-react-pure-annotations" "^7.22.5" + +"@babel/preset-typescript@^7.13.0", "@babel/preset-typescript@^7.21.0", "@babel/preset-typescript@^7.22.5": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/preset-typescript/-/preset-typescript-7.22.15.tgz#43db30516fae1d417d748105a0bc95f637239d48" + integrity sha512-HblhNmh6yM+cU4VwbBRpxFhxsTdfS1zsvH9W+gEjD0ARV9+8B4sNfpI6GuhePti84nuvhiwKS539jKPFHskA9A== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + "@babel/helper-validator-option" "^7.22.15" + "@babel/plugin-syntax-jsx" "^7.22.5" + "@babel/plugin-transform-modules-commonjs" "^7.22.15" + "@babel/plugin-transform-typescript" "^7.22.15" + +"@babel/register@^7.13.16": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/register/-/register-7.22.15.tgz#c2c294a361d59f5fa7bcc8b97ef7319c32ecaec7" + integrity sha512-V3Q3EqoQdn65RCgTLwauZaTfd1ShhwPmbBv+1dkZV/HpCGMKVyn6oFcRlI7RaKqiDQjX2Qd3AuoEguBgdjIKlg== + dependencies: + clone-deep "^4.0.1" + find-cache-dir "^2.0.0" + make-dir "^2.1.0" + pirates "^4.0.5" + source-map-support "^0.5.16" + +"@babel/regjsgen@^0.8.0": + version "0.8.0" + resolved "https://registry.yarnpkg.com/@babel/regjsgen/-/regjsgen-0.8.0.tgz#f0ba69b075e1f05fb2825b7fad991e7adbb18310" + integrity sha512-x/rqGMdzj+fWZvCOYForTghzbtqPDZ5gPwaoNGHdgDfF2QA/XZbCBp4Moo5scrkAMPhB7z26XM/AaHuIJdgauA== + +"@babel/runtime@^7.1.2", "@babel/runtime@^7.12.5", "@babel/runtime@^7.13.10", "@babel/runtime@^7.17.8", "@babel/runtime@^7.20.7", "@babel/runtime@^7.21.0", "@babel/runtime@^7.22.6", "@babel/runtime@^7.5.5", "@babel/runtime@^7.7.6", "@babel/runtime@^7.8.4", "@babel/runtime@^7.8.7": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/runtime/-/runtime-7.22.15.tgz#38f46494ccf6cf020bd4eed7124b425e83e523b8" + integrity sha512-T0O+aa+4w0u06iNmapipJXMV4HoUir03hpx3/YqXXhu9xim3w+dVphjFWl1OH8NbZHw5Lbm9k45drDkgq2VNNA== + dependencies: + regenerator-runtime "^0.14.0" + +"@babel/template@^7.20.7", "@babel/template@^7.22.15", "@babel/template@^7.22.5": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/template/-/template-7.22.15.tgz#09576efc3830f0430f4548ef971dde1350ef2f38" + integrity sha512-QPErUVm4uyJa60rkI73qneDacvdvzxshT3kksGqlGWYdOTIUOwJ7RDUL8sGqslY1uXWSL6xMFKEXDS3ox2uF0w== + dependencies: + "@babel/code-frame" "^7.22.13" + "@babel/parser" "^7.22.15" + "@babel/types" "^7.22.15" + +"@babel/traverse@^7.1.6", "@babel/traverse@^7.21.2", "@babel/traverse@^7.22.15", "@babel/traverse@^7.22.8": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/traverse/-/traverse-7.22.15.tgz#75be4d2d6e216e880e93017f4e2389aeb77ef2d9" + integrity sha512-DdHPwvJY0sEeN4xJU5uRLmZjgMMDIvMPniLuYzUVXj/GGzysPl0/fwt44JBkyUIzGJPV8QgHMcQdQ34XFuKTYQ== + dependencies: + "@babel/code-frame" "^7.22.13" + "@babel/generator" "^7.22.15" + "@babel/helper-environment-visitor" "^7.22.5" + "@babel/helper-function-name" "^7.22.5" + "@babel/helper-hoist-variables" "^7.22.5" + "@babel/helper-split-export-declaration" "^7.22.6" + "@babel/parser" "^7.22.15" + "@babel/types" "^7.22.15" + debug "^4.1.0" + globals "^11.1.0" + +"@babel/types@^7.0.0", "@babel/types@^7.2.0", "@babel/types@^7.20.7", "@babel/types@^7.21.5", "@babel/types@^7.22.10", "@babel/types@^7.22.15", "@babel/types@^7.22.5", "@babel/types@^7.4.4": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/types/-/types-7.22.15.tgz#266cb21d2c5fd0b3931e7a91b6dd72d2f617d282" + integrity sha512-X+NLXr0N8XXmN5ZsaQdm9U2SSC3UbIYq/doL++sueHOTisgZHoKaQtZxGuV2cUPQHMfjKEfg/g6oy7Hm6SKFtA== + dependencies: + "@babel/helper-string-parser" "^7.22.5" + "@babel/helper-validator-identifier" "^7.22.15" + to-fast-properties "^2.0.0" + +"@base2/pretty-print-object@1.0.1": + version "1.0.1" + resolved "https://registry.yarnpkg.com/@base2/pretty-print-object/-/pretty-print-object-1.0.1.tgz#371ba8be66d556812dc7fb169ebc3c08378f69d4" + integrity sha512-4iri8i1AqYHJE2DstZYkyEprg6Pq6sKx3xn5FpySk9sNhH7qN2LLlHJCfDTZRILNwQNPD7mATWM0TBui7uC1pA== + +"@bcoe/v8-coverage@^0.2.3": + version "0.2.3" + resolved "https://registry.yarnpkg.com/@bcoe/v8-coverage/-/v8-coverage-0.2.3.tgz#75a2e8b51cb758a7553d6804a5932d7aace75c39" + integrity sha512-0hYQ8SB4Db5zvZB4axdMHGwEaQjkZzFjQiN9LVYvIFB2nSUHW9tYpxWriPrWDASIxiaXax83REcLxuSdnGPZtw== + +"@colors/colors@1.5.0": + version "1.5.0" + resolved "https://registry.yarnpkg.com/@colors/colors/-/colors-1.5.0.tgz#bb504579c1cae923e6576a4f5da43d25f97bdbd9" + integrity sha512-ooWCrlZP11i8GImSjTHYHLkvFDP48nS4+204nGb1RiX/WXYHmJA2III9/e2DWVabCESdW7hBAEzHRqUn9OUVvQ== + +"@discoveryjs/json-ext@^0.5.3": + version "0.5.7" + resolved "https://registry.yarnpkg.com/@discoveryjs/json-ext/-/json-ext-0.5.7.tgz#1d572bfbbe14b7704e0ba0f39b74815b84870d70" + integrity sha512-dBVuXR082gk3jsFp7Rd/JI4kytwGHecnCoTtXFb7DB6CNHp4rg5k1bhg0nWdLGLnOV71lmDzGQaLMy8iPLY0pw== + +"@emotion/is-prop-valid@^1.2.1": + version "1.2.1" + resolved "https://registry.yarnpkg.com/@emotion/is-prop-valid/-/is-prop-valid-1.2.1.tgz#23116cf1ed18bfeac910ec6436561ecb1a3885cc" + integrity sha512-61Mf7Ufx4aDxx1xlDeOm8aFFigGHE4z+0sKCa+IHCeZKiyP9RLD0Mmx7m8b9/Cf37f7NAvQOOJAbQQGVr5uERw== + dependencies: + "@emotion/memoize" "^0.8.1" + +"@emotion/memoize@^0.8.1": + version "0.8.1" + resolved "https://registry.yarnpkg.com/@emotion/memoize/-/memoize-0.8.1.tgz#c1ddb040429c6d21d38cc945fe75c818cfb68e17" + integrity sha512-W2P2c/VRW1/1tLox0mVUalvnWXxavmv/Oum2aPsRcoDJuob75FC3Y8FbpfLwUegRcxINtGUMPq0tFCvYNTBXNA== + +"@emotion/unitless@^0.8.0": + version "0.8.1" + resolved "https://registry.yarnpkg.com/@emotion/unitless/-/unitless-0.8.1.tgz#182b5a4704ef8ad91bde93f7a860a88fd92c79a3" + integrity sha512-KOEGMu6dmJZtpadb476IsZBclKvILjopjUii3V+7MnXIQCYh8W3NgNcgwo21n9LXZX6EDIKvqfjYxXebDwxKmQ== + +"@emotion/use-insertion-effect-with-fallbacks@^1.0.0": + version "1.0.1" + resolved "https://registry.yarnpkg.com/@emotion/use-insertion-effect-with-fallbacks/-/use-insertion-effect-with-fallbacks-1.0.1.tgz#08de79f54eb3406f9daaf77c76e35313da963963" + integrity sha512-jT/qyKZ9rzLErtrjGgdkMBn2OP8wl0G3sQlBb3YPryvKHsjvINUhVaPFfP+fpBcOkmrVOVEEHQFJ7nbj2TH2gw== + +"@esbuild/android-arm64@0.18.20": + version "0.18.20" + resolved "https://registry.yarnpkg.com/@esbuild/android-arm64/-/android-arm64-0.18.20.tgz#984b4f9c8d0377443cc2dfcef266d02244593622" + integrity sha512-Nz4rJcchGDtENV0eMKUNa6L12zz2zBDXuhj/Vjh18zGqB44Bi7MBMSXjgunJgjRhCmKOjnPuZp4Mb6OKqtMHLQ== + +"@esbuild/android-arm@0.18.20": + version "0.18.20" + resolved "https://registry.yarnpkg.com/@esbuild/android-arm/-/android-arm-0.18.20.tgz#fedb265bc3a589c84cc11f810804f234947c3682" + integrity sha512-fyi7TDI/ijKKNZTUJAQqiG5T7YjJXgnzkURqmGj13C6dCqckZBLdl4h7bkhHt/t0WP+zO9/zwroDvANaOqO5Sw== + +"@esbuild/android-x64@0.18.20": + version "0.18.20" + resolved "https://registry.yarnpkg.com/@esbuild/android-x64/-/android-x64-0.18.20.tgz#35cf419c4cfc8babe8893d296cd990e9e9f756f2" + integrity sha512-8GDdlePJA8D6zlZYJV/jnrRAi6rOiNaCC/JclcXpB+KIuvfBN4owLtgzY2bsxnx666XjJx2kDPUmnTtR8qKQUg== + +"@esbuild/darwin-arm64@0.18.20": + version "0.18.20" + resolved "https://registry.yarnpkg.com/@esbuild/darwin-arm64/-/darwin-arm64-0.18.20.tgz#08172cbeccf95fbc383399a7f39cfbddaeb0d7c1" + integrity sha512-bxRHW5kHU38zS2lPTPOyuyTm+S+eobPUnTNkdJEfAddYgEcll4xkT8DB9d2008DtTbl7uJag2HuE5NZAZgnNEA== + +"@esbuild/darwin-x64@0.18.20": + version "0.18.20" + resolved "https://registry.yarnpkg.com/@esbuild/darwin-x64/-/darwin-x64-0.18.20.tgz#d70d5790d8bf475556b67d0f8b7c5bdff053d85d" + integrity sha512-pc5gxlMDxzm513qPGbCbDukOdsGtKhfxD1zJKXjCCcU7ju50O7MeAZ8c4krSJcOIJGFR+qx21yMMVYwiQvyTyQ== + +"@esbuild/freebsd-arm64@0.18.20": + version "0.18.20" + resolved "https://registry.yarnpkg.com/@esbuild/freebsd-arm64/-/freebsd-arm64-0.18.20.tgz#98755cd12707f93f210e2494d6a4b51b96977f54" + integrity sha512-yqDQHy4QHevpMAaxhhIwYPMv1NECwOvIpGCZkECn8w2WFHXjEwrBn3CeNIYsibZ/iZEUemj++M26W3cNR5h+Tw== + +"@esbuild/freebsd-x64@0.18.20": + version "0.18.20" + resolved "https://registry.yarnpkg.com/@esbuild/freebsd-x64/-/freebsd-x64-0.18.20.tgz#c1eb2bff03915f87c29cece4c1a7fa1f423b066e" + integrity sha512-tgWRPPuQsd3RmBZwarGVHZQvtzfEBOreNuxEMKFcd5DaDn2PbBxfwLcj4+aenoh7ctXcbXmOQIn8HI6mCSw5MQ== + +"@esbuild/linux-arm64@0.18.20": + version "0.18.20" + resolved "https://registry.yarnpkg.com/@esbuild/linux-arm64/-/linux-arm64-0.18.20.tgz#bad4238bd8f4fc25b5a021280c770ab5fc3a02a0" + integrity sha512-2YbscF+UL7SQAVIpnWvYwM+3LskyDmPhe31pE7/aoTMFKKzIc9lLbyGUpmmb8a8AixOL61sQ/mFh3jEjHYFvdA== + +"@esbuild/linux-arm@0.18.20": + version "0.18.20" + resolved "https://registry.yarnpkg.com/@esbuild/linux-arm/-/linux-arm-0.18.20.tgz#3e617c61f33508a27150ee417543c8ab5acc73b0" + integrity sha512-/5bHkMWnq1EgKr1V+Ybz3s1hWXok7mDFUMQ4cG10AfW3wL02PSZi5kFpYKrptDsgb2WAJIvRcDm+qIvXf/apvg== + +"@esbuild/linux-ia32@0.18.20": + version "0.18.20" + resolved "https://registry.yarnpkg.com/@esbuild/linux-ia32/-/linux-ia32-0.18.20.tgz#699391cccba9aee6019b7f9892eb99219f1570a7" + integrity sha512-P4etWwq6IsReT0E1KHU40bOnzMHoH73aXp96Fs8TIT6z9Hu8G6+0SHSw9i2isWrD2nbx2qo5yUqACgdfVGx7TA== + +"@esbuild/linux-loong64@0.18.20": + version "0.18.20" + resolved "https://registry.yarnpkg.com/@esbuild/linux-loong64/-/linux-loong64-0.18.20.tgz#e6fccb7aac178dd2ffb9860465ac89d7f23b977d" + integrity sha512-nXW8nqBTrOpDLPgPY9uV+/1DjxoQ7DoB2N8eocyq8I9XuqJ7BiAMDMf9n1xZM9TgW0J8zrquIb/A7s3BJv7rjg== + +"@esbuild/linux-mips64el@0.18.20": + version "0.18.20" + resolved "https://registry.yarnpkg.com/@esbuild/linux-mips64el/-/linux-mips64el-0.18.20.tgz#eeff3a937de9c2310de30622a957ad1bd9183231" + integrity sha512-d5NeaXZcHp8PzYy5VnXV3VSd2D328Zb+9dEq5HE6bw6+N86JVPExrA6O68OPwobntbNJ0pzCpUFZTo3w0GyetQ== + +"@esbuild/linux-ppc64@0.18.20": + version "0.18.20" + resolved "https://registry.yarnpkg.com/@esbuild/linux-ppc64/-/linux-ppc64-0.18.20.tgz#2f7156bde20b01527993e6881435ad79ba9599fb" + integrity sha512-WHPyeScRNcmANnLQkq6AfyXRFr5D6N2sKgkFo2FqguP44Nw2eyDlbTdZwd9GYk98DZG9QItIiTlFLHJHjxP3FA== + +"@esbuild/linux-riscv64@0.18.20": + version "0.18.20" + resolved "https://registry.yarnpkg.com/@esbuild/linux-riscv64/-/linux-riscv64-0.18.20.tgz#6628389f210123d8b4743045af8caa7d4ddfc7a6" + integrity sha512-WSxo6h5ecI5XH34KC7w5veNnKkju3zBRLEQNY7mv5mtBmrP/MjNBCAlsM2u5hDBlS3NGcTQpoBvRzqBcRtpq1A== + +"@esbuild/linux-s390x@0.18.20": + version "0.18.20" + resolved "https://registry.yarnpkg.com/@esbuild/linux-s390x/-/linux-s390x-0.18.20.tgz#255e81fb289b101026131858ab99fba63dcf0071" + integrity sha512-+8231GMs3mAEth6Ja1iK0a1sQ3ohfcpzpRLH8uuc5/KVDFneH6jtAJLFGafpzpMRO6DzJ6AvXKze9LfFMrIHVQ== + +"@esbuild/linux-x64@0.18.20": + version "0.18.20" + resolved "https://registry.yarnpkg.com/@esbuild/linux-x64/-/linux-x64-0.18.20.tgz#c7690b3417af318a9b6f96df3031a8865176d338" + integrity sha512-UYqiqemphJcNsFEskc73jQ7B9jgwjWrSayxawS6UVFZGWrAAtkzjxSqnoclCXxWtfwLdzU+vTpcNYhpn43uP1w== + +"@esbuild/netbsd-x64@0.18.20": + version "0.18.20" + resolved "https://registry.yarnpkg.com/@esbuild/netbsd-x64/-/netbsd-x64-0.18.20.tgz#30e8cd8a3dded63975e2df2438ca109601ebe0d1" + integrity sha512-iO1c++VP6xUBUmltHZoMtCUdPlnPGdBom6IrO4gyKPFFVBKioIImVooR5I83nTew5UOYrk3gIJhbZh8X44y06A== + +"@esbuild/openbsd-x64@0.18.20": + version "0.18.20" + resolved "https://registry.yarnpkg.com/@esbuild/openbsd-x64/-/openbsd-x64-0.18.20.tgz#7812af31b205055874c8082ea9cf9ab0da6217ae" + integrity sha512-e5e4YSsuQfX4cxcygw/UCPIEP6wbIL+se3sxPdCiMbFLBWu0eiZOJ7WoD+ptCLrmjZBK1Wk7I6D/I3NglUGOxg== + +"@esbuild/sunos-x64@0.18.20": + version "0.18.20" + resolved "https://registry.yarnpkg.com/@esbuild/sunos-x64/-/sunos-x64-0.18.20.tgz#d5c275c3b4e73c9b0ecd38d1ca62c020f887ab9d" + integrity sha512-kDbFRFp0YpTQVVrqUd5FTYmWo45zGaXe0X8E1G/LKFC0v8x0vWrhOWSLITcCn63lmZIxfOMXtCfti/RxN/0wnQ== + +"@esbuild/win32-arm64@0.18.20": + version "0.18.20" + resolved "https://registry.yarnpkg.com/@esbuild/win32-arm64/-/win32-arm64-0.18.20.tgz#73bc7f5a9f8a77805f357fab97f290d0e4820ac9" + integrity sha512-ddYFR6ItYgoaq4v4JmQQaAI5s7npztfV4Ag6NrhiaW0RrnOXqBkgwZLofVTlq1daVTQNhtI5oieTvkRPfZrePg== + +"@esbuild/win32-ia32@0.18.20": + version "0.18.20" + resolved "https://registry.yarnpkg.com/@esbuild/win32-ia32/-/win32-ia32-0.18.20.tgz#ec93cbf0ef1085cc12e71e0d661d20569ff42102" + integrity sha512-Wv7QBi3ID/rROT08SABTS7eV4hX26sVduqDOTe1MvGMjNd3EjOz4b7zeexIR62GTIEKrfJXKL9LFxTYgkyeu7g== + +"@esbuild/win32-x64@0.18.20": + version "0.18.20" + resolved "https://registry.yarnpkg.com/@esbuild/win32-x64/-/win32-x64-0.18.20.tgz#786c5f41f043b07afb1af37683d7c33668858f6d" + integrity sha512-kTdfRcSiDfQca/y9QIkng02avJ+NCaQvrMejlsB3RRv5sE9rRoeBPISaZpKxHELzRxZyLvNts1P27W3wV+8geQ== + +"@eslint-community/eslint-utils@^4.2.0": + version "4.4.0" + resolved "https://registry.yarnpkg.com/@eslint-community/eslint-utils/-/eslint-utils-4.4.0.tgz#a23514e8fb9af1269d5f7788aa556798d61c6b59" + integrity sha512-1/sA4dwrzBAyeUoQ6oxahHKmrZvsnLCg4RfxW3ZFGGmQkSNQPFNLV9CUEFQP1x9EYXHTo5p6xdhZM1Ne9p/AfA== + dependencies: + eslint-visitor-keys "^3.3.0" + +"@eslint-community/regexpp@^4.6.1": + version "4.8.0" + resolved "https://registry.yarnpkg.com/@eslint-community/regexpp/-/regexpp-4.8.0.tgz#11195513186f68d42fbf449f9a7136b2c0c92005" + integrity sha512-JylOEEzDiOryeUnFbQz+oViCXS0KsvR1mvHkoMiu5+UiBvy+RYX7tzlIIIEstF/gVa2tj9AQXk3dgnxv6KxhFg== + +"@eslint/eslintrc@^2.1.2": + version "2.1.2" + resolved "https://registry.yarnpkg.com/@eslint/eslintrc/-/eslintrc-2.1.2.tgz#c6936b4b328c64496692f76944e755738be62396" + integrity sha512-+wvgpDsrB1YqAMdEUCcnTlpfVBH7Vqn6A/NT3D8WVXFIaKMlErPIZT3oCIAVCOtarRpMtelZLqJeU3t7WY6X6g== + dependencies: + ajv "^6.12.4" + debug "^4.3.2" + espree "^9.6.0" + globals "^13.19.0" + ignore "^5.2.0" + import-fresh "^3.2.1" + js-yaml "^4.1.0" + minimatch "^3.1.2" + strip-json-comments "^3.1.1" + +"@eslint/js@8.48.0": + version "8.48.0" + resolved "https://registry.yarnpkg.com/@eslint/js/-/js-8.48.0.tgz#642633964e217905436033a2bd08bf322849b7fb" + integrity sha512-ZSjtmelB7IJfWD2Fvb7+Z+ChTIKWq6kjda95fLcQKNS5aheVHn4IkfgRQE3sIIzTcSLwLcLZUD9UBt+V7+h+Pw== + +"@fal-works/esbuild-plugin-global-externals@^2.1.2": + version "2.1.2" + resolved "https://registry.yarnpkg.com/@fal-works/esbuild-plugin-global-externals/-/esbuild-plugin-global-externals-2.1.2.tgz#c05ed35ad82df8e6ac616c68b92c2282bd083ba4" + integrity sha512-cEee/Z+I12mZcFJshKcCqC8tuX5hG3s+d+9nZ3LabqKF1vKdF41B92pJVCBggjAGORAeOzyyDDKrZwIkLffeOQ== + +"@floating-ui/core@^1.4.1": + version "1.4.1" + resolved "https://registry.yarnpkg.com/@floating-ui/core/-/core-1.4.1.tgz#0d633f4b76052668afb932492ac452f7ebe97f17" + integrity sha512-jk3WqquEJRlcyu7997NtR5PibI+y5bi+LS3hPmguVClypenMsCY3CBa3LAQnozRCtCrYWSEtAdiskpamuJRFOQ== + dependencies: + "@floating-ui/utils" "^0.1.1" + +"@floating-ui/dom@^1.2.1", "@floating-ui/dom@^1.5.1": + version "1.5.1" + resolved "https://registry.yarnpkg.com/@floating-ui/dom/-/dom-1.5.1.tgz#88b70defd002fe851f17b4a25efb2d3c04d7a8d7" + integrity sha512-KwvVcPSXg6mQygvA1TjbN/gh///36kKtllIF8SUm0qpFj8+rvYrpvlYdL1JoA71SHpDqgSSdGOSoQ0Mp3uY5aw== + dependencies: + "@floating-ui/core" "^1.4.1" + "@floating-ui/utils" "^0.1.1" + +"@floating-ui/react-dom@^1.3.0": + version "1.3.0" + resolved "https://registry.yarnpkg.com/@floating-ui/react-dom/-/react-dom-1.3.0.tgz#4d35d416eb19811c2b0e9271100a6aa18c1579b3" + integrity sha512-htwHm67Ji5E/pROEAr7f8IKFShuiCKHwUC/UY4vC3I5jiSvGFAYnSYiZO5MlGmads+QqvUkR9ANHEguGrDv72g== + dependencies: + "@floating-ui/dom" "^1.2.1" + +"@floating-ui/react-dom@^2.0.0": + version "2.0.2" + resolved "https://registry.yarnpkg.com/@floating-ui/react-dom/-/react-dom-2.0.2.tgz#fab244d64db08e6bed7be4b5fcce65315ef44d20" + integrity sha512-5qhlDvjaLmAst/rKb3VdlCinwTF4EYMiVxuuc/HVUjs46W0zgtbMmAZ1UTsDrRTxRmUEzl92mOtWbeeXL26lSQ== + dependencies: + "@floating-ui/dom" "^1.5.1" + +"@floating-ui/react@^0.19.1": + version "0.19.2" + resolved "https://registry.yarnpkg.com/@floating-ui/react/-/react-0.19.2.tgz#c6e4d2097ed0dca665a7c042ddf9cdecc95e9412" + integrity sha512-JyNk4A0Ezirq8FlXECvRtQOX/iBe5Ize0W/pLkrZjfHW9GUV7Xnq6zm6fyZuQzaHHqEnVizmvlA96e1/CkZv+w== + dependencies: + "@floating-ui/react-dom" "^1.3.0" + aria-hidden "^1.1.3" + tabbable "^6.0.1" + +"@floating-ui/utils@^0.1.1": + version "0.1.1" + resolved "https://registry.yarnpkg.com/@floating-ui/utils/-/utils-0.1.1.tgz#1a5b1959a528e374e8037c4396c3e825d6cf4a83" + integrity sha512-m0G6wlnhm/AX0H12IOWtK8gASEMffnX08RtKkCgTdHb9JpHKGloI7icFfLg9ZmQeavcvR0PKmzxClyuFPSjKWw== + +"@grpc/grpc-js@^1.9.2": + version "1.9.2" + resolved "https://registry.yarnpkg.com/@grpc/grpc-js/-/grpc-js-1.9.2.tgz#151148f6485eab8fb13fe53042d25f4ffa9c7d09" + integrity sha512-Lf2pUhNTaviEdEaGgjU+29qw3arX7Qd/45q66F3z1EV5hroE6wM9xSHPvjB8EY+b1RmKZgwnLWXQorC6fZ9g5g== + dependencies: + "@grpc/proto-loader" "^0.7.8" + "@types/node" ">=12.12.47" + +"@grpc/proto-loader@^0.7.8": + version "0.7.9" + resolved "https://registry.yarnpkg.com/@grpc/proto-loader/-/proto-loader-0.7.9.tgz#3ca68236f1a0d77566dafa53c715eb31d096279a" + integrity sha512-YJsOehVXzgurc+lLAxYnlSMc1p/Gu6VAvnfx0ATi2nzvr0YZcjhmZDeY8SeAKv1M7zE3aEJH0Xo9mK1iZ8GYoQ== + dependencies: + lodash.camelcase "^4.3.0" + long "^5.0.0" + protobufjs "^7.2.4" + yargs "^17.7.2" + +"@headlessui/react@^1.7.14": + version "1.7.17" + resolved "https://registry.yarnpkg.com/@headlessui/react/-/react-1.7.17.tgz#a0ec23af21b527c030967245fd99776aa7352bc6" + integrity sha512-4am+tzvkqDSSgiwrsEpGWqgGo9dz8qU5M3znCkC4PgkpY4HcCZzEDEvozltGGGHIKl9jbXbZPSH5TWn4sWJdow== + dependencies: + client-only "^0.0.1" + +"@headlessui/tailwindcss@^0.1.3": + version "0.1.3" + resolved "https://registry.yarnpkg.com/@headlessui/tailwindcss/-/tailwindcss-0.1.3.tgz#a9b8b4c2677a7ef37889708d4401c7871b2e6105" + integrity sha512-3aMdDyYZx9A15euRehpppSyQnb2gIw2s/Uccn2ELIoLQ9oDy0+9oRygNWNjXCD5Dt+w1pxo7C+XoiYvGcqA4Kg== + +"@humanwhocodes/config-array@^0.11.10": + version "0.11.11" + resolved "https://registry.yarnpkg.com/@humanwhocodes/config-array/-/config-array-0.11.11.tgz#88a04c570dbbc7dd943e4712429c3df09bc32844" + integrity sha512-N2brEuAadi0CcdeMXUkhbZB84eskAc8MEX1By6qEchoVywSgXPIjou4rYsl0V3Hj0ZnuGycGCjdNgockbzeWNA== + dependencies: + "@humanwhocodes/object-schema" "^1.2.1" + debug "^4.1.1" + minimatch "^3.0.5" + +"@humanwhocodes/module-importer@^1.0.1": + version "1.0.1" + resolved "https://registry.yarnpkg.com/@humanwhocodes/module-importer/-/module-importer-1.0.1.tgz#af5b2691a22b44be847b0ca81641c5fb6ad0172c" + integrity sha512-bxveV4V8v5Yb4ncFTT3rPSgZBOpCkjfK0y4oVVVJwIuDVBRMDXrPyXRL988i5ap9m9bnyEEjWfm5WkBmtffLfA== + +"@humanwhocodes/object-schema@^1.2.1": + version "1.2.1" + resolved "https://registry.yarnpkg.com/@humanwhocodes/object-schema/-/object-schema-1.2.1.tgz#b520529ec21d8e5945a1851dfd1c32e94e39ff45" + integrity sha512-ZnQMnLV4e7hDlUvw8H+U8ASL02SS2Gn6+9Ac3wGGLIe7+je2AeAOxPY+izIPJDfFDb7eDjev0Us8MO1iFRN8hA== + +"@isaacs/cliui@^8.0.2": + version "8.0.2" + resolved "https://registry.yarnpkg.com/@isaacs/cliui/-/cliui-8.0.2.tgz#b37667b7bc181c168782259bab42474fbf52b550" + integrity sha512-O8jcjabXaleOG9DQ0+ARXWZBTfnP4WNAqzuiJK7ll44AmxGKv/J2M4TPjxjY3znBCfvBXFzucm1twdyFybFqEA== + dependencies: + string-width "^5.1.2" + string-width-cjs "npm:string-width@^4.2.0" + strip-ansi "^7.0.1" + strip-ansi-cjs "npm:strip-ansi@^6.0.1" + wrap-ansi "^8.1.0" + wrap-ansi-cjs "npm:wrap-ansi@^7.0.0" + +"@istanbuljs/load-nyc-config@^1.0.0": + version "1.1.0" + resolved "https://registry.yarnpkg.com/@istanbuljs/load-nyc-config/-/load-nyc-config-1.1.0.tgz#fd3db1d59ecf7cf121e80650bb86712f9b55eced" + integrity sha512-VjeHSlIzpv/NyD3N0YuHfXOPDIixcA1q2ZV98wsMqcYlPmv2n3Yb2lYP9XMElnaFVXg5A7YLTeLu6V84uQDjmQ== + dependencies: + camelcase "^5.3.1" + find-up "^4.1.0" + get-package-type "^0.1.0" + js-yaml "^3.13.1" + resolve-from "^5.0.0" + +"@istanbuljs/schema@^0.1.2", "@istanbuljs/schema@^0.1.3": + version "0.1.3" + resolved "https://registry.yarnpkg.com/@istanbuljs/schema/-/schema-0.1.3.tgz#e45e384e4b8ec16bce2fd903af78450f6bf7ec98" + integrity sha512-ZXRY4jNvVgSVQ8DL3LTcakaAtXwTVUxE81hslsyD2AtoXW/wVob10HkOJ1X/pAlcI7D+2YoZKg5do8G/w6RYgA== + +"@jest/schemas@^29.6.3": + version "29.6.3" + resolved "https://registry.yarnpkg.com/@jest/schemas/-/schemas-29.6.3.tgz#430b5ce8a4e0044a7e3819663305a7b3091c8e03" + integrity sha512-mo5j5X+jIZmJQveBKeS/clAueipV7KgiX1vMgCxam1RNYiqE1w62n0/tJJnHtjW8ZHcQco5gY85jA3mi0L+nSA== + dependencies: + "@sinclair/typebox" "^0.27.8" + +"@jest/transform@^29.3.1": + version "29.6.4" + resolved "https://registry.yarnpkg.com/@jest/transform/-/transform-29.6.4.tgz#a6bc799ef597c5d85b2e65a11fd96b6b239bab5a" + integrity sha512-8thgRSiXUqtr/pPGY/OsyHuMjGyhVnWrFAwoxmIemlBuiMyU1WFs0tXoNxzcr4A4uErs/ABre76SGmrr5ab/AA== + dependencies: + "@babel/core" "^7.11.6" + "@jest/types" "^29.6.3" + "@jridgewell/trace-mapping" "^0.3.18" + babel-plugin-istanbul "^6.1.1" + chalk "^4.0.0" + convert-source-map "^2.0.0" + fast-json-stable-stringify "^2.1.0" + graceful-fs "^4.2.9" + jest-haste-map "^29.6.4" + jest-regex-util "^29.6.3" + jest-util "^29.6.3" + micromatch "^4.0.4" + pirates "^4.0.4" + slash "^3.0.0" + write-file-atomic "^4.0.2" + +"@jest/types@^27.5.1": + version "27.5.1" + resolved "https://registry.yarnpkg.com/@jest/types/-/types-27.5.1.tgz#3c79ec4a8ba61c170bf937bcf9e98a9df175ec80" + integrity sha512-Cx46iJ9QpwQTjIdq5VJu2QTMMs3QlEjI0x1QbBP5W1+nMzyc2XmimiRR/CbX9TO0cPTeUlxWMOu8mslYsJ8DEw== + dependencies: + "@types/istanbul-lib-coverage" "^2.0.0" + "@types/istanbul-reports" "^3.0.0" + "@types/node" "*" + "@types/yargs" "^16.0.0" + chalk "^4.0.0" + +"@jest/types@^29.6.3": + version "29.6.3" + resolved "https://registry.yarnpkg.com/@jest/types/-/types-29.6.3.tgz#1131f8cf634e7e84c5e77bab12f052af585fba59" + integrity sha512-u3UPsIilWKOM3F9CXtrG8LEJmNxwoCQC/XVj4IKYXvvpx7QIi/Kg1LI5uDmDpKlac62NUtX7eLjRh+jVZcLOzw== + dependencies: + "@jest/schemas" "^29.6.3" + "@types/istanbul-lib-coverage" "^2.0.0" + "@types/istanbul-reports" "^3.0.0" + "@types/node" "*" + "@types/yargs" "^17.0.8" + chalk "^4.0.0" + +"@jridgewell/gen-mapping@^0.3.0", "@jridgewell/gen-mapping@^0.3.2": + version "0.3.3" + resolved "https://registry.yarnpkg.com/@jridgewell/gen-mapping/-/gen-mapping-0.3.3.tgz#7e02e6eb5df901aaedb08514203b096614024098" + integrity sha512-HLhSWOLRi875zjjMG/r+Nv0oCW8umGb0BgEhyX3dDX3egwZtB8PqLnjz3yedt8R5StBrzcg4aBpnh8UA9D1BoQ== + dependencies: + "@jridgewell/set-array" "^1.0.1" + "@jridgewell/sourcemap-codec" "^1.4.10" + "@jridgewell/trace-mapping" "^0.3.9" + +"@jridgewell/resolve-uri@^3.1.0": + version "3.1.1" + resolved "https://registry.yarnpkg.com/@jridgewell/resolve-uri/-/resolve-uri-3.1.1.tgz#c08679063f279615a3326583ba3a90d1d82cc721" + integrity sha512-dSYZh7HhCDtCKm4QakX0xFpsRDqjjtZf/kjI/v3T3Nwt5r8/qz/M19F9ySyOqU94SXBmeG9ttTul+YnR4LOxFA== + +"@jridgewell/set-array@^1.0.1": + version "1.1.2" + resolved "https://registry.yarnpkg.com/@jridgewell/set-array/-/set-array-1.1.2.tgz#7c6cf998d6d20b914c0a55a91ae928ff25965e72" + integrity sha512-xnkseuNADM0gt2bs+BvhO0p78Mk762YnZdsuzFV018NoG1Sj1SCQvpSqa7XUaTam5vAGasABV9qXASMKnFMwMw== + +"@jridgewell/source-map@^0.3.3": + version "0.3.5" + resolved "https://registry.yarnpkg.com/@jridgewell/source-map/-/source-map-0.3.5.tgz#a3bb4d5c6825aab0d281268f47f6ad5853431e91" + integrity sha512-UTYAUj/wviwdsMfzoSJspJxbkH5o1snzwX0//0ENX1u/55kkZZkcTZP6u9bwKGkv+dkk9at4m1Cpt0uY80kcpQ== + dependencies: + "@jridgewell/gen-mapping" "^0.3.0" + "@jridgewell/trace-mapping" "^0.3.9" + +"@jridgewell/sourcemap-codec@^1.4.10", "@jridgewell/sourcemap-codec@^1.4.14": + version "1.4.15" + resolved "https://registry.yarnpkg.com/@jridgewell/sourcemap-codec/-/sourcemap-codec-1.4.15.tgz#d7c6e6755c78567a951e04ab52ef0fd26de59f32" + integrity sha512-eF2rxCRulEKXHTRiDrDy6erMYWqNw4LPdQ8UQA4huuxaQsVeRPFl2oM8oDGxMFhJUWZf9McpLtJasDDZb/Bpeg== + +"@jridgewell/trace-mapping@^0.3.12", "@jridgewell/trace-mapping@^0.3.17", "@jridgewell/trace-mapping@^0.3.18", "@jridgewell/trace-mapping@^0.3.9": + version "0.3.19" + resolved "https://registry.yarnpkg.com/@jridgewell/trace-mapping/-/trace-mapping-0.3.19.tgz#f8a3249862f91be48d3127c3cfe992f79b4b8811" + integrity sha512-kf37QtfW+Hwx/buWGMPcR60iF9ziHa6r/CZJIHbmcm4+0qrXiVdxegAH0F6yddEVQ7zdkjcGCgCzUu+BcbhQxw== + dependencies: + "@jridgewell/resolve-uri" "^3.1.0" + "@jridgewell/sourcemap-codec" "^1.4.14" + +"@juggle/resize-observer@^3.3.1": + version "3.4.0" + resolved "https://registry.yarnpkg.com/@juggle/resize-observer/-/resize-observer-3.4.0.tgz#08d6c5e20cf7e4cc02fd181c4b0c225cd31dbb60" + integrity sha512-dfLbk+PwWvFzSxwk3n5ySL0hfBog779o8h68wK/7/APo/7cgyWp5jcXockbxdk5kFRkbeXWm4Fbi9FrdN381sA== + +"@mdx-js/react@^2.1.5": + version "2.3.0" + resolved "https://registry.yarnpkg.com/@mdx-js/react/-/react-2.3.0.tgz#4208bd6d70f0d0831def28ef28c26149b03180b3" + integrity sha512-zQH//gdOmuu7nt2oJR29vFhDv88oGPmVw6BggmrHeMI+xgEkp1B2dX9/bMBSYtK0dyLX/aOmesKS09g222K1/g== + dependencies: + "@types/mdx" "^2.0.0" + "@types/react" ">=16" + +"@ndelangen/get-tarball@^3.0.7": + version "3.0.9" + resolved "https://registry.yarnpkg.com/@ndelangen/get-tarball/-/get-tarball-3.0.9.tgz#727ff4454e65f34707e742a59e5e6b1f525d8964" + integrity sha512-9JKTEik4vq+yGosHYhZ1tiH/3WpUS0Nh0kej4Agndhox8pAdWhEx5knFVRcb/ya9knCRCs1rPxNrSXTDdfVqpA== + dependencies: + gunzip-maybe "^1.4.2" + pump "^3.0.0" + tar-fs "^2.1.1" + +"@next/env@13.4.16": + version "13.4.16" + resolved "https://registry.yarnpkg.com/@next/env/-/env-13.4.16.tgz#382b565b35a2a69bd0e6b50f74c7b95f0c4b1097" + integrity sha512-pCU0sJBqdfKP9mwDadxvZd+eLz3fZrTlmmDHY12Hdpl3DD0vy8ou5HWKVfG0zZS6tqhL4wnQqRbspdY5nqa7MA== + +"@next/eslint-plugin-next@13.4.19": + version "13.4.19" + resolved "https://registry.yarnpkg.com/@next/eslint-plugin-next/-/eslint-plugin-next-13.4.19.tgz#93d130c37b47fd120f6d111aee36a60611148df1" + integrity sha512-N/O+zGb6wZQdwu6atMZHbR7T9Np5SUFUjZqCbj0sXm+MwQO35M8TazVB4otm87GkXYs2l6OPwARd3/PUWhZBVQ== + dependencies: + glob "7.1.7" + +"@next/swc-darwin-arm64@13.4.16": + version "13.4.16" + resolved "https://registry.yarnpkg.com/@next/swc-darwin-arm64/-/swc-darwin-arm64-13.4.16.tgz#ed6a342f95e5f21213fdadbceb65b40ae678cee0" + integrity sha512-Rl6i1uUq0ciRa3VfEpw6GnWAJTSKo9oM2OrkGXPsm7rMxdd2FR5NkKc0C9xzFCI4+QtmBviWBdF2m3ur3Nqstw== + +"@next/swc-darwin-x64@13.4.16": + version "13.4.16" + resolved "https://registry.yarnpkg.com/@next/swc-darwin-x64/-/swc-darwin-x64-13.4.16.tgz#36c16066a1a3ef8211e84a6a5d72bef15826b291" + integrity sha512-o1vIKYbZORyDmTrPV1hApt9NLyWrS5vr2p5hhLGpOnkBY1cz6DAXjv8Lgan8t6X87+83F0EUDlu7klN8ieZ06A== + +"@next/swc-linux-arm64-gnu@13.4.16": + version "13.4.16" + resolved "https://registry.yarnpkg.com/@next/swc-linux-arm64-gnu/-/swc-linux-arm64-gnu-13.4.16.tgz#a5b5500737f07e3aa7f184014d8df7973420df26" + integrity sha512-JRyAl8lCfyTng4zoOmE6hNI2f1MFUr7JyTYCHl1RxX42H4a5LMwJhDVQ7a9tmDZ/yj+0hpBn+Aan+d6lA3v0UQ== + +"@next/swc-linux-arm64-musl@13.4.16": + version "13.4.16" + resolved "https://registry.yarnpkg.com/@next/swc-linux-arm64-musl/-/swc-linux-arm64-musl-13.4.16.tgz#381b7662c5b10ed5750dce41dd57841aa0713e77" + integrity sha512-9gqVqNzUMWbUDgDiND18xoUqhwSm2gmksqXgCU0qaOKt6oAjWz8cWYjgpPVD0WICKFylEY/gvPEP1fMZDVFZ/g== + +"@next/swc-linux-x64-gnu@13.4.16": + version "13.4.16" + resolved "https://registry.yarnpkg.com/@next/swc-linux-x64-gnu/-/swc-linux-x64-gnu-13.4.16.tgz#6e0b0eab1c316506950aeb4a09a5ea5c38edabe7" + integrity sha512-KcQGwchAKmZVPa8i5PLTxvTs1/rcFnSltfpTm803Tr/BtBV3AxCkHLfhtoyVtVzx/kl/oue8oS+DSmbepQKwhw== + +"@next/swc-linux-x64-musl@13.4.16": + version "13.4.16" + resolved "https://registry.yarnpkg.com/@next/swc-linux-x64-musl/-/swc-linux-x64-musl-13.4.16.tgz#36b84e4509168a5cadf9dfd728c239002d4311fe" + integrity sha512-2RbMZNxYnJmW8EPHVBsGZPq5zqWAyBOc/YFxq/jIQ/Yn3RMFZ1dZVCjtIcsiaKmgh7mjA/W0ApbumutHNxRqqQ== + +"@next/swc-win32-arm64-msvc@13.4.16": + version "13.4.16" + resolved "https://registry.yarnpkg.com/@next/swc-win32-arm64-msvc/-/swc-win32-arm64-msvc-13.4.16.tgz#52d36f909ccdefa2761617b6d4e9ae65f99880a9" + integrity sha512-thDcGonELN7edUKzjzlHrdoKkm7y8IAdItQpRvvMxNUXa4d9r0ElofhTZj5emR7AiXft17hpen+QAkcWpqG7Jg== + +"@next/swc-win32-ia32-msvc@13.4.16": + version "13.4.16" + resolved "https://registry.yarnpkg.com/@next/swc-win32-ia32-msvc/-/swc-win32-ia32-msvc-13.4.16.tgz#a9cb0556d19c33fbb39ac9bef195fd490d6c7673" + integrity sha512-f7SE1Mo4JAchUWl0LQsbtySR9xCa+x55C0taetjUApKtcLR3AgAjASrrP+oE1inmLmw573qRnE1eZN8YJfEBQw== + +"@next/swc-win32-x64-msvc@13.4.16": + version "13.4.16" + resolved "https://registry.yarnpkg.com/@next/swc-win32-x64-msvc/-/swc-win32-x64-msvc-13.4.16.tgz#79a151d94583e03992c80df3d3e7f7686390ddac" + integrity sha512-WamDZm1M/OEM4QLce3lOmD1XdLEl37zYZwlmOLhmF7qYJ2G6oYm9+ejZVv+LakQIsIuXhSpVlOvrxIAHqwRkPQ== + +"@nicolo-ribaudo/chokidar-2@2.1.8-no-fsevents.3": + version "2.1.8-no-fsevents.3" + resolved "https://registry.yarnpkg.com/@nicolo-ribaudo/chokidar-2/-/chokidar-2-2.1.8-no-fsevents.3.tgz#323d72dd25103d0c4fbdce89dadf574a787b1f9b" + integrity sha512-s88O1aVtXftvp5bCPB7WnmXc5IwOZZ7YPuwNPt+GtOOXpPvad1LfbmjYv+qII7zP6RU2QGnqve27dnLycEnyEQ== + +"@nodelib/fs.scandir@2.1.5": + version "2.1.5" + resolved "https://registry.yarnpkg.com/@nodelib/fs.scandir/-/fs.scandir-2.1.5.tgz#7619c2eb21b25483f6d167548b4cfd5a7488c3d5" + integrity sha512-vq24Bq3ym5HEQm2NKCr3yXDwjc7vTsEThRDnkp2DK9p1uqLR+DHurm/NOTo0KG7HYHU7eppKZj3MyqYuMBf62g== + dependencies: + "@nodelib/fs.stat" "2.0.5" + run-parallel "^1.1.9" + +"@nodelib/fs.stat@2.0.5", "@nodelib/fs.stat@^2.0.2": + version "2.0.5" + resolved "https://registry.yarnpkg.com/@nodelib/fs.stat/-/fs.stat-2.0.5.tgz#5bd262af94e9d25bd1e71b05deed44876a222e8b" + integrity sha512-RkhPPp2zrqDAQA/2jNhnztcPAlv64XdhIp7a7454A5ovI7Bukxgt7MX7udwAu3zg1DcpPU0rz3VV1SeaqvY4+A== + +"@nodelib/fs.walk@^1.2.3", "@nodelib/fs.walk@^1.2.8": + version "1.2.8" + resolved "https://registry.yarnpkg.com/@nodelib/fs.walk/-/fs.walk-1.2.8.tgz#e95737e8bb6746ddedf69c556953494f196fe69a" + integrity sha512-oGB+UxlgWcgQkgwo8GcEGwemoTFt3FIO9ababBmaGwXIoBKZ+GTy0pP185beGg7Llih/NSHSV2XAs1lnznocSg== + dependencies: + "@nodelib/fs.scandir" "2.1.5" + fastq "^1.6.0" + +"@pkgjs/parseargs@^0.11.0": + version "0.11.0" + resolved "https://registry.yarnpkg.com/@pkgjs/parseargs/-/parseargs-0.11.0.tgz#a77ea742fab25775145434eb1d2328cf5013ac33" + integrity sha512-+1VkjdD0QBLPodGrJUeqarH8VAIvQODIbwh9XpP5Syisf7YoQgsJKPNFoqqLQlu+VQ/tVSshMR6loPMn8U+dPg== + +"@pmmmwh/react-refresh-webpack-plugin@^0.5.5": + version "0.5.11" + resolved "https://registry.yarnpkg.com/@pmmmwh/react-refresh-webpack-plugin/-/react-refresh-webpack-plugin-0.5.11.tgz#7c2268cedaa0644d677e8c4f377bc8fb304f714a" + integrity sha512-7j/6vdTym0+qZ6u4XbSAxrWBGYSdCfTzySkj7WAFgDLmSyWlOrWvpyzxlFh5jtw9dn0oL/jtW+06XfFiisN3JQ== + dependencies: + ansi-html-community "^0.0.8" + common-path-prefix "^3.0.0" + core-js-pure "^3.23.3" + error-stack-parser "^2.0.6" + find-up "^5.0.0" + html-entities "^2.1.0" + loader-utils "^2.0.4" + schema-utils "^3.0.0" + source-map "^0.7.3" + +"@protobufjs/aspromise@^1.1.1", "@protobufjs/aspromise@^1.1.2": + version "1.1.2" + resolved "https://registry.yarnpkg.com/@protobufjs/aspromise/-/aspromise-1.1.2.tgz#9b8b0cc663d669a7d8f6f5d0893a14d348f30fbf" + integrity sha512-j+gKExEuLmKwvz3OgROXtrJ2UG2x8Ch2YZUxahh+s1F2HZ+wAceUNLkvy6zKCPVRkU++ZWQrdxsUeQXmcg4uoQ== + +"@protobufjs/base64@^1.1.2": + version "1.1.2" + resolved "https://registry.yarnpkg.com/@protobufjs/base64/-/base64-1.1.2.tgz#4c85730e59b9a1f1f349047dbf24296034bb2735" + integrity sha512-AZkcAA5vnN/v4PDqKyMR5lx7hZttPDgClv83E//FMNhR2TMcLUhfRUBHCmSl0oi9zMgDDqRUJkSxO3wm85+XLg== + +"@protobufjs/codegen@^2.0.4": + version "2.0.4" + resolved "https://registry.yarnpkg.com/@protobufjs/codegen/-/codegen-2.0.4.tgz#7ef37f0d010fb028ad1ad59722e506d9262815cb" + integrity sha512-YyFaikqM5sH0ziFZCN3xDC7zeGaB/d0IUb9CATugHWbd1FRFwWwt4ld4OYMPWu5a3Xe01mGAULCdqhMlPl29Jg== + +"@protobufjs/eventemitter@^1.1.0": + version "1.1.0" + resolved "https://registry.yarnpkg.com/@protobufjs/eventemitter/-/eventemitter-1.1.0.tgz#355cbc98bafad5978f9ed095f397621f1d066b70" + integrity sha512-j9ednRT81vYJ9OfVuXG6ERSTdEL1xVsNgqpkxMsbIabzSo3goCjDIveeGv5d03om39ML71RdmrGNjG5SReBP/Q== + +"@protobufjs/fetch@^1.1.0": + version "1.1.0" + resolved "https://registry.yarnpkg.com/@protobufjs/fetch/-/fetch-1.1.0.tgz#ba99fb598614af65700c1619ff06d454b0d84c45" + integrity sha512-lljVXpqXebpsijW71PZaCYeIcE5on1w5DlQy5WH6GLbFryLUrBD4932W/E2BSpfRJWseIL4v/KPgBFxDOIdKpQ== + dependencies: + "@protobufjs/aspromise" "^1.1.1" + "@protobufjs/inquire" "^1.1.0" + +"@protobufjs/float@^1.0.2": + version "1.0.2" + resolved "https://registry.yarnpkg.com/@protobufjs/float/-/float-1.0.2.tgz#5e9e1abdcb73fc0a7cb8b291df78c8cbd97b87d1" + integrity sha512-Ddb+kVXlXst9d+R9PfTIxh1EdNkgoRe5tOX6t01f1lYWOvJnSPDBlG241QLzcyPdoNTsblLUdujGSE4RzrTZGQ== + +"@protobufjs/inquire@^1.1.0": + version "1.1.0" + resolved "https://registry.yarnpkg.com/@protobufjs/inquire/-/inquire-1.1.0.tgz#ff200e3e7cf2429e2dcafc1140828e8cc638f089" + integrity sha512-kdSefcPdruJiFMVSbn801t4vFK7KB/5gd2fYvrxhuJYg8ILrmn9SKSX2tZdV6V+ksulWqS7aXjBcRXl3wHoD9Q== + +"@protobufjs/path@^1.1.2": + version "1.1.2" + resolved "https://registry.yarnpkg.com/@protobufjs/path/-/path-1.1.2.tgz#6cc2b20c5c9ad6ad0dccfd21ca7673d8d7fbf68d" + integrity sha512-6JOcJ5Tm08dOHAbdR3GrvP+yUUfkjG5ePsHYczMFLq3ZmMkAD98cDgcT2iA1lJ9NVwFd4tH/iSSoe44YWkltEA== + +"@protobufjs/pool@^1.1.0": + version "1.1.0" + resolved "https://registry.yarnpkg.com/@protobufjs/pool/-/pool-1.1.0.tgz#09fd15f2d6d3abfa9b65bc366506d6ad7846ff54" + integrity sha512-0kELaGSIDBKvcgS4zkjz1PeddatrjYcmMWOlAuAPwAeccUrPHdUqo/J6LiymHHEiJT5NrF1UVwxY14f+fy4WQw== + +"@protobufjs/utf8@^1.1.0": + version "1.1.0" + resolved "https://registry.yarnpkg.com/@protobufjs/utf8/-/utf8-1.1.0.tgz#a777360b5b39a1a2e5106f8e858f2fd2d060c570" + integrity sha512-Vvn3zZrhQZkkBE8LSuW3em98c0FwgO4nxzv6OdSxPKJIEKY2bGbHn+mhGIPerzI4twdxaP8/0+06HBpwf345Lw== + +"@radix-ui/number@1.0.1": + version "1.0.1" + resolved "https://registry.yarnpkg.com/@radix-ui/number/-/number-1.0.1.tgz#644161a3557f46ed38a042acf4a770e826021674" + integrity sha512-T5gIdVO2mmPW3NNhjNgEP3cqMXjXL9UbO0BzWcXfvdBs+BohbQxvd/K5hSVKmn9/lbTdsQVKbUcP5WLCwvUbBg== + dependencies: + "@babel/runtime" "^7.13.10" + +"@radix-ui/primitive@1.0.1": + version "1.0.1" + resolved "https://registry.yarnpkg.com/@radix-ui/primitive/-/primitive-1.0.1.tgz#e46f9958b35d10e9f6dc71c497305c22e3e55dbd" + integrity sha512-yQ8oGX2GVsEYMWGxcovu1uGWPCxV5BFfeeYxqPmuAzUyLT9qmaMXSAhXpb0WrspIeqYzdJpkh2vHModJPgRIaw== + dependencies: + "@babel/runtime" "^7.13.10" + +"@radix-ui/react-arrow@1.0.3": + version "1.0.3" + resolved "https://registry.yarnpkg.com/@radix-ui/react-arrow/-/react-arrow-1.0.3.tgz#c24f7968996ed934d57fe6cde5d6ec7266e1d25d" + integrity sha512-wSP+pHsB/jQRaL6voubsQ/ZlrGBHHrOjmBnr19hxYgtS0WvAFwZhK2WP/YY5yF9uKECCEEDGxuLxq1NBK51wFA== + dependencies: + "@babel/runtime" "^7.13.10" + "@radix-ui/react-primitive" "1.0.3" + +"@radix-ui/react-checkbox@^1.0.4": + version "1.0.4" + resolved "https://registry.yarnpkg.com/@radix-ui/react-checkbox/-/react-checkbox-1.0.4.tgz#98f22c38d5010dd6df4c5744cac74087e3275f4b" + integrity sha512-CBuGQa52aAYnADZVt/KBQzXrwx6TqnlwtcIPGtVt5JkkzQwMOLJjPukimhfKEr4GQNd43C+djUh5Ikopj8pSLg== + dependencies: + "@babel/runtime" "^7.13.10" + "@radix-ui/primitive" "1.0.1" + "@radix-ui/react-compose-refs" "1.0.1" + "@radix-ui/react-context" "1.0.1" + "@radix-ui/react-presence" "1.0.1" + "@radix-ui/react-primitive" "1.0.3" + "@radix-ui/react-use-controllable-state" "1.0.1" + "@radix-ui/react-use-previous" "1.0.1" + "@radix-ui/react-use-size" "1.0.1" + +"@radix-ui/react-collection@1.0.3": + version "1.0.3" + resolved "https://registry.yarnpkg.com/@radix-ui/react-collection/-/react-collection-1.0.3.tgz#9595a66e09026187524a36c6e7e9c7d286469159" + integrity sha512-3SzW+0PW7yBBoQlT8wNcGtaxaD0XSu0uLUFgrtHY08Acx05TaHaOmVLR73c0j/cqpDy53KBMO7s0dx2wmOIDIA== + dependencies: + "@babel/runtime" "^7.13.10" + "@radix-ui/react-compose-refs" "1.0.1" + "@radix-ui/react-context" "1.0.1" + "@radix-ui/react-primitive" "1.0.3" + "@radix-ui/react-slot" "1.0.2" + +"@radix-ui/react-compose-refs@1.0.1": + version "1.0.1" + resolved "https://registry.yarnpkg.com/@radix-ui/react-compose-refs/-/react-compose-refs-1.0.1.tgz#7ed868b66946aa6030e580b1ffca386dd4d21989" + integrity sha512-fDSBgd44FKHa1FRMU59qBMPFcl2PZE+2nmqunj+BWFyYYjnhIDWL2ItDs3rrbJDQOtzt5nIebLCQc4QRfz6LJw== + dependencies: + "@babel/runtime" "^7.13.10" + +"@radix-ui/react-context@1.0.1": + version "1.0.1" + resolved "https://registry.yarnpkg.com/@radix-ui/react-context/-/react-context-1.0.1.tgz#fe46e67c96b240de59187dcb7a1a50ce3e2ec00c" + integrity sha512-ebbrdFoYTcuZ0v4wG5tedGnp9tzcV8awzsxYph7gXUyvnNLuTIcCk1q17JEbnVhXAKG9oX3KtchwiMIAYp9NLg== + dependencies: + "@babel/runtime" "^7.13.10" + +"@radix-ui/react-dialog@^1.0.4": + version "1.0.4" + resolved "https://registry.yarnpkg.com/@radix-ui/react-dialog/-/react-dialog-1.0.4.tgz#06bce6c16bb93eb36d7a8589e665a20f4c1c52c1" + integrity sha512-hJtRy/jPULGQZceSAP2Re6/4NpKo8im6V8P2hUqZsdFiSL8l35kYsw3qbRI6Ay5mQd2+wlLqje770eq+RJ3yZg== + dependencies: + "@babel/runtime" "^7.13.10" + "@radix-ui/primitive" "1.0.1" + "@radix-ui/react-compose-refs" "1.0.1" + "@radix-ui/react-context" "1.0.1" + "@radix-ui/react-dismissable-layer" "1.0.4" + "@radix-ui/react-focus-guards" "1.0.1" + "@radix-ui/react-focus-scope" "1.0.3" + "@radix-ui/react-id" "1.0.1" + "@radix-ui/react-portal" "1.0.3" + "@radix-ui/react-presence" "1.0.1" + "@radix-ui/react-primitive" "1.0.3" + "@radix-ui/react-slot" "1.0.2" + "@radix-ui/react-use-controllable-state" "1.0.1" + aria-hidden "^1.1.1" + react-remove-scroll "2.5.5" + +"@radix-ui/react-direction@1.0.1": + version "1.0.1" + resolved "https://registry.yarnpkg.com/@radix-ui/react-direction/-/react-direction-1.0.1.tgz#9cb61bf2ccf568f3421422d182637b7f47596c9b" + integrity sha512-RXcvnXgyvYvBEOhCBuddKecVkoMiI10Jcm5cTI7abJRAHYfFxeu+FBQs/DvdxSYucxR5mna0dNsL6QFlds5TMA== + dependencies: + "@babel/runtime" "^7.13.10" + +"@radix-ui/react-dismissable-layer@1.0.4": + version "1.0.4" + resolved "https://registry.yarnpkg.com/@radix-ui/react-dismissable-layer/-/react-dismissable-layer-1.0.4.tgz#883a48f5f938fa679427aa17fcba70c5494c6978" + integrity sha512-7UpBa/RKMoHJYjie1gkF1DlK8l1fdU/VKDpoS3rCCo8YBJR294GwcEHyxHw72yvphJ7ld0AXEcSLAzY2F/WyCg== + dependencies: + "@babel/runtime" "^7.13.10" + "@radix-ui/primitive" "1.0.1" + "@radix-ui/react-compose-refs" "1.0.1" + "@radix-ui/react-primitive" "1.0.3" + "@radix-ui/react-use-callback-ref" "1.0.1" + "@radix-ui/react-use-escape-keydown" "1.0.3" + +"@radix-ui/react-focus-guards@1.0.1": + version "1.0.1" + resolved "https://registry.yarnpkg.com/@radix-ui/react-focus-guards/-/react-focus-guards-1.0.1.tgz#1ea7e32092216b946397866199d892f71f7f98ad" + integrity sha512-Rect2dWbQ8waGzhMavsIbmSVCgYxkXLxxR3ZvCX79JOglzdEy4JXMb98lq4hPxUbLr77nP0UOGf4rcMU+s1pUA== + dependencies: + "@babel/runtime" "^7.13.10" + +"@radix-ui/react-focus-scope@1.0.3": + version "1.0.3" + resolved "https://registry.yarnpkg.com/@radix-ui/react-focus-scope/-/react-focus-scope-1.0.3.tgz#9c2e8d4ed1189a1d419ee61edd5c1828726472f9" + integrity sha512-upXdPfqI4islj2CslyfUBNlaJCPybbqRHAi1KER7Isel9Q2AtSJ0zRBZv8mWQiFXD2nyAJ4BhC3yXgZ6kMBSrQ== + dependencies: + "@babel/runtime" "^7.13.10" + "@radix-ui/react-compose-refs" "1.0.1" + "@radix-ui/react-primitive" "1.0.3" + "@radix-ui/react-use-callback-ref" "1.0.1" + +"@radix-ui/react-form@^0.0.3": + version "0.0.3" + resolved "https://registry.yarnpkg.com/@radix-ui/react-form/-/react-form-0.0.3.tgz#328e7163e723ccc748459d66a2d685d7b4f85d5a" + integrity sha512-kgE+Z/haV6fxE5WqIXj05KkaXa3OkZASoTDy25yX2EIp/x0c54rOH/vFr5nOZTg7n7T1z8bSyXmiVIFP9bbhPQ== + dependencies: + "@babel/runtime" "^7.13.10" + "@radix-ui/primitive" "1.0.1" + "@radix-ui/react-compose-refs" "1.0.1" + "@radix-ui/react-context" "1.0.1" + "@radix-ui/react-id" "1.0.1" + "@radix-ui/react-label" "2.0.2" + "@radix-ui/react-primitive" "1.0.3" + +"@radix-ui/react-id@1.0.1": + version "1.0.1" + resolved "https://registry.yarnpkg.com/@radix-ui/react-id/-/react-id-1.0.1.tgz#73cdc181f650e4df24f0b6a5b7aa426b912c88c0" + integrity sha512-tI7sT/kqYp8p96yGWY1OAnLHrqDgzHefRBKQ2YAkBS5ja7QLcZ9Z/uY7bEjPUatf8RomoXM8/1sMj1IJaE5UzQ== + dependencies: + "@babel/runtime" "^7.13.10" + "@radix-ui/react-use-layout-effect" "1.0.1" + +"@radix-ui/react-label@2.0.2": + version "2.0.2" + resolved "https://registry.yarnpkg.com/@radix-ui/react-label/-/react-label-2.0.2.tgz#9c72f1d334aac996fdc27b48a8bdddd82108fb6d" + integrity sha512-N5ehvlM7qoTLx7nWPodsPYPgMzA5WM8zZChQg8nyFJKnDO5WHdba1vv5/H6IO5LtJMfD2Q3wh1qHFGNtK0w3bQ== + dependencies: + "@babel/runtime" "^7.13.10" + "@radix-ui/react-primitive" "1.0.3" + +"@radix-ui/react-popper@1.1.2": + version "1.1.2" + resolved "https://registry.yarnpkg.com/@radix-ui/react-popper/-/react-popper-1.1.2.tgz#4c0b96fcd188dc1f334e02dba2d538973ad842e9" + integrity sha512-1CnGGfFi/bbqtJZZ0P/NQY20xdG3E0LALJaLUEoKwPLwl6PPPfbeiCqMVQnhoFRAxjJj4RpBRJzDmUgsex2tSg== + dependencies: + "@babel/runtime" "^7.13.10" + "@floating-ui/react-dom" "^2.0.0" + "@radix-ui/react-arrow" "1.0.3" + "@radix-ui/react-compose-refs" "1.0.1" + "@radix-ui/react-context" "1.0.1" + "@radix-ui/react-primitive" "1.0.3" + "@radix-ui/react-use-callback-ref" "1.0.1" + "@radix-ui/react-use-layout-effect" "1.0.1" + "@radix-ui/react-use-rect" "1.0.1" + "@radix-ui/react-use-size" "1.0.1" + "@radix-ui/rect" "1.0.1" + +"@radix-ui/react-portal@1.0.3": + version "1.0.3" + resolved "https://registry.yarnpkg.com/@radix-ui/react-portal/-/react-portal-1.0.3.tgz#ffb961244c8ed1b46f039e6c215a6c4d9989bda1" + integrity sha512-xLYZeHrWoPmA5mEKEfZZevoVRK/Q43GfzRXkWV6qawIWWK8t6ifIiLQdd7rmQ4Vk1bmI21XhqF9BN3jWf+phpA== + dependencies: + "@babel/runtime" "^7.13.10" + "@radix-ui/react-primitive" "1.0.3" + +"@radix-ui/react-presence@1.0.1": + version "1.0.1" + resolved "https://registry.yarnpkg.com/@radix-ui/react-presence/-/react-presence-1.0.1.tgz#491990ba913b8e2a5db1b06b203cb24b5cdef9ba" + integrity sha512-UXLW4UAbIY5ZjcvzjfRFo5gxva8QirC9hF7wRE4U5gz+TP0DbRk+//qyuAQ1McDxBt1xNMBTaciFGvEmJvAZCg== + dependencies: + "@babel/runtime" "^7.13.10" + "@radix-ui/react-compose-refs" "1.0.1" + "@radix-ui/react-use-layout-effect" "1.0.1" + +"@radix-ui/react-primitive@1.0.3": + version "1.0.3" + resolved "https://registry.yarnpkg.com/@radix-ui/react-primitive/-/react-primitive-1.0.3.tgz#d49ea0f3f0b2fe3ab1cb5667eb03e8b843b914d0" + integrity sha512-yi58uVyoAcK/Nq1inRY56ZSjKypBNKTa/1mcL8qdl6oJeEaDbOldlzrGn7P6Q3Id5d+SYNGc5AJgc4vGhjs5+g== + dependencies: + "@babel/runtime" "^7.13.10" + "@radix-ui/react-slot" "1.0.2" + +"@radix-ui/react-progress@^1.0.3": + version "1.0.3" + resolved "https://registry.yarnpkg.com/@radix-ui/react-progress/-/react-progress-1.0.3.tgz#8380272fdc64f15cbf263a294dea70a7d5d9b4fa" + integrity sha512-5G6Om/tYSxjSeEdrb1VfKkfZfn/1IlPWd731h2RfPuSbIfNUgfqAwbKfJCg/PP6nuUCTrYzalwHSpSinoWoCag== + dependencies: + "@babel/runtime" "^7.13.10" + "@radix-ui/react-context" "1.0.1" + "@radix-ui/react-primitive" "1.0.3" + +"@radix-ui/react-radio-group@^1.1.3": + version "1.1.3" + resolved "https://registry.yarnpkg.com/@radix-ui/react-radio-group/-/react-radio-group-1.1.3.tgz#3197f5dcce143bcbf961471bf89320735c0212d3" + integrity sha512-x+yELayyefNeKeTx4fjK6j99Fs6c4qKm3aY38G3swQVTN6xMpsrbigC0uHs2L//g8q4qR7qOcww8430jJmi2ag== + dependencies: + "@babel/runtime" "^7.13.10" + "@radix-ui/primitive" "1.0.1" + "@radix-ui/react-compose-refs" "1.0.1" + "@radix-ui/react-context" "1.0.1" + "@radix-ui/react-direction" "1.0.1" + "@radix-ui/react-presence" "1.0.1" + "@radix-ui/react-primitive" "1.0.3" + "@radix-ui/react-roving-focus" "1.0.4" + "@radix-ui/react-use-controllable-state" "1.0.1" + "@radix-ui/react-use-previous" "1.0.1" + "@radix-ui/react-use-size" "1.0.1" + +"@radix-ui/react-roving-focus@1.0.4": + version "1.0.4" + resolved "https://registry.yarnpkg.com/@radix-ui/react-roving-focus/-/react-roving-focus-1.0.4.tgz#e90c4a6a5f6ac09d3b8c1f5b5e81aab2f0db1974" + integrity sha512-2mUg5Mgcu001VkGy+FfzZyzbmuUWzgWkj3rvv4yu+mLw03+mTzbxZHvfcGyFp2b8EkQeMkpRQ5FiA2Vr2O6TeQ== + dependencies: + "@babel/runtime" "^7.13.10" + "@radix-ui/primitive" "1.0.1" + "@radix-ui/react-collection" "1.0.3" + "@radix-ui/react-compose-refs" "1.0.1" + "@radix-ui/react-context" "1.0.1" + "@radix-ui/react-direction" "1.0.1" + "@radix-ui/react-id" "1.0.1" + "@radix-ui/react-primitive" "1.0.3" + "@radix-ui/react-use-callback-ref" "1.0.1" + "@radix-ui/react-use-controllable-state" "1.0.1" + +"@radix-ui/react-select@^1.2.2": + version "1.2.2" + resolved "https://registry.yarnpkg.com/@radix-ui/react-select/-/react-select-1.2.2.tgz#caa981fa0d672cf3c1b2a5240135524e69b32181" + integrity sha512-zI7McXr8fNaSrUY9mZe4x/HC0jTLY9fWNhO1oLWYMQGDXuV4UCivIGTxwioSzO0ZCYX9iSLyWmAh/1TOmX3Cnw== + dependencies: + "@babel/runtime" "^7.13.10" + "@radix-ui/number" "1.0.1" + "@radix-ui/primitive" "1.0.1" + "@radix-ui/react-collection" "1.0.3" + "@radix-ui/react-compose-refs" "1.0.1" + "@radix-ui/react-context" "1.0.1" + "@radix-ui/react-direction" "1.0.1" + "@radix-ui/react-dismissable-layer" "1.0.4" + "@radix-ui/react-focus-guards" "1.0.1" + "@radix-ui/react-focus-scope" "1.0.3" + "@radix-ui/react-id" "1.0.1" + "@radix-ui/react-popper" "1.1.2" + "@radix-ui/react-portal" "1.0.3" + "@radix-ui/react-primitive" "1.0.3" + "@radix-ui/react-slot" "1.0.2" + "@radix-ui/react-use-callback-ref" "1.0.1" + "@radix-ui/react-use-controllable-state" "1.0.1" + "@radix-ui/react-use-layout-effect" "1.0.1" + "@radix-ui/react-use-previous" "1.0.1" + "@radix-ui/react-visually-hidden" "1.0.3" + aria-hidden "^1.1.1" + react-remove-scroll "2.5.5" + +"@radix-ui/react-separator@1.0.3": + version "1.0.3" + resolved "https://registry.yarnpkg.com/@radix-ui/react-separator/-/react-separator-1.0.3.tgz#be5a931a543d5726336b112f465f58585c04c8aa" + integrity sha512-itYmTy/kokS21aiV5+Z56MZB54KrhPgn6eHDKkFeOLR34HMN2s8PaN47qZZAGnvupcjxHaFZnW4pQEh0BvvVuw== + dependencies: + "@babel/runtime" "^7.13.10" + "@radix-ui/react-primitive" "1.0.3" + +"@radix-ui/react-slot@1.0.2": + version "1.0.2" + resolved "https://registry.yarnpkg.com/@radix-ui/react-slot/-/react-slot-1.0.2.tgz#a9ff4423eade67f501ffb32ec22064bc9d3099ab" + integrity sha512-YeTpuq4deV+6DusvVUW4ivBgnkHwECUu0BiN43L5UCDFgdhsRUWAghhTF5MbvNTPzmiFOx90asDSUjWuCNapwg== + dependencies: + "@babel/runtime" "^7.13.10" + "@radix-ui/react-compose-refs" "1.0.1" + +"@radix-ui/react-switch@^1.0.3": + version "1.0.3" + resolved "https://registry.yarnpkg.com/@radix-ui/react-switch/-/react-switch-1.0.3.tgz#6119f16656a9eafb4424c600fdb36efa5ec5837e" + integrity sha512-mxm87F88HyHztsI7N+ZUmEoARGkC22YVW5CaC+Byc+HRpuvCrOBPTAnXgf+tZ/7i0Sg/eOePGdMhUKhPaQEqow== + dependencies: + "@babel/runtime" "^7.13.10" + "@radix-ui/primitive" "1.0.1" + "@radix-ui/react-compose-refs" "1.0.1" + "@radix-ui/react-context" "1.0.1" + "@radix-ui/react-primitive" "1.0.3" + "@radix-ui/react-use-controllable-state" "1.0.1" + "@radix-ui/react-use-previous" "1.0.1" + "@radix-ui/react-use-size" "1.0.1" + +"@radix-ui/react-toast@^1.1.4": + version "1.1.4" + resolved "https://registry.yarnpkg.com/@radix-ui/react-toast/-/react-toast-1.1.4.tgz#9a7fc2d71700886f3292f7699c905f1e01be59e1" + integrity sha512-wf+fc8DOywrpRK3jlPlWVe+ELYGHdKDaaARJZNuUTWyWYq7+ANCFLp4rTjZ/mcGkJJQ/vZ949Zis9xxEpfq9OA== + dependencies: + "@babel/runtime" "^7.13.10" + "@radix-ui/primitive" "1.0.1" + "@radix-ui/react-collection" "1.0.3" + "@radix-ui/react-compose-refs" "1.0.1" + "@radix-ui/react-context" "1.0.1" + "@radix-ui/react-dismissable-layer" "1.0.4" + "@radix-ui/react-portal" "1.0.3" + "@radix-ui/react-presence" "1.0.1" + "@radix-ui/react-primitive" "1.0.3" + "@radix-ui/react-use-callback-ref" "1.0.1" + "@radix-ui/react-use-controllable-state" "1.0.1" + "@radix-ui/react-use-layout-effect" "1.0.1" + "@radix-ui/react-visually-hidden" "1.0.3" + +"@radix-ui/react-toggle-group@1.0.4", "@radix-ui/react-toggle-group@^1.0.4": + version "1.0.4" + resolved "https://registry.yarnpkg.com/@radix-ui/react-toggle-group/-/react-toggle-group-1.0.4.tgz#f5b5c8c477831b013bec3580c55e20a68179d6ec" + integrity sha512-Uaj/M/cMyiyT9Bx6fOZO0SAG4Cls0GptBWiBmBxofmDbNVnYYoyRWj/2M/6VCi/7qcXFWnHhRUfdfZFvvkuu8A== + dependencies: + "@babel/runtime" "^7.13.10" + "@radix-ui/primitive" "1.0.1" + "@radix-ui/react-context" "1.0.1" + "@radix-ui/react-direction" "1.0.1" + "@radix-ui/react-primitive" "1.0.3" + "@radix-ui/react-roving-focus" "1.0.4" + "@radix-ui/react-toggle" "1.0.3" + "@radix-ui/react-use-controllable-state" "1.0.1" + +"@radix-ui/react-toggle@1.0.3", "@radix-ui/react-toggle@^1.0.3": + version "1.0.3" + resolved "https://registry.yarnpkg.com/@radix-ui/react-toggle/-/react-toggle-1.0.3.tgz#aecb2945630d1dc5c512997556c57aba894e539e" + integrity sha512-Pkqg3+Bc98ftZGsl60CLANXQBBQ4W3mTFS9EJvNxKMZ7magklKV69/id1mlAlOFDDfHvlCms0fx8fA4CMKDJHg== + dependencies: + "@babel/runtime" "^7.13.10" + "@radix-ui/primitive" "1.0.1" + "@radix-ui/react-primitive" "1.0.3" + "@radix-ui/react-use-controllable-state" "1.0.1" + +"@radix-ui/react-toolbar@^1.0.4": + version "1.0.4" + resolved "https://registry.yarnpkg.com/@radix-ui/react-toolbar/-/react-toolbar-1.0.4.tgz#3211a105567fa016e89921b5b514877f833de559" + integrity sha512-tBgmM/O7a07xbaEkYJWYTXkIdU/1pW4/KZORR43toC/4XWyBCURK0ei9kMUdp+gTPPKBgYLxXmRSH1EVcIDp8Q== + dependencies: + "@babel/runtime" "^7.13.10" + "@radix-ui/primitive" "1.0.1" + "@radix-ui/react-context" "1.0.1" + "@radix-ui/react-direction" "1.0.1" + "@radix-ui/react-primitive" "1.0.3" + "@radix-ui/react-roving-focus" "1.0.4" + "@radix-ui/react-separator" "1.0.3" + "@radix-ui/react-toggle-group" "1.0.4" + +"@radix-ui/react-tooltip@^1.0.6": + version "1.0.6" + resolved "https://registry.yarnpkg.com/@radix-ui/react-tooltip/-/react-tooltip-1.0.6.tgz#87a7786cd9f2b4de957ac645afae1575339c58b0" + integrity sha512-DmNFOiwEc2UDigsYj6clJENma58OelxD24O4IODoZ+3sQc3Zb+L8w1EP+y9laTuKCLAysPw4fD6/v0j4KNV8rg== + dependencies: + "@babel/runtime" "^7.13.10" + "@radix-ui/primitive" "1.0.1" + "@radix-ui/react-compose-refs" "1.0.1" + "@radix-ui/react-context" "1.0.1" + "@radix-ui/react-dismissable-layer" "1.0.4" + "@radix-ui/react-id" "1.0.1" + "@radix-ui/react-popper" "1.1.2" + "@radix-ui/react-portal" "1.0.3" + "@radix-ui/react-presence" "1.0.1" + "@radix-ui/react-primitive" "1.0.3" + "@radix-ui/react-slot" "1.0.2" + "@radix-ui/react-use-controllable-state" "1.0.1" + "@radix-ui/react-visually-hidden" "1.0.3" + +"@radix-ui/react-use-callback-ref@1.0.1": + version "1.0.1" + resolved "https://registry.yarnpkg.com/@radix-ui/react-use-callback-ref/-/react-use-callback-ref-1.0.1.tgz#f4bb1f27f2023c984e6534317ebc411fc181107a" + integrity sha512-D94LjX4Sp0xJFVaoQOd3OO9k7tpBYNOXdVhkltUbGv2Qb9OXdrg/CpsjlZv7ia14Sylv398LswWBVVu5nqKzAQ== + dependencies: + "@babel/runtime" "^7.13.10" + +"@radix-ui/react-use-controllable-state@1.0.1": + version "1.0.1" + resolved "https://registry.yarnpkg.com/@radix-ui/react-use-controllable-state/-/react-use-controllable-state-1.0.1.tgz#ecd2ced34e6330caf89a82854aa2f77e07440286" + integrity sha512-Svl5GY5FQeN758fWKrjM6Qb7asvXeiZltlT4U2gVfl8Gx5UAv2sMR0LWo8yhsIZh2oQ0eFdZ59aoOOMV7b47VA== + dependencies: + "@babel/runtime" "^7.13.10" + "@radix-ui/react-use-callback-ref" "1.0.1" + +"@radix-ui/react-use-escape-keydown@1.0.3": + version "1.0.3" + resolved "https://registry.yarnpkg.com/@radix-ui/react-use-escape-keydown/-/react-use-escape-keydown-1.0.3.tgz#217b840c250541609c66f67ed7bab2b733620755" + integrity sha512-vyL82j40hcFicA+M4Ex7hVkB9vHgSse1ZWomAqV2Je3RleKGO5iM8KMOEtfoSB0PnIelMd2lATjTGMYqN5ylTg== + dependencies: + "@babel/runtime" "^7.13.10" + "@radix-ui/react-use-callback-ref" "1.0.1" + +"@radix-ui/react-use-layout-effect@1.0.1": + version "1.0.1" + resolved "https://registry.yarnpkg.com/@radix-ui/react-use-layout-effect/-/react-use-layout-effect-1.0.1.tgz#be8c7bc809b0c8934acf6657b577daf948a75399" + integrity sha512-v/5RegiJWYdoCvMnITBkNNx6bCj20fiaJnWtRkU18yITptraXjffz5Qbn05uOiQnOvi+dbkznkoaMltz1GnszQ== + dependencies: + "@babel/runtime" "^7.13.10" + +"@radix-ui/react-use-previous@1.0.1": + version "1.0.1" + resolved "https://registry.yarnpkg.com/@radix-ui/react-use-previous/-/react-use-previous-1.0.1.tgz#b595c087b07317a4f143696c6a01de43b0d0ec66" + integrity sha512-cV5La9DPwiQ7S0gf/0qiD6YgNqM5Fk97Kdrlc5yBcrF3jyEZQwm7vYFqMo4IfeHgJXsRaMvLABFtd0OVEmZhDw== + dependencies: + "@babel/runtime" "^7.13.10" + +"@radix-ui/react-use-rect@1.0.1": + version "1.0.1" + resolved "https://registry.yarnpkg.com/@radix-ui/react-use-rect/-/react-use-rect-1.0.1.tgz#fde50b3bb9fd08f4a1cd204572e5943c244fcec2" + integrity sha512-Cq5DLuSiuYVKNU8orzJMbl15TXilTnJKUCltMVQg53BQOF1/C5toAaGrowkgksdBQ9H+SRL23g0HDmg9tvmxXw== + dependencies: + "@babel/runtime" "^7.13.10" + "@radix-ui/rect" "1.0.1" + +"@radix-ui/react-use-size@1.0.1": + version "1.0.1" + resolved "https://registry.yarnpkg.com/@radix-ui/react-use-size/-/react-use-size-1.0.1.tgz#1c5f5fea940a7d7ade77694bb98116fb49f870b2" + integrity sha512-ibay+VqrgcaI6veAojjofPATwledXiSmX+C0KrBk/xgpX9rBzPV3OsfwlhQdUOFbh+LKQorLYT+xTXW9V8yd0g== + dependencies: + "@babel/runtime" "^7.13.10" + "@radix-ui/react-use-layout-effect" "1.0.1" + +"@radix-ui/react-visually-hidden@1.0.3": + version "1.0.3" + resolved "https://registry.yarnpkg.com/@radix-ui/react-visually-hidden/-/react-visually-hidden-1.0.3.tgz#51aed9dd0fe5abcad7dee2a234ad36106a6984ac" + integrity sha512-D4w41yN5YRKtu464TLnByKzMDG/JlMPHtfZgQAu9v6mNakUqGUI9vUrfQKz8NK41VMm/xbZbh76NUTVtIYqOMA== + dependencies: + "@babel/runtime" "^7.13.10" + "@radix-ui/react-primitive" "1.0.3" + +"@radix-ui/rect@1.0.1": + version "1.0.1" + resolved "https://registry.yarnpkg.com/@radix-ui/rect/-/rect-1.0.1.tgz#bf8e7d947671996da2e30f4904ece343bc4a883f" + integrity sha512-fyrgCaedtvMg9NK3en0pnOYJdtfwxUcNolezkNPUsoX57X8oQk+NkqcvzHXD2uKNij6GXmWU9NDru2IWjrO4BQ== + dependencies: + "@babel/runtime" "^7.13.10" + +"@rushstack/eslint-patch@^1.1.3": + version "1.3.3" + resolved "https://registry.yarnpkg.com/@rushstack/eslint-patch/-/eslint-patch-1.3.3.tgz#16ab6c727d8c2020a5b6e4a176a243ecd88d8d69" + integrity sha512-0xd7qez0AQ+MbHatZTlI1gu5vkG8r7MYRUJAHPAHJBmGLs16zpkrpAVLvjQKQOqaXPDUBwOiJzNc00znHSCVBw== + +"@sinclair/typebox@^0.27.8": + version "0.27.8" + resolved "https://registry.yarnpkg.com/@sinclair/typebox/-/typebox-0.27.8.tgz#6667fac16c436b5434a387a34dedb013198f6e6e" + integrity sha512-+Fj43pSMwJs4KRrH/938Uf+uAELIgVBmQzg/q1YG10djyfA3TnrU8N8XzqCh/okZdszqBQTZf96idMfE5lnwTA== + +"@storybook/addon-actions@7.4.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/addon-actions/-/addon-actions-7.4.0.tgz#709988f46422b85b3672d2e6f90bf623af59faa9" + integrity sha512-0lHLLUlrGE7CBFrfmAXrBKu7fUIsiQlnNekuE3cIAjSgVR481bJEzYHUUoMATqpPC4GGErBdP1CZxVDDwWV8jA== + dependencies: + "@storybook/client-logger" "7.4.0" + "@storybook/components" "7.4.0" + "@storybook/core-events" "7.4.0" + "@storybook/global" "^5.0.0" + "@storybook/manager-api" "7.4.0" + "@storybook/preview-api" "7.4.0" + "@storybook/theming" "7.4.0" + "@storybook/types" "7.4.0" + dequal "^2.0.2" + lodash "^4.17.21" + polished "^4.2.2" + prop-types "^15.7.2" + react-inspector "^6.0.0" + telejson "^7.2.0" + ts-dedent "^2.0.0" + uuid "^9.0.0" + +"@storybook/addon-backgrounds@7.4.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/addon-backgrounds/-/addon-backgrounds-7.4.0.tgz#7d3048329b8ef73145a2e9b435b7b35004a65f86" + integrity sha512-cEO/Tp/eRE+5bf1FGN4wKLqLDBv3EYp9enJyXV7B3cFdciqtoE7VJPZuFZkzjJN1rRcOKSZp8g5agsx+x9uNGQ== + dependencies: + "@storybook/client-logger" "7.4.0" + "@storybook/components" "7.4.0" + "@storybook/core-events" "7.4.0" + "@storybook/global" "^5.0.0" + "@storybook/manager-api" "7.4.0" + "@storybook/preview-api" "7.4.0" + "@storybook/theming" "7.4.0" + "@storybook/types" "7.4.0" + memoizerific "^1.11.3" + ts-dedent "^2.0.0" + +"@storybook/addon-controls@7.4.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/addon-controls/-/addon-controls-7.4.0.tgz#b212d60fd74d69f6b63c53e4d52ab6c77ee51247" + integrity sha512-tYDfqpTR+c9y4kElmr3aWNHPot6kYd+nruYb697LpkCdy4lFErqSo0mhvPyZfMZp2KEajfp6YJAurhQWbvbj/A== + dependencies: + "@storybook/blocks" "7.4.0" + "@storybook/client-logger" "7.4.0" + "@storybook/components" "7.4.0" + "@storybook/core-common" "7.4.0" + "@storybook/core-events" "7.4.0" + "@storybook/manager-api" "7.4.0" + "@storybook/node-logger" "7.4.0" + "@storybook/preview-api" "7.4.0" + "@storybook/theming" "7.4.0" + "@storybook/types" "7.4.0" + lodash "^4.17.21" + ts-dedent "^2.0.0" + +"@storybook/addon-docs@7.4.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/addon-docs/-/addon-docs-7.4.0.tgz#e07233c264eaec149a0fcca0e27c586d4e80b403" + integrity sha512-LJE92LUeVTgi8W4tLBEbSvCqF54snmBfTFCr46vhCFov2CE2VBgEvIX1XT3dfUgYUOtPu3RXR2C89fYgU6VYZw== + dependencies: + "@jest/transform" "^29.3.1" + "@mdx-js/react" "^2.1.5" + "@storybook/blocks" "7.4.0" + "@storybook/client-logger" "7.4.0" + "@storybook/components" "7.4.0" + "@storybook/csf-plugin" "7.4.0" + "@storybook/csf-tools" "7.4.0" + "@storybook/global" "^5.0.0" + "@storybook/mdx2-csf" "^1.0.0" + "@storybook/node-logger" "7.4.0" + "@storybook/postinstall" "7.4.0" + "@storybook/preview-api" "7.4.0" + "@storybook/react-dom-shim" "7.4.0" + "@storybook/theming" "7.4.0" + "@storybook/types" "7.4.0" + fs-extra "^11.1.0" + remark-external-links "^8.0.0" + remark-slug "^6.0.0" + ts-dedent "^2.0.0" + +"@storybook/addon-essentials@^7.3.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/addon-essentials/-/addon-essentials-7.4.0.tgz#b5d19c60233e5bd5e1a29b76b51059c889f18d52" + integrity sha512-nZmNM9AKw2JXxnYUXyFKLeUF/cL7Z9E1WTeZyOFTDtU2aITRt8+LvaepwjchtPqu2B0GcQxLB5FRDdhy0I19nw== + dependencies: + "@storybook/addon-actions" "7.4.0" + "@storybook/addon-backgrounds" "7.4.0" + "@storybook/addon-controls" "7.4.0" + "@storybook/addon-docs" "7.4.0" + "@storybook/addon-highlight" "7.4.0" + "@storybook/addon-measure" "7.4.0" + "@storybook/addon-outline" "7.4.0" + "@storybook/addon-toolbars" "7.4.0" + "@storybook/addon-viewport" "7.4.0" + "@storybook/core-common" "7.4.0" + "@storybook/manager-api" "7.4.0" + "@storybook/node-logger" "7.4.0" + "@storybook/preview-api" "7.4.0" + ts-dedent "^2.0.0" + +"@storybook/addon-highlight@7.4.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/addon-highlight/-/addon-highlight-7.4.0.tgz#ea33826a7f610f5e76cfa59ff22283e01cfd76cd" + integrity sha512-kpYSb3oXI9t/1+aRJhToDZ0/1W4mu+SzTBfv9Bl2d/DogEkFzgJricoy5LtvS5EpcXUmKO1FJsw/DCm9buSL2g== + dependencies: + "@storybook/core-events" "7.4.0" + "@storybook/global" "^5.0.0" + "@storybook/preview-api" "7.4.0" + +"@storybook/addon-interactions@^7.3.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/addon-interactions/-/addon-interactions-7.4.0.tgz#d0c15303999ac1e8f33705146e9a0a6db6df339c" + integrity sha512-nEWP+Ib0Y/ShXfpCm40FBTbBy1/MT8XxTEAhcNN+3ZJ07Vhhkrb8GMlWHTKQv2PyghEVBYEoPFHhElUJQOe00g== + dependencies: + "@storybook/client-logger" "7.4.0" + "@storybook/components" "7.4.0" + "@storybook/core-common" "7.4.0" + "@storybook/core-events" "7.4.0" + "@storybook/global" "^5.0.0" + "@storybook/instrumenter" "7.4.0" + "@storybook/manager-api" "7.4.0" + "@storybook/preview-api" "7.4.0" + "@storybook/theming" "7.4.0" + "@storybook/types" "7.4.0" + jest-mock "^27.0.6" + polished "^4.2.2" + ts-dedent "^2.2.0" + +"@storybook/addon-links@^7.3.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/addon-links/-/addon-links-7.4.0.tgz#f10ba388143d0de75150a27e94241d5fb4dfba7e" + integrity sha512-lFj8fiokWKk3jx5YUQ4anQo1uCNDMP1y6nJ/92Y85vnOd1vJr3w4GlLy8eOWMABRE33AKLI5Yp6wcpWZDe7hhQ== + dependencies: + "@storybook/client-logger" "7.4.0" + "@storybook/core-events" "7.4.0" + "@storybook/csf" "^0.1.0" + "@storybook/global" "^5.0.0" + "@storybook/manager-api" "7.4.0" + "@storybook/preview-api" "7.4.0" + "@storybook/router" "7.4.0" + "@storybook/types" "7.4.0" + prop-types "^15.7.2" + ts-dedent "^2.0.0" + +"@storybook/addon-measure@7.4.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/addon-measure/-/addon-measure-7.4.0.tgz#61bc0d0af5af8c22e81b70e1690b2f58262944cd" + integrity sha512-8YjBqm6jPOBgkRn9YnJkLN0+ghgJiukdHOa0VB3qhiT+oww4ZOZ7mc2aQRwXQoFb05UbVVG9UNxE7lhyTyaG2w== + dependencies: + "@storybook/client-logger" "7.4.0" + "@storybook/components" "7.4.0" + "@storybook/core-events" "7.4.0" + "@storybook/global" "^5.0.0" + "@storybook/manager-api" "7.4.0" + "@storybook/preview-api" "7.4.0" + "@storybook/types" "7.4.0" + tiny-invariant "^1.3.1" + +"@storybook/addon-outline@7.4.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/addon-outline/-/addon-outline-7.4.0.tgz#63fef45815f209a3ad7ac2b3765f0734093af668" + integrity sha512-CCAWFC3bfkmYPzFjOemfH/kjpqJOHt+SdJgBKmwujDy+zum0DHlUL/7rd+U32cEpezCA8bapd0hlWn59C4agHQ== + dependencies: + "@storybook/client-logger" "7.4.0" + "@storybook/components" "7.4.0" + "@storybook/core-events" "7.4.0" + "@storybook/global" "^5.0.0" + "@storybook/manager-api" "7.4.0" + "@storybook/preview-api" "7.4.0" + "@storybook/types" "7.4.0" + ts-dedent "^2.0.0" + +"@storybook/addon-styling@^1.3.7": + version "1.3.7" + resolved "https://registry.yarnpkg.com/@storybook/addon-styling/-/addon-styling-1.3.7.tgz#a440fafcbf2899fac28c3c778bec9f3c6fc02dd9" + integrity sha512-JSBZMOrSw/3rlq5YoEI7Qyq703KSNP0Jd+gxTWu3/tP6245mpjn2dXnR8FvqVxCi+FG4lt2kQyPzgsuwEw1SSA== + dependencies: + "@babel/template" "^7.20.7" + "@babel/types" "^7.21.5" + "@storybook/api" "^7.0.12" + "@storybook/components" "^7.0.12" + "@storybook/core-common" "^7.0.12" + "@storybook/core-events" "^7.0.12" + "@storybook/manager-api" "^7.0.12" + "@storybook/node-logger" "^7.0.12" + "@storybook/preview-api" "^7.0.12" + "@storybook/theming" "^7.0.12" + "@storybook/types" "^7.0.12" + css-loader "^6.7.3" + less-loader "^11.1.0" + postcss-loader "^7.2.4" + prettier "^2.8.0" + resolve-url-loader "^5.0.0" + sass-loader "^13.2.2" + style-loader "^3.3.2" + +"@storybook/addon-toolbars@7.4.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/addon-toolbars/-/addon-toolbars-7.4.0.tgz#db1a3bc1d6e6aa0142b62aaf8c44d5a9f82fd6b7" + integrity sha512-00PDLchlQXI3ZClQHU0YQBfikAAxHOhVNv2QKW54yFKmxPl+P2c/VIeir9LcPhA04smKrJTD1u+Nszd66A9xAA== + dependencies: + "@storybook/client-logger" "7.4.0" + "@storybook/components" "7.4.0" + "@storybook/manager-api" "7.4.0" + "@storybook/preview-api" "7.4.0" + "@storybook/theming" "7.4.0" + +"@storybook/addon-viewport@7.4.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/addon-viewport/-/addon-viewport-7.4.0.tgz#a9bc167b822d31491cec6aad21cc0a420f1ae5b7" + integrity sha512-Bfoilf9eJV/C7tR8XHDxz3h8JlZ+iggoESp2Tc0bW9tlRvz+PsCqeyHhF/IgHY+gLnPal2PkK/PIM+ruO45HXA== + dependencies: + "@storybook/client-logger" "7.4.0" + "@storybook/components" "7.4.0" + "@storybook/core-events" "7.4.0" + "@storybook/global" "^5.0.0" + "@storybook/manager-api" "7.4.0" + "@storybook/preview-api" "7.4.0" + "@storybook/theming" "7.4.0" + memoizerific "^1.11.3" + prop-types "^15.7.2" + +"@storybook/api@^7.0.12": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/api/-/api-7.4.0.tgz#8cbe7edfdd4d23ab7aaa375869e0b93fdec9ccf5" + integrity sha512-L6CT3YCuUov9VHZUSA0euNuPB1Kczd2qVDgBzw8CFjQLei06ELoGCWFPS1X1HtY7d9BC+UjhU16uA4pjt833yw== + dependencies: + "@storybook/client-logger" "7.4.0" + "@storybook/manager-api" "7.4.0" + +"@storybook/blocks@7.4.0", "@storybook/blocks@^7.3.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/blocks/-/blocks-7.4.0.tgz#6a9240e2b58bac99a998c559d719be7ff4e19dcc" + integrity sha512-YQznNjJm+l32fCfPxrZso9+MbcyG0pWZSpx3RKI1+pxDMsAs4mbXsIw4//jKfjoDP/6/Cz/FJcSx8LT7i4BJ2w== + dependencies: + "@storybook/channels" "7.4.0" + "@storybook/client-logger" "7.4.0" + "@storybook/components" "7.4.0" + "@storybook/core-events" "7.4.0" + "@storybook/csf" "^0.1.0" + "@storybook/docs-tools" "7.4.0" + "@storybook/global" "^5.0.0" + "@storybook/manager-api" "7.4.0" + "@storybook/preview-api" "7.4.0" + "@storybook/theming" "7.4.0" + "@storybook/types" "7.4.0" + "@types/lodash" "^4.14.167" + color-convert "^2.0.1" + dequal "^2.0.2" + lodash "^4.17.21" + markdown-to-jsx "^7.1.8" + memoizerific "^1.11.3" + polished "^4.2.2" + react-colorful "^5.1.2" + telejson "^7.2.0" + tocbot "^4.20.1" + ts-dedent "^2.0.0" + util-deprecate "^1.0.2" + +"@storybook/builder-manager@7.4.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/builder-manager/-/builder-manager-7.4.0.tgz#80cf72ea83f88e16d585c5bdb40d563874c7d8ca" + integrity sha512-4fuxVzBIBbZh2aVBizSOU5EJ8b74IhR6x2TAZjifZZf5Gdxgfgio8sAyrrd/C78vrFOFhFEgmQhMqZRuCLHxvQ== + dependencies: + "@fal-works/esbuild-plugin-global-externals" "^2.1.2" + "@storybook/core-common" "7.4.0" + "@storybook/manager" "7.4.0" + "@storybook/node-logger" "7.4.0" + "@types/ejs" "^3.1.1" + "@types/find-cache-dir" "^3.2.1" + "@yarnpkg/esbuild-plugin-pnp" "^3.0.0-rc.10" + browser-assert "^1.2.1" + ejs "^3.1.8" + esbuild "^0.18.0" + esbuild-plugin-alias "^0.2.1" + express "^4.17.3" + find-cache-dir "^3.0.0" + fs-extra "^11.1.0" + process "^0.11.10" + util "^0.12.4" + +"@storybook/builder-webpack5@7.4.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/builder-webpack5/-/builder-webpack5-7.4.0.tgz#c9a4ee5a6424dd70f5b35f057de24afd268a5fd3" + integrity sha512-CYeXppqGACzDUpLCFvWvwD7IjN7VNi7+nwQ1uRNgW2NgBMOIldZe+gcTXcc0BuHyIitU5/vvquYM0qjis05LYw== + dependencies: + "@babel/core" "^7.22.0" + "@storybook/channels" "7.4.0" + "@storybook/client-logger" "7.4.0" + "@storybook/core-common" "7.4.0" + "@storybook/core-events" "7.4.0" + "@storybook/core-webpack" "7.4.0" + "@storybook/node-logger" "7.4.0" + "@storybook/preview" "7.4.0" + "@storybook/preview-api" "7.4.0" + "@swc/core" "^1.3.49" + "@types/node" "^16.0.0" + "@types/semver" "^7.3.4" + babel-loader "^9.0.0" + babel-plugin-named-exports-order "^0.0.2" + browser-assert "^1.2.1" + case-sensitive-paths-webpack-plugin "^2.4.0" + constants-browserify "^1.0.0" + css-loader "^6.7.1" + express "^4.17.3" + fork-ts-checker-webpack-plugin "^8.0.0" + fs-extra "^11.1.0" + html-webpack-plugin "^5.5.0" + path-browserify "^1.0.1" + process "^0.11.10" + semver "^7.3.7" + style-loader "^3.3.1" + swc-loader "^0.2.3" + terser-webpack-plugin "^5.3.1" + ts-dedent "^2.0.0" + url "^0.11.0" + util "^0.12.4" + util-deprecate "^1.0.2" + webpack "5" + webpack-dev-middleware "^6.1.1" + webpack-hot-middleware "^2.25.1" + webpack-virtual-modules "^0.5.0" + +"@storybook/channels@7.4.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/channels/-/channels-7.4.0.tgz#4ab69fce09c0fe7299f1595628b3de10b0fdcd8f" + integrity sha512-/1CU0s3npFumzVHLGeubSyPs21O3jNqtSppOjSB9iDTyV2GtQrjh5ntVwebfKpCkUSitx3x7TkCb9dylpEZ8+w== + dependencies: + "@storybook/client-logger" "7.4.0" + "@storybook/core-events" "7.4.0" + "@storybook/global" "^5.0.0" + qs "^6.10.0" + telejson "^7.2.0" + tiny-invariant "^1.3.1" + +"@storybook/cli@7.4.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/cli/-/cli-7.4.0.tgz#a50f435d55e3056547c983c0bfacb2eed63cd692" + integrity sha512-yn27cn3LzhTqpEVX6CzUz13KTJ3jPLA2eM4bO1t7SYUqpDlzw3lET9DIcYIaUAIiL+0r2Js3jW2BsyN/5KmO5w== + dependencies: + "@babel/core" "^7.22.9" + "@babel/preset-env" "^7.22.9" + "@babel/types" "^7.22.5" + "@ndelangen/get-tarball" "^3.0.7" + "@storybook/codemod" "7.4.0" + "@storybook/core-common" "7.4.0" + "@storybook/core-server" "7.4.0" + "@storybook/csf-tools" "7.4.0" + "@storybook/node-logger" "7.4.0" + "@storybook/telemetry" "7.4.0" + "@storybook/types" "7.4.0" + "@types/semver" "^7.3.4" + "@yarnpkg/fslib" "2.10.3" + "@yarnpkg/libzip" "2.3.0" + chalk "^4.1.0" + commander "^6.2.1" + cross-spawn "^7.0.3" + detect-indent "^6.1.0" + envinfo "^7.7.3" + execa "^5.0.0" + express "^4.17.3" + find-up "^5.0.0" + fs-extra "^11.1.0" + get-npm-tarball-url "^2.0.3" + get-port "^5.1.1" + giget "^1.0.0" + globby "^11.0.2" + jscodeshift "^0.14.0" + leven "^3.1.0" + ora "^5.4.1" + prettier "^2.8.0" + prompts "^2.4.0" + puppeteer-core "^2.1.1" + read-pkg-up "^7.0.1" + semver "^7.3.7" + simple-update-notifier "^2.0.0" + strip-json-comments "^3.0.1" + tempy "^1.0.1" + ts-dedent "^2.0.0" + util-deprecate "^1.0.2" + +"@storybook/client-logger@7.4.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/client-logger/-/client-logger-7.4.0.tgz#f90aa5ee29d540074f6e4890bae71836ac87273c" + integrity sha512-4pBnf7+df1wXEVcF1civqxbrtccGGHQkfWQkJo49s53RXvF7SRTcif6XTx0V3cQV0v7I1C5mmLm0LNlmjPRP1Q== + dependencies: + "@storybook/global" "^5.0.0" + +"@storybook/codemod@7.4.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/codemod/-/codemod-7.4.0.tgz#c23ef80253b5a5998c83e49e74bd6ff62683d27a" + integrity sha512-XqNhv5bec+L7TJ5tXdsMalmJazwaFMVVxoNlnb0f9zKhovAEF2F6hl6+Pnd2avRomH9+1q7EM+GwrTCAvzAfzg== + dependencies: + "@babel/core" "^7.22.9" + "@babel/preset-env" "^7.22.9" + "@babel/types" "^7.22.5" + "@storybook/csf" "^0.1.0" + "@storybook/csf-tools" "7.4.0" + "@storybook/node-logger" "7.4.0" + "@storybook/types" "7.4.0" + "@types/cross-spawn" "^6.0.2" + cross-spawn "^7.0.3" + globby "^11.0.2" + jscodeshift "^0.14.0" + lodash "^4.17.21" + prettier "^2.8.0" + recast "^0.23.1" + +"@storybook/components@7.4.0", "@storybook/components@^7.0.12": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/components/-/components-7.4.0.tgz#0cc83ff89dd9cdcde3eaeeb7b3fbcf2036ba6fb8" + integrity sha512-GGnQrI4NXwri/PqNjhO1vNv4tC7RBjY87ce9WHBq1ueat3kBakdqV97NzScoldXarkkKK6grBqmhw9jE5PfzhQ== + dependencies: + "@radix-ui/react-select" "^1.2.2" + "@radix-ui/react-toolbar" "^1.0.4" + "@storybook/client-logger" "7.4.0" + "@storybook/csf" "^0.1.0" + "@storybook/global" "^5.0.0" + "@storybook/theming" "7.4.0" + "@storybook/types" "7.4.0" + memoizerific "^1.11.3" + use-resize-observer "^9.1.0" + util-deprecate "^1.0.2" + +"@storybook/core-client@7.4.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/core-client/-/core-client-7.4.0.tgz#b2b683ebc44d0dfaa7a886f7bb1a5fc74a3d0965" + integrity sha512-AhysJS2HnydB8Jc+BMVzK5VLHa1liJjxroNsd+ZTgGUhD7R8wvozrswQgY4MLFtcaLwN/wDWlK2YavSBqmc94Q== + dependencies: + "@storybook/client-logger" "7.4.0" + "@storybook/preview-api" "7.4.0" + +"@storybook/core-common@7.4.0", "@storybook/core-common@^7.0.12": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/core-common/-/core-common-7.4.0.tgz#da71afd79a12cfb5565351f184f6797214a5da79" + integrity sha512-QKrBL46ZFdfTjlZE3f7b59Q5+frOHWIJ64sC9BZ2PHkZkGjFeYRDdJJ6EHLYBb+nToynl33dYN1GQz+hQn2vww== + dependencies: + "@storybook/node-logger" "7.4.0" + "@storybook/types" "7.4.0" + "@types/find-cache-dir" "^3.2.1" + "@types/node" "^16.0.0" + "@types/node-fetch" "^2.6.4" + "@types/pretty-hrtime" "^1.0.0" + chalk "^4.1.0" + esbuild "^0.18.0" + esbuild-register "^3.4.0" + file-system-cache "2.3.0" + find-cache-dir "^3.0.0" + find-up "^5.0.0" + fs-extra "^11.1.0" + glob "^10.0.0" + handlebars "^4.7.7" + lazy-universal-dotenv "^4.0.0" + node-fetch "^2.0.0" + picomatch "^2.3.0" + pkg-dir "^5.0.0" + pretty-hrtime "^1.0.3" + resolve-from "^5.0.0" + ts-dedent "^2.0.0" + +"@storybook/core-events@7.4.0", "@storybook/core-events@^7.0.12": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/core-events/-/core-events-7.4.0.tgz#0d50d254d65a678065d5906ac1dcab64396f2f6a" + integrity sha512-JavEo4dw7TQdF5pSKjk4RtqLgsG2R/eWRI8vZ3ANKa0ploGAnQR/eMTfSxf6TUH3ElBWLJhi+lvUCkKXPQD+dw== + dependencies: + ts-dedent "^2.0.0" + +"@storybook/core-server@7.4.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/core-server/-/core-server-7.4.0.tgz#9e624789ff30d9538ac014b038c48fac0ebb7272" + integrity sha512-AcbfXatHVx1by4R2CiPIMgjQlOL3sUbVarkhmgUcL0AWT0zC0SCQWUZdo22en+jZhAraazgXyLGNCVP7A+6Tqg== + dependencies: + "@aw-web-design/x-default-browser" "1.4.126" + "@discoveryjs/json-ext" "^0.5.3" + "@storybook/builder-manager" "7.4.0" + "@storybook/channels" "7.4.0" + "@storybook/core-common" "7.4.0" + "@storybook/core-events" "7.4.0" + "@storybook/csf" "^0.1.0" + "@storybook/csf-tools" "7.4.0" + "@storybook/docs-mdx" "^0.1.0" + "@storybook/global" "^5.0.0" + "@storybook/manager" "7.4.0" + "@storybook/node-logger" "7.4.0" + "@storybook/preview-api" "7.4.0" + "@storybook/telemetry" "7.4.0" + "@storybook/types" "7.4.0" + "@types/detect-port" "^1.3.0" + "@types/node" "^16.0.0" + "@types/pretty-hrtime" "^1.0.0" + "@types/semver" "^7.3.4" + better-opn "^3.0.2" + chalk "^4.1.0" + cli-table3 "^0.6.1" + compression "^1.7.4" + detect-port "^1.3.0" + express "^4.17.3" + fs-extra "^11.1.0" + globby "^11.0.2" + ip "^2.0.0" + lodash "^4.17.21" + open "^8.4.0" + pretty-hrtime "^1.0.3" + prompts "^2.4.0" + read-pkg-up "^7.0.1" + semver "^7.3.7" + serve-favicon "^2.5.0" + telejson "^7.2.0" + tiny-invariant "^1.3.1" + ts-dedent "^2.0.0" + util "^0.12.4" + util-deprecate "^1.0.2" + watchpack "^2.2.0" + ws "^8.2.3" + +"@storybook/core-webpack@7.4.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/core-webpack/-/core-webpack-7.4.0.tgz#0ff348a1590e9b8d425e9aec1ed850e3cfa3e75c" + integrity sha512-1zxzJjRbkcjl++OjYBVTDi0V/yO22Kz3ciPASTvXwrg0fXTXgxwxhJBmgOI4r17oY0kOWnJ1RDsmd95NLGAbGw== + dependencies: + "@storybook/core-common" "7.4.0" + "@storybook/node-logger" "7.4.0" + "@storybook/types" "7.4.0" + "@types/node" "^16.0.0" + ts-dedent "^2.0.0" + +"@storybook/csf-plugin@7.4.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/csf-plugin/-/csf-plugin-7.4.0.tgz#f25ebb30affbc9b4dd61b1fdb12c4a4257a275dc" + integrity sha512-X1L3l/dpz2UYjCEQlFLkW7w1A13pmzDZpJ0lotkV79PALlakMXBeoX3I2E0VMjJATV8wC9RSj56COBAs6HsPeg== + dependencies: + "@storybook/csf-tools" "7.4.0" + unplugin "^1.3.1" + +"@storybook/csf-tools@7.4.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/csf-tools/-/csf-tools-7.4.0.tgz#db5c97ee603da9a68511192d701534e356f9e592" + integrity sha512-bKyOmWPyvT50Neq2wCRr2PmVGLVVm6pOw8WL5t5jueD8sRRzo9QdfhEkqmuSyqdsBdt3SiJKL5oA6dqY5Vl9ww== + dependencies: + "@babel/generator" "^7.22.9" + "@babel/parser" "^7.22.7" + "@babel/traverse" "^7.22.8" + "@babel/types" "^7.22.5" + "@storybook/csf" "^0.1.0" + "@storybook/types" "7.4.0" + fs-extra "^11.1.0" + recast "^0.23.1" + ts-dedent "^2.0.0" + +"@storybook/csf@^0.0.1": + version "0.0.1" + resolved "https://registry.yarnpkg.com/@storybook/csf/-/csf-0.0.1.tgz#95901507dc02f0bc6f9ac8ee1983e2fc5bb98ce6" + integrity sha512-USTLkZze5gkel8MYCujSRBVIrUQ3YPBrLOx7GNk/0wttvVtlzWXAq9eLbQ4p/NicGxP+3T7KPEMVV//g+yubpw== + dependencies: + lodash "^4.17.15" + +"@storybook/csf@^0.1.0": + version "0.1.1" + resolved "https://registry.yarnpkg.com/@storybook/csf/-/csf-0.1.1.tgz#abccc8c3e49aed0a6a7e87beb0d1c262b1921c06" + integrity sha512-4hE3AlNVxR60Wc5KSC68ASYzUobjPqtSKyhV6G+ge0FIXU55N5nTY7dXGRZHQGDBPq+XqchMkIdlkHPRs8nTHg== + dependencies: + type-fest "^2.19.0" + +"@storybook/docs-mdx@^0.1.0": + version "0.1.0" + resolved "https://registry.yarnpkg.com/@storybook/docs-mdx/-/docs-mdx-0.1.0.tgz#33ba0e39d1461caf048b57db354b2cc410705316" + integrity sha512-JDaBR9lwVY4eSH5W8EGHrhODjygPd6QImRbwjAuJNEnY0Vw4ie3bPkeGfnacB3OBW6u/agqPv2aRlR46JcAQLg== + +"@storybook/docs-tools@7.4.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/docs-tools/-/docs-tools-7.4.0.tgz#d9109c9c8ec4e90bb24d1acfcc16834a252618eb" + integrity sha512-DzXmt4JorAOePoS+sjQznf8jLPI9D5mdB1eSXjfvmGBQyyehKTZv5+TXuxYvT3iPN4rW4OPrIrQCSIrbULFdwA== + dependencies: + "@storybook/core-common" "7.4.0" + "@storybook/preview-api" "7.4.0" + "@storybook/types" "7.4.0" + "@types/doctrine" "^0.0.3" + doctrine "^3.0.0" + lodash "^4.17.21" + +"@storybook/global@^5.0.0": + version "5.0.0" + resolved "https://registry.yarnpkg.com/@storybook/global/-/global-5.0.0.tgz#b793d34b94f572c1d7d9e0f44fac4e0dbc9572ed" + integrity sha512-FcOqPAXACP0I3oJ/ws6/rrPT9WGhu915Cg8D02a9YxLo0DE9zI+a9A5gRGvmQ09fiWPukqI8ZAEoQEdWUKMQdQ== + +"@storybook/instrumenter@7.4.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/instrumenter/-/instrumenter-7.4.0.tgz#197335f25a45ecdc2c5f458bff1c2481d7ffe08c" + integrity sha512-jZKxLK0lGKxY8LEul6GP7s+PDlNuXT4JU6MnPY9+SVSo23lP0pAOxo/ojV8WTLf48tcoyL3ztSfbYhxnaJvBfw== + dependencies: + "@storybook/channels" "7.4.0" + "@storybook/client-logger" "7.4.0" + "@storybook/core-events" "7.4.0" + "@storybook/global" "^5.0.0" + "@storybook/preview-api" "7.4.0" + +"@storybook/manager-api@7.4.0", "@storybook/manager-api@^7.0.12": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/manager-api/-/manager-api-7.4.0.tgz#aee0153df1583459b7e1e64e1d8c46fb49a584c8" + integrity sha512-sBfkkt0eZGTozeKrbzMtWLEOQrgqdk24OUJlkc2IDaucR1CBNjoCMjNeYg7cLDw0rXE8W3W3AdWtJnfsUbLMAQ== + dependencies: + "@storybook/channels" "7.4.0" + "@storybook/client-logger" "7.4.0" + "@storybook/core-events" "7.4.0" + "@storybook/csf" "^0.1.0" + "@storybook/global" "^5.0.0" + "@storybook/router" "7.4.0" + "@storybook/theming" "7.4.0" + "@storybook/types" "7.4.0" + dequal "^2.0.2" + lodash "^4.17.21" + memoizerific "^1.11.3" + semver "^7.3.7" + store2 "^2.14.2" + telejson "^7.2.0" + ts-dedent "^2.0.0" + +"@storybook/manager@7.4.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/manager/-/manager-7.4.0.tgz#21a825c9145f56ca6c38d3e9d3546b311a6db14e" + integrity sha512-uOSdPBEBKg8WORUZ5HKHb4KnKcTyA5j5Q8MWy/NBaRd22JR3fQkZiKuHer9WJIOQTU+fb6KDmzhZbCTKg5Euog== + +"@storybook/mdx2-csf@^1.0.0": + version "1.1.0" + resolved "https://registry.yarnpkg.com/@storybook/mdx2-csf/-/mdx2-csf-1.1.0.tgz#97f6df04d0bf616991cc1005a073ac004a7281e5" + integrity sha512-TXJJd5RAKakWx4BtpwvSNdgTDkKM6RkXU8GK34S/LhidQ5Pjz3wcnqb0TxEkfhK/ztbP8nKHqXFwLfa2CYkvQw== + +"@storybook/nextjs@^7.3.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/nextjs/-/nextjs-7.4.0.tgz#6456c0bdca4e53ade56791fc432dd2524f16ecfe" + integrity sha512-nGmer4Hu1/XX3+XyxfAkQ9d16Qsj467aLc7MTNQ2uFyYAksCqT3bvznooUOcD/X5NfoyL2YA78OczGdt1HFFpQ== + dependencies: + "@babel/core" "^7.22.9" + "@babel/plugin-proposal-class-properties" "^7.18.6" + "@babel/plugin-proposal-export-namespace-from" "^7.18.9" + "@babel/plugin-proposal-numeric-separator" "^7.18.6" + "@babel/plugin-proposal-object-rest-spread" "^7.20.7" + "@babel/plugin-syntax-bigint" "^7.8.3" + "@babel/plugin-syntax-dynamic-import" "^7.8.3" + "@babel/plugin-syntax-import-assertions" "^7.22.5" + "@babel/plugin-transform-runtime" "^7.22.9" + "@babel/preset-env" "^7.22.9" + "@babel/preset-react" "^7.22.5" + "@babel/preset-typescript" "^7.22.5" + "@babel/runtime" "^7.22.6" + "@storybook/addon-actions" "7.4.0" + "@storybook/builder-webpack5" "7.4.0" + "@storybook/core-common" "7.4.0" + "@storybook/node-logger" "7.4.0" + "@storybook/preset-react-webpack" "7.4.0" + "@storybook/preview-api" "7.4.0" + "@storybook/react" "7.4.0" + "@types/node" "^16.0.0" + css-loader "^6.7.3" + find-up "^5.0.0" + fs-extra "^11.1.0" + image-size "^1.0.0" + loader-utils "^3.2.0" + node-polyfill-webpack-plugin "^2.0.1" + pnp-webpack-plugin "^1.7.0" + postcss "^8.4.21" + postcss-loader "^7.0.2" + resolve-url-loader "^5.0.0" + sass-loader "^12.4.0" + semver "^7.3.5" + style-loader "^3.3.1" + styled-jsx "5.1.1" + ts-dedent "^2.0.0" + tsconfig-paths "^4.0.0" + tsconfig-paths-webpack-plugin "^4.0.1" + +"@storybook/node-logger@7.4.0", "@storybook/node-logger@^7.0.12": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/node-logger/-/node-logger-7.4.0.tgz#808ed8a63e3bc2f97a2d276b4e8ddaa72b79deb0" + integrity sha512-tWSWkYyAvp6SxjIBaTklg29avzv/3Lv4c0dOG2o5tz79PyZkq9v6sQtwLLoI8EJA9Mo8Z08vaJp8NZyDQ9RCuA== + +"@storybook/postinstall@7.4.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/postinstall/-/postinstall-7.4.0.tgz#81f3bef31b566e26d616f9c3ce567f07ff143cc7" + integrity sha512-ZVBZggqkuj7ysfuHSCd/J7ovWV06zY9uWf+VU+Zw7ZeojDT8QHFrCurPsN7D9679j9vRU1/kSzqvAiStALS33g== + +"@storybook/preset-react-webpack@7.4.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/preset-react-webpack/-/preset-react-webpack-7.4.0.tgz#5d8c1a776fe46ab732a165129da57b89483e1e6b" + integrity sha512-9iZ9lvhRUYtxXmJMqR7txNyatrHryqo6FSKzfpUzmcCySn3d7mu9I6LEPxEir43TkPnBio3W4EsbvtIhjJ5ekA== + dependencies: + "@babel/preset-flow" "^7.22.5" + "@babel/preset-react" "^7.22.5" + "@pmmmwh/react-refresh-webpack-plugin" "^0.5.5" + "@storybook/core-webpack" "7.4.0" + "@storybook/docs-tools" "7.4.0" + "@storybook/node-logger" "7.4.0" + "@storybook/react" "7.4.0" + "@storybook/react-docgen-typescript-plugin" "1.0.6--canary.9.0c3f3b7.0" + "@types/node" "^16.0.0" + "@types/semver" "^7.3.4" + babel-plugin-add-react-displayname "^0.0.5" + babel-plugin-react-docgen "^4.2.1" + fs-extra "^11.1.0" + react-refresh "^0.11.0" + semver "^7.3.7" + webpack "5" + +"@storybook/preview-api@7.4.0", "@storybook/preview-api@^7.0.12": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/preview-api/-/preview-api-7.4.0.tgz#46818910545735bef43965651eef380a6f481f4b" + integrity sha512-ndXO0Nx+eE7ktVE4EqHpQZ0guX7yYBdruDdJ7B739C0+OoPWsJN7jAzUqq0NXaBcYrdaU5gTy+KnWJUt8R+OyA== + dependencies: + "@storybook/channels" "7.4.0" + "@storybook/client-logger" "7.4.0" + "@storybook/core-events" "7.4.0" + "@storybook/csf" "^0.1.0" + "@storybook/global" "^5.0.0" + "@storybook/types" "7.4.0" + "@types/qs" "^6.9.5" + dequal "^2.0.2" + lodash "^4.17.21" + memoizerific "^1.11.3" + qs "^6.10.0" + synchronous-promise "^2.0.15" + ts-dedent "^2.0.0" + util-deprecate "^1.0.2" + +"@storybook/preview@7.4.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/preview/-/preview-7.4.0.tgz#a58756ac9b12ea21f203032eca47991946257b53" + integrity sha512-R4LMTvUrVAbcUetRbAXpY3frkwD0eysqHrByiR73040+ngzDwtZOBAy0JfO3jw3WrWv2dn3kWlao5aEwVc9Exw== + +"@storybook/react-docgen-typescript-plugin@1.0.6--canary.9.0c3f3b7.0": + version "1.0.6--canary.9.0c3f3b7.0" + resolved "https://registry.yarnpkg.com/@storybook/react-docgen-typescript-plugin/-/react-docgen-typescript-plugin-1.0.6--canary.9.0c3f3b7.0.tgz#7f10f3c641f32e4513a8b6ffb5036933e7059534" + integrity sha512-KUqXC3oa9JuQ0kZJLBhVdS4lOneKTOopnNBK4tUAgoxWQ3u/IjzdueZjFr7gyBrXMoU6duutk3RQR9u8ZpYJ4Q== + dependencies: + debug "^4.1.1" + endent "^2.0.1" + find-cache-dir "^3.3.1" + flat-cache "^3.0.4" + micromatch "^4.0.2" + react-docgen-typescript "^2.2.2" + tslib "^2.0.0" + +"@storybook/react-dom-shim@7.4.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/react-dom-shim/-/react-dom-shim-7.4.0.tgz#12f137f00f2a209cb49a4084475dd93f23e0678a" + integrity sha512-TLpb8a2hnWJoRLqoXpMADh82BFfRZll6JI2Waf1FjnvJ4SF9eS0zBbxybrjW3lFAHWy2XJi+rwcK8FiPj0iBoQ== + +"@storybook/react@7.4.0", "@storybook/react@^7.3.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/react/-/react-7.4.0.tgz#18d29aa49f0b784b46613d26a243caf473177403" + integrity sha512-QWsFw/twsNkcWI6brW06sugQQ5dV+fJm4IrEeI28cA4cBHK9G9HKOwCHoXDUWikzZx48XYMpNfs/WyIkuGmEqg== + dependencies: + "@storybook/client-logger" "7.4.0" + "@storybook/core-client" "7.4.0" + "@storybook/docs-tools" "7.4.0" + "@storybook/global" "^5.0.0" + "@storybook/preview-api" "7.4.0" + "@storybook/react-dom-shim" "7.4.0" + "@storybook/types" "7.4.0" + "@types/escodegen" "^0.0.6" + "@types/estree" "^0.0.51" + "@types/node" "^16.0.0" + acorn "^7.4.1" + acorn-jsx "^5.3.1" + acorn-walk "^7.2.0" + escodegen "^2.1.0" + html-tags "^3.1.0" + lodash "^4.17.21" + prop-types "^15.7.2" + react-element-to-jsx-string "^15.0.0" + ts-dedent "^2.0.0" + type-fest "~2.19" + util-deprecate "^1.0.2" + +"@storybook/router@7.4.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/router/-/router-7.4.0.tgz#627f824bfd9cc4653ee84581fc09373ab1463336" + integrity sha512-IATdtFL5C3ryjNQSwaQfrmiOZiVFoVNMevMoBGDC++g0laSW40TGiNK6fUjUDBKuOgbuDt4Svfbl29k21GefEg== + dependencies: + "@storybook/client-logger" "7.4.0" + memoizerific "^1.11.3" + qs "^6.10.0" + +"@storybook/telemetry@7.4.0": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/telemetry/-/telemetry-7.4.0.tgz#04e47a2d9decf7671273130a9af9d231a8c3d2e8" + integrity sha512-oxCB3kIbpiDWuXEtQhk/j6t1/h0KKWAuvxmcwGPxwhEvj/uNtoM+f1qhoDID9waxNo4AccU9Px+1ZJQ+2ejcDg== + dependencies: + "@storybook/client-logger" "7.4.0" + "@storybook/core-common" "7.4.0" + "@storybook/csf-tools" "7.4.0" + chalk "^4.1.0" + detect-package-manager "^2.0.1" + fetch-retry "^5.0.2" + fs-extra "^11.1.0" + read-pkg-up "^7.0.1" + +"@storybook/testing-library@^0.2.0": + version "0.2.0" + resolved "https://registry.yarnpkg.com/@storybook/testing-library/-/testing-library-0.2.0.tgz#09202b90ea5bd67b503dbb1a0b1f3ab3eb005d04" + integrity sha512-Ff6jNnrsosmDshgCf0Eb5Cz7IA34p/1Ps5N3Kp3598kfXpBSccSkQQvVFUXC3kIHw/isIXWPqntZuKqnWUz7Gw== + dependencies: + "@testing-library/dom" "^9.0.0" + "@testing-library/user-event" "^14.0.0" + ts-dedent "^2.2.0" + +"@storybook/theming@7.4.0", "@storybook/theming@^7.0.12": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/theming/-/theming-7.4.0.tgz#f5d9f8f55c41e08c0f50b57d9fb0e159ed595274" + integrity sha512-eLjEf6G3cqlegfutF/iUrec9LrUjKDj7K4ZhGdACWrf7bQcODs99EK62e9/d8GNKr4b+QMSEuM6XNGaqdPnuzQ== + dependencies: + "@emotion/use-insertion-effect-with-fallbacks" "^1.0.0" + "@storybook/client-logger" "7.4.0" + "@storybook/global" "^5.0.0" + memoizerific "^1.11.3" + +"@storybook/types@7.4.0", "@storybook/types@^7.0.12": + version "7.4.0" + resolved "https://registry.yarnpkg.com/@storybook/types/-/types-7.4.0.tgz#71ce550d4d469f6aaf9777fc7432db9fb67f53f9" + integrity sha512-XyzYkmeklywxvElPrIWLczi/PWtEdgTL6ToT3++FVxptsC2LZKS3Ue+sBcQ9xRZhkRemw4HQHwed5EW3dO8yUg== + dependencies: + "@storybook/channels" "7.4.0" + "@types/babel__core" "^7.0.0" + "@types/express" "^4.7.0" + "@types/react" "^16.14.34" + file-system-cache "2.3.0" + +"@swc/core-darwin-arm64@1.3.82": + version "1.3.82" + resolved "https://registry.yarnpkg.com/@swc/core-darwin-arm64/-/core-darwin-arm64-1.3.82.tgz#bbf9874747b51053d8a59ea26c3e235c326f24a3" + integrity sha512-JfsyDW34gVKD3uE0OUpUqYvAD3yseEaicnFP6pB292THtLJb0IKBBnK50vV/RzEJtc1bR3g1kNfxo2PeurZTrA== + +"@swc/core-darwin-x64@1.3.82": + version "1.3.82" + resolved "https://registry.yarnpkg.com/@swc/core-darwin-x64/-/core-darwin-x64-1.3.82.tgz#145cdde16678e0d793620035783e5b413a16ac43" + integrity sha512-ogQWgNMq7qTpITjcP3dnzkFNj7bh6SwMr859GvtOTrE75H7L7jDWxESfH4f8foB/LGxBKiDNmxKhitCuAsZK4A== + +"@swc/core-linux-arm-gnueabihf@1.3.82": + version "1.3.82" + resolved "https://registry.yarnpkg.com/@swc/core-linux-arm-gnueabihf/-/core-linux-arm-gnueabihf-1.3.82.tgz#0c2f32c5793f2ac8e8ccf416aec84d016c30ef7b" + integrity sha512-7TMXG1lXlNhD0kUiEqs+YlGV4irAdBa2quuy+XI3oJf2fBK6dQfEq4xBy65B3khrorzQS3O0oDGQ+cmdpHExHA== + +"@swc/core-linux-arm64-gnu@1.3.82": + version "1.3.82" + resolved "https://registry.yarnpkg.com/@swc/core-linux-arm64-gnu/-/core-linux-arm64-gnu-1.3.82.tgz#2313d4901fa0ebdd2a0f189909073e1e8a07f1d6" + integrity sha512-26JkOujbzcItPAmIbD5vHJxQVy5ihcSu3YHTKwope1h28sApZdtE7S3e2G3gsZRTIdsCQkXUtAQeqHxGWWR3pw== + +"@swc/core-linux-arm64-musl@1.3.82": + version "1.3.82" + resolved "https://registry.yarnpkg.com/@swc/core-linux-arm64-musl/-/core-linux-arm64-musl-1.3.82.tgz#6e96cf6e52e647fecf27511d766bea90e96f8a2f" + integrity sha512-8Izj9tuuMpoc3cqiPBRtwqpO1BZ/+sfZVsEhLxrbOFlcSb8LnKyMle1g3JMMUwI4EU75RGVIzZMn8A6GOKdJbA== + +"@swc/core-linux-x64-gnu@1.3.82": + version "1.3.82" + resolved "https://registry.yarnpkg.com/@swc/core-linux-x64-gnu/-/core-linux-x64-gnu-1.3.82.tgz#6275c10d7c8c0768550bc7934c9dd8cde4881d92" + integrity sha512-0GSrIBScQwTaPv46T2qB7XnDYxndRCpwH4HMjh6FN+I+lfPUhTSJKW8AonqrqT1TbpFIgvzQs7EnTsD7AnSCow== + +"@swc/core-linux-x64-musl@1.3.82": + version "1.3.82" + resolved "https://registry.yarnpkg.com/@swc/core-linux-x64-musl/-/core-linux-x64-musl-1.3.82.tgz#edb98c30bd0de42bf1a63469937630d942c71988" + integrity sha512-KJUnaaepDKNzrEbwz4jv0iC3/t9x0NSoe06fnkAlhh2+NFKWKKJhVCOBTrpds8n7eylBDIXUlK34XQafjVMUdg== + +"@swc/core-win32-arm64-msvc@1.3.82": + version "1.3.82" + resolved "https://registry.yarnpkg.com/@swc/core-win32-arm64-msvc/-/core-win32-arm64-msvc-1.3.82.tgz#0a8e9b361aac37d01f684c8a3d3e94e5f8c3b14f" + integrity sha512-TR3MHKhDYIyGyFcyl2d/p1ftceXcubAhX5wRSOdtOyr5+K/v3jbyCCqN7bbqO5o43wQVCwwR/drHleYyDZvg8Q== + +"@swc/core-win32-ia32-msvc@1.3.82": + version "1.3.82" + resolved "https://registry.yarnpkg.com/@swc/core-win32-ia32-msvc/-/core-win32-ia32-msvc-1.3.82.tgz#096854ff764282766271f1354ee1214358a8bf01" + integrity sha512-ZX4HzVVt6hs84YUg70UvyBJnBOIspmQQM0iXSzBvOikk3zRoN7BnDwQH4GScvevCEBuou60+i4I6d5kHLOfh8Q== + +"@swc/core-win32-x64-msvc@1.3.82": + version "1.3.82" + resolved "https://registry.yarnpkg.com/@swc/core-win32-x64-msvc/-/core-win32-x64-msvc-1.3.82.tgz#1181070bff4a13a7fcc7f1020eef1571f8c1257a" + integrity sha512-4mJMnex21kbQoaHeAmHnVwQN9/XAfPszJ6n9HI7SVH+aAHnbBIR0M59/b50/CJMjTj5niUGk7EwQ3nhVNOG32g== + +"@swc/core@^1.3.49": + version "1.3.82" + resolved "https://registry.yarnpkg.com/@swc/core/-/core-1.3.82.tgz#8f6c53db3c23a1769b6c5085fbcb3b1df9548a40" + integrity sha512-jpC1a18HMH67018Ij2jh+hT7JBFu7ZKcQVfrZ8K6JuEY+kjXmbea07P9MbQUZbAe0FB+xi3CqEVCP73MebodJQ== + dependencies: + "@swc/types" "^0.1.4" + optionalDependencies: + "@swc/core-darwin-arm64" "1.3.82" + "@swc/core-darwin-x64" "1.3.82" + "@swc/core-linux-arm-gnueabihf" "1.3.82" + "@swc/core-linux-arm64-gnu" "1.3.82" + "@swc/core-linux-arm64-musl" "1.3.82" + "@swc/core-linux-x64-gnu" "1.3.82" + "@swc/core-linux-x64-musl" "1.3.82" + "@swc/core-win32-arm64-msvc" "1.3.82" + "@swc/core-win32-ia32-msvc" "1.3.82" + "@swc/core-win32-x64-msvc" "1.3.82" + +"@swc/helpers@0.5.1": + version "0.5.1" + resolved "https://registry.yarnpkg.com/@swc/helpers/-/helpers-0.5.1.tgz#e9031491aa3f26bfcc974a67f48bd456c8a5357a" + integrity sha512-sJ902EfIzn1Fa+qYmjdQqh8tPsoxyBz+8yBKC2HKUxyezKJFwPGOn7pv4WY6QuQW//ySQi5lJjA/ZT9sNWWNTg== + dependencies: + tslib "^2.4.0" + +"@swc/types@^0.1.4": + version "0.1.4" + resolved "https://registry.yarnpkg.com/@swc/types/-/types-0.1.4.tgz#8d647e111dc97a8e2881bf71c2ee2d011698ff10" + integrity sha512-z/G02d+59gyyUb7KYhKi9jOhicek6QD2oMaotUyG+lUkybpXoV49dY9bj7Ah5Q+y7knK2jU67UTX9FyfGzaxQg== + +"@testing-library/dom@^9.0.0": + version "9.3.1" + resolved "https://registry.yarnpkg.com/@testing-library/dom/-/dom-9.3.1.tgz#8094f560e9389fb973fe957af41bf766937a9ee9" + integrity sha512-0DGPd9AR3+iDTjGoMpxIkAsUihHZ3Ai6CneU6bRRrffXMgzCdlNk43jTrD2/5LT6CBb3MWTP8v510JzYtahD2w== + dependencies: + "@babel/code-frame" "^7.10.4" + "@babel/runtime" "^7.12.5" + "@types/aria-query" "^5.0.1" + aria-query "5.1.3" + chalk "^4.1.0" + dom-accessibility-api "^0.5.9" + lz-string "^1.5.0" + pretty-format "^27.0.2" + +"@testing-library/user-event@^14.0.0": + version "14.4.3" + resolved "https://registry.yarnpkg.com/@testing-library/user-event/-/user-event-14.4.3.tgz#af975e367743fa91989cd666666aec31a8f50591" + integrity sha512-kCUc5MEwaEMakkO5x7aoD+DLi02ehmEM2QCGWvNqAS1dV/fAvORWEjnjsEIvml59M7Y5kCkWN6fCCyPOe8OL6Q== + +"@tremor/react@^3.6.6": + version "3.7.2" + resolved "https://registry.yarnpkg.com/@tremor/react/-/react-3.7.2.tgz#96136af0a65bf88dd395fb80610a42dbebbf0cb6" + integrity sha512-WEFsXL9uot5bL/T2YCtkj7PeTC1XVgevWaD2Ql8h/9xKTpA6YCwEaj/jQqtz83b/8GAYx3ZsI8choKqPNGGb9Q== + dependencies: + "@floating-ui/react" "^0.19.1" + "@headlessui/react" "^1.7.14" + "@headlessui/tailwindcss" "^0.1.3" + date-fns "^2.28.0" + react-day-picker "^8.7.1" + react-transition-group "^4.4.5" + recharts "^2.7.1" + tailwind-merge "^1.9.1" + +"@types/aria-query@^5.0.1": + version "5.0.1" + resolved "https://registry.yarnpkg.com/@types/aria-query/-/aria-query-5.0.1.tgz#3286741fb8f1e1580ac28784add4c7a1d49bdfbc" + integrity sha512-XTIieEY+gvJ39ChLcB4If5zHtPxt3Syj5rgZR+e1ctpmK8NjPf0zFqsz4JpLJT0xla9GFDKjy8Cpu331nrmE1Q== + +"@types/babel__core@^7.0.0": + version "7.20.1" + resolved "https://registry.yarnpkg.com/@types/babel__core/-/babel__core-7.20.1.tgz#916ecea274b0c776fec721e333e55762d3a9614b" + integrity sha512-aACu/U/omhdk15O4Nfb+fHgH/z3QsfQzpnvRZhYhThms83ZnAOZz7zZAWO7mn2yyNQaA4xTO8GLK3uqFU4bYYw== + dependencies: + "@babel/parser" "^7.20.7" + "@babel/types" "^7.20.7" + "@types/babel__generator" "*" + "@types/babel__template" "*" + "@types/babel__traverse" "*" + +"@types/babel__generator@*": + version "7.6.4" + resolved "https://registry.yarnpkg.com/@types/babel__generator/-/babel__generator-7.6.4.tgz#1f20ce4c5b1990b37900b63f050182d28c2439b7" + integrity sha512-tFkciB9j2K755yrTALxD44McOrk+gfpIpvC3sxHjRawj6PfnQxrse4Clq5y/Rq+G3mrBurMax/lG8Qn2t9mSsg== + dependencies: + "@babel/types" "^7.0.0" + +"@types/babel__template@*": + version "7.4.1" + resolved "https://registry.yarnpkg.com/@types/babel__template/-/babel__template-7.4.1.tgz#3d1a48fd9d6c0edfd56f2ff578daed48f36c8969" + integrity sha512-azBFKemX6kMg5Io+/rdGT0dkGreboUVR0Cdm3fz9QJWpaQGJRQXl7C+6hOTCZcMll7KFyEQpgbYI2lHdsS4U7g== + dependencies: + "@babel/parser" "^7.1.0" + "@babel/types" "^7.0.0" + +"@types/babel__traverse@*": + version "7.20.1" + resolved "https://registry.yarnpkg.com/@types/babel__traverse/-/babel__traverse-7.20.1.tgz#dd6f1d2411ae677dcb2db008c962598be31d6acf" + integrity sha512-MitHFXnhtgwsGZWtT68URpOvLN4EREih1u3QtQiN4VdAxWKRVvGCSvw/Qth0M0Qq3pJpnGOu5JaM/ydK7OGbqg== + dependencies: + "@babel/types" "^7.20.7" + +"@types/body-parser@*": + version "1.19.2" + resolved "https://registry.yarnpkg.com/@types/body-parser/-/body-parser-1.19.2.tgz#aea2059e28b7658639081347ac4fab3de166e6f0" + integrity sha512-ALYone6pm6QmwZoAgeyNksccT9Q4AWZQ6PvfwR37GT6r6FWUPguq6sUmNGSMV2Wr761oQoBxwGGa6DR5o1DC9g== + dependencies: + "@types/connect" "*" + "@types/node" "*" + +"@types/connect@*": + version "3.4.36" + resolved "https://registry.yarnpkg.com/@types/connect/-/connect-3.4.36.tgz#e511558c15a39cb29bd5357eebb57bd1459cd1ab" + integrity sha512-P63Zd/JUGq+PdrM1lv0Wv5SBYeA2+CORvbrXbngriYY0jzLUWfQMQQxOhjONEz/wlHOAxOdY7CY65rgQdTjq2w== + dependencies: + "@types/node" "*" + +"@types/cross-spawn@^6.0.2": + version "6.0.3" + resolved "https://registry.yarnpkg.com/@types/cross-spawn/-/cross-spawn-6.0.3.tgz#c743cb2608f55860ee9776d8c99135d6032c763c" + integrity sha512-BDAkU7WHHRHnvBf5z89lcvACsvkz/n7Tv+HyD/uW76O29HoH1Tk/W6iQrepaZVbisvlEek4ygwT8IW7ow9XLAA== + dependencies: + "@types/node" "*" + +"@types/d3-array@^3.0.3": + version "3.0.7" + resolved "https://registry.yarnpkg.com/@types/d3-array/-/d3-array-3.0.7.tgz#b128a0c0b0d9481d3281df47de0955730db384a1" + integrity sha512-4/Q0FckQ8TBjsB0VdGFemJOG8BLXUB2KKlL0VmZ+eOYeOnTb/wDRQqYWpBmQ6IlvWkXwkYiot+n9Px2aTJ7zGQ== + +"@types/d3-color@*": + version "3.1.0" + resolved "https://registry.yarnpkg.com/@types/d3-color/-/d3-color-3.1.0.tgz#6594da178ded6c7c3842f3cc0ac84b156f12f2d4" + integrity sha512-HKuicPHJuvPgCD+np6Se9MQvS6OCbJmOjGvylzMJRlDwUXjKTTXs6Pwgk79O09Vj/ho3u1ofXnhFOaEWWPrlwA== + +"@types/d3-ease@^3.0.0": + version "3.0.0" + resolved "https://registry.yarnpkg.com/@types/d3-ease/-/d3-ease-3.0.0.tgz#c29926f8b596f9dadaeca062a32a45365681eae0" + integrity sha512-aMo4eaAOijJjA6uU+GIeW018dvy9+oH5Y2VPPzjjfxevvGQ/oRDs+tfYC9b50Q4BygRR8yE2QCLsrT0WtAVseA== + +"@types/d3-interpolate@^3.0.1": + version "3.0.1" + resolved "https://registry.yarnpkg.com/@types/d3-interpolate/-/d3-interpolate-3.0.1.tgz#e7d17fa4a5830ad56fe22ce3b4fac8541a9572dc" + integrity sha512-jx5leotSeac3jr0RePOH1KdR9rISG91QIE4Q2PYTu4OymLTZfA3SrnURSLzKH48HmXVUru50b8nje4E79oQSQw== + dependencies: + "@types/d3-color" "*" + +"@types/d3-path@*": + version "3.0.0" + resolved "https://registry.yarnpkg.com/@types/d3-path/-/d3-path-3.0.0.tgz#939e3a784ae4f80b1fde8098b91af1776ff1312b" + integrity sha512-0g/A+mZXgFkQxN3HniRDbXMN79K3CdTpLsevj+PXiTcb2hVyvkZUBg37StmgCQkaD84cUJ4uaDAWq7UJOQy2Tg== + +"@types/d3-scale@^4.0.2": + version "4.0.4" + resolved "https://registry.yarnpkg.com/@types/d3-scale/-/d3-scale-4.0.4.tgz#3c5e2263eea5a3670cd91043b9f4d150a94c43f1" + integrity sha512-eq1ZeTj0yr72L8MQk6N6heP603ubnywSDRfNpi5enouR112HzGLS6RIvExCzZTraFF4HdzNpJMwA/zGiMoHUUw== + dependencies: + "@types/d3-time" "*" + +"@types/d3-shape@^3.1.0": + version "3.1.2" + resolved "https://registry.yarnpkg.com/@types/d3-shape/-/d3-shape-3.1.2.tgz#a3d421d8b0bc0c6c67cb3f4b4471ddc133cb0117" + integrity sha512-NN4CXr3qeOUNyK5WasVUV8NCSAx/CRVcwcb0BuuS1PiTqwIm6ABi1SyasLZ/vsVCFDArF+W4QiGzSry1eKYQ7w== + dependencies: + "@types/d3-path" "*" + +"@types/d3-time@*", "@types/d3-time@^3.0.0": + version "3.0.0" + resolved "https://registry.yarnpkg.com/@types/d3-time/-/d3-time-3.0.0.tgz#e1ac0f3e9e195135361fa1a1d62f795d87e6e819" + integrity sha512-sZLCdHvBUcNby1cB6Fd3ZBrABbjz3v1Vm90nysCQ6Vt7vd6e/h9Lt7SiJUoEX0l4Dzc7P5llKyhqSi1ycSf1Hg== + +"@types/d3-timer@^3.0.0": + version "3.0.0" + resolved "https://registry.yarnpkg.com/@types/d3-timer/-/d3-timer-3.0.0.tgz#e2505f1c21ec08bda8915238e397fb71d2fc54ce" + integrity sha512-HNB/9GHqu7Fo8AQiugyJbv6ZxYz58wef0esl4Mv828w1ZKpAshw/uFWVDUcIB9KKFeFKoxS3cHY07FFgtTRZ1g== + +"@types/detect-port@^1.3.0": + version "1.3.3" + resolved "https://registry.yarnpkg.com/@types/detect-port/-/detect-port-1.3.3.tgz#124c5d4c283f48a21f80826bcf39433b3e64aa81" + integrity sha512-bV/jQlAJ/nPY3XqSatkGpu+nGzou+uSwrH1cROhn+jBFg47yaNH+blW4C7p9KhopC7QxCv/6M86s37k8dMk0Yg== + +"@types/doctrine@^0.0.3": + version "0.0.3" + resolved "https://registry.yarnpkg.com/@types/doctrine/-/doctrine-0.0.3.tgz#e892d293c92c9c1d3f9af72c15a554fbc7e0895a" + integrity sha512-w5jZ0ee+HaPOaX25X2/2oGR/7rgAQSYII7X7pp0m9KgBfMP7uKfMfTvcpl5Dj+eDBbpxKGiqE+flqDr6XTd2RA== + +"@types/ejs@^3.1.1": + version "3.1.2" + resolved "https://registry.yarnpkg.com/@types/ejs/-/ejs-3.1.2.tgz#75d277b030bc11b3be38c807e10071f45ebc78d9" + integrity sha512-ZmiaE3wglXVWBM9fyVC17aGPkLo/UgaOjEiI2FXQfyczrCefORPxIe+2dVmnmk3zkVIbizjrlQzmPGhSYGXG5g== + +"@types/emscripten@^1.39.6": + version "1.39.7" + resolved "https://registry.yarnpkg.com/@types/emscripten/-/emscripten-1.39.7.tgz#3025183ea56e12bf4d096aadc48ce74ca051233d" + integrity sha512-tLqYV94vuqDrXh515F/FOGtBcRMTPGvVV1LzLbtYDcQmmhtpf/gLYf+hikBbQk8MzOHNz37wpFfJbYAuSn8HqA== + +"@types/escodegen@^0.0.6": + version "0.0.6" + resolved "https://registry.yarnpkg.com/@types/escodegen/-/escodegen-0.0.6.tgz#5230a9ce796e042cda6f086dbf19f22ea330659c" + integrity sha512-AjwI4MvWx3HAOaZqYsjKWyEObT9lcVV0Y0V8nXo6cXzN8ZiMxVhf6F3d/UNvXVGKrEzL/Dluc5p+y9GkzlTWig== + +"@types/eslint-scope@^3.7.3": + version "3.7.4" + resolved "https://registry.yarnpkg.com/@types/eslint-scope/-/eslint-scope-3.7.4.tgz#37fc1223f0786c39627068a12e94d6e6fc61de16" + integrity sha512-9K4zoImiZc3HlIp6AVUDE4CWYx22a+lhSZMYNpbjW04+YF0KWj4pJXnEMjdnFTiQibFFmElcsasJXDbdI/EPhA== + dependencies: + "@types/eslint" "*" + "@types/estree" "*" + +"@types/eslint@*": + version "8.44.2" + resolved "https://registry.yarnpkg.com/@types/eslint/-/eslint-8.44.2.tgz#0d21c505f98a89b8dd4d37fa162b09da6089199a" + integrity sha512-sdPRb9K6iL5XZOmBubg8yiFp5yS/JdUDQsq5e6h95km91MCYMuvp7mh1fjPEYUhvHepKpZOjnEaMBR4PxjWDzg== + dependencies: + "@types/estree" "*" + "@types/json-schema" "*" + +"@types/estree@*", "@types/estree@^1.0.0": + version "1.0.1" + resolved "https://registry.yarnpkg.com/@types/estree/-/estree-1.0.1.tgz#aa22750962f3bf0e79d753d3cc067f010c95f194" + integrity sha512-LG4opVs2ANWZ1TJoKc937iMmNstM/d0ae1vNbnBvBhqCSezgVUOzcLCqbI5elV8Vy6WKwKjaqR+zO9VKirBBCA== + +"@types/estree@^0.0.51": + version "0.0.51" + resolved "https://registry.yarnpkg.com/@types/estree/-/estree-0.0.51.tgz#cfd70924a25a3fd32b218e5e420e6897e1ac4f40" + integrity sha512-CuPgU6f3eT/XgKKPqKd/gLZV1Xmvf1a2R5POBOGQa6uv82xpls89HU5zKeVoyR8XzHd1RGNOlQlvUe3CFkjWNQ== + +"@types/express-serve-static-core@^4.17.33": + version "4.17.36" + resolved "https://registry.yarnpkg.com/@types/express-serve-static-core/-/express-serve-static-core-4.17.36.tgz#baa9022119bdc05a4adfe740ffc97b5f9360e545" + integrity sha512-zbivROJ0ZqLAtMzgzIUC4oNqDG9iF0lSsAqpOD9kbs5xcIM3dTiyuHvBc7R8MtWBp3AAWGaovJa+wzWPjLYW7Q== + dependencies: + "@types/node" "*" + "@types/qs" "*" + "@types/range-parser" "*" + "@types/send" "*" + +"@types/express@^4.7.0": + version "4.17.17" + resolved "https://registry.yarnpkg.com/@types/express/-/express-4.17.17.tgz#01d5437f6ef9cfa8668e616e13c2f2ac9a491ae4" + integrity sha512-Q4FmmuLGBG58btUnfS1c1r/NQdlp3DMfGDGig8WhfpA2YRUtEkxAjkZb0yvplJGYdF1fsQ81iMDcH24sSCNC/Q== + dependencies: + "@types/body-parser" "*" + "@types/express-serve-static-core" "^4.17.33" + "@types/qs" "*" + "@types/serve-static" "*" + +"@types/find-cache-dir@^3.2.1": + version "3.2.1" + resolved "https://registry.yarnpkg.com/@types/find-cache-dir/-/find-cache-dir-3.2.1.tgz#7b959a4b9643a1e6a1a5fe49032693cc36773501" + integrity sha512-frsJrz2t/CeGifcu/6uRo4b+SzAwT4NYCVPu1GN8IB9XTzrpPkGuV0tmh9mN+/L0PklAlsC3u5Fxt0ju00LXIw== + +"@types/graceful-fs@^4.1.3": + version "4.1.6" + resolved "https://registry.yarnpkg.com/@types/graceful-fs/-/graceful-fs-4.1.6.tgz#e14b2576a1c25026b7f02ede1de3b84c3a1efeae" + integrity sha512-Sig0SNORX9fdW+bQuTEovKj3uHcUL6LQKbCrrqb1X7J6/ReAbhCXRAhc+SMejhLELFj2QcyuxmUooZ4bt5ReSw== + dependencies: + "@types/node" "*" + +"@types/html-minifier-terser@^6.0.0": + version "6.1.0" + resolved "https://registry.yarnpkg.com/@types/html-minifier-terser/-/html-minifier-terser-6.1.0.tgz#4fc33a00c1d0c16987b1a20cf92d20614c55ac35" + integrity sha512-oh/6byDPnL1zeNXFrDXFLyZjkr1MsBG667IM792caf1L2UPOOMf65NFzjUH/ltyfwjAGfs1rsX1eftK0jC/KIg== + +"@types/http-errors@*": + version "2.0.1" + resolved "https://registry.yarnpkg.com/@types/http-errors/-/http-errors-2.0.1.tgz#20172f9578b225f6c7da63446f56d4ce108d5a65" + integrity sha512-/K3ds8TRAfBvi5vfjuz8y6+GiAYBZ0x4tXv1Av6CWBWn0IlADc+ZX9pMq7oU0fNQPnBwIZl3rmeLp6SBApbxSQ== + +"@types/istanbul-lib-coverage@*", "@types/istanbul-lib-coverage@^2.0.0", "@types/istanbul-lib-coverage@^2.0.1": + version "2.0.4" + resolved "https://registry.yarnpkg.com/@types/istanbul-lib-coverage/-/istanbul-lib-coverage-2.0.4.tgz#8467d4b3c087805d63580480890791277ce35c44" + integrity sha512-z/QT1XN4K4KYuslS23k62yDIDLwLFkzxOuMplDtObz0+y7VqJCaO2o+SPwHCvLFZh7xazvvoor2tA/hPz9ee7g== + +"@types/istanbul-lib-report@*": + version "3.0.0" + resolved "https://registry.yarnpkg.com/@types/istanbul-lib-report/-/istanbul-lib-report-3.0.0.tgz#c14c24f18ea8190c118ee7562b7ff99a36552686" + integrity sha512-plGgXAPfVKFoYfa9NpYDAkseG+g6Jr294RqeqcqDixSbU34MZVJRi/P+7Y8GDpzkEwLaGZZOpKIEmeVZNtKsrg== + dependencies: + "@types/istanbul-lib-coverage" "*" + +"@types/istanbul-reports@^3.0.0": + version "3.0.1" + resolved "https://registry.yarnpkg.com/@types/istanbul-reports/-/istanbul-reports-3.0.1.tgz#9153fe98bba2bd565a63add9436d6f0d7f8468ff" + integrity sha512-c3mAZEuK0lvBp8tmuL74XRKn1+y2dcwOUpH7x4WrF6gk1GIgiluDRgMYQtw2OFcBvAJWlt6ASU3tSqxp0Uu0Aw== + dependencies: + "@types/istanbul-lib-report" "*" + +"@types/json-schema@*", "@types/json-schema@^7.0.8", "@types/json-schema@^7.0.9": + version "7.0.12" + resolved "https://registry.yarnpkg.com/@types/json-schema/-/json-schema-7.0.12.tgz#d70faba7039d5fca54c83c7dbab41051d2b6f6cb" + integrity sha512-Hr5Jfhc9eYOQNPYO5WLDq/n4jqijdHNlDXjuAQkkt+mWdQR+XJToOHrsD4cPaMXpn6KO7y2+wM8AZEs8VpBLVA== + +"@types/json5@^0.0.29": + version "0.0.29" + resolved "https://registry.yarnpkg.com/@types/json5/-/json5-0.0.29.tgz#ee28707ae94e11d2b827bcbe5270bcea7f3e71ee" + integrity sha512-dRLjCWHYg4oaA77cxO64oO+7JwCwnIzkZPdrrC71jQmQtlhM556pwKo5bUzqvZndkVbeFLIIi+9TC40JNF5hNQ== + +"@types/lodash@^4.14.167": + version "4.14.197" + resolved "https://registry.yarnpkg.com/@types/lodash/-/lodash-4.14.197.tgz#e95c5ddcc814ec3e84c891910a01e0c8a378c54b" + integrity sha512-BMVOiWs0uNxHVlHBgzTIqJYmj+PgCo4euloGF+5m4okL3rEYzM2EEv78mw8zWSMM57dM7kVIgJ2QDvwHSoCI5g== + +"@types/mdx@^2.0.0": + version "2.0.7" + resolved "https://registry.yarnpkg.com/@types/mdx/-/mdx-2.0.7.tgz#c7482e995673e01b83f8e96df83b3843ea76401f" + integrity sha512-BG4tyr+4amr3WsSEmHn/fXPqaCba/AYZ7dsaQTiavihQunHSIxk+uAtqsjvicNpyHN6cm+B9RVrUOtW9VzIKHw== + +"@types/mime-types@^2.1.0": + version "2.1.1" + resolved "https://registry.yarnpkg.com/@types/mime-types/-/mime-types-2.1.1.tgz#d9ba43490fa3a3df958759adf69396c3532cf2c1" + integrity sha512-vXOTGVSLR2jMw440moWTC7H19iUyLtP3Z1YTj7cSsubOICinjMxFeb/V57v9QdyyPGbbWolUFSSmSiRSn94tFw== + +"@types/mime@*": + version "3.0.1" + resolved "https://registry.yarnpkg.com/@types/mime/-/mime-3.0.1.tgz#5f8f2bca0a5863cb69bc0b0acd88c96cb1d4ae10" + integrity sha512-Y4XFY5VJAuw0FgAqPNd6NNoV44jbq9Bz2L7Rh/J6jLTiHBSBJa9fxqQIvkIld4GsoDOcCbvzOUAbLPsSKKg+uA== + +"@types/mime@^1": + version "1.3.2" + resolved "https://registry.yarnpkg.com/@types/mime/-/mime-1.3.2.tgz#93e25bf9ee75fe0fd80b594bc4feb0e862111b5a" + integrity sha512-YATxVxgRqNH6nHEIsvg6k2Boc1JHI9ZbH5iWFFv/MTkchz3b1ieGDa5T0a9RznNdI0KhVbdbWSN+KWWrQZRxTw== + +"@types/node-fetch@^2.6.4": + version "2.6.4" + resolved "https://registry.yarnpkg.com/@types/node-fetch/-/node-fetch-2.6.4.tgz#1bc3a26de814f6bf466b25aeb1473fa1afe6a660" + integrity sha512-1ZX9fcN4Rvkvgv4E6PAY5WXUFWFcRWxZa3EW83UjycOB9ljJCedb2CupIP4RZMEwF/M3eTcCihbBRgwtGbg5Rg== + dependencies: + "@types/node" "*" + form-data "^3.0.0" + +"@types/node@*", "@types/node@>=12.12.47", "@types/node@>=13.7.0": + version "20.5.9" + resolved "https://registry.yarnpkg.com/@types/node/-/node-20.5.9.tgz#a70ec9d8fa0180a314c3ede0e20ea56ff71aed9a" + integrity sha512-PcGNd//40kHAS3sTlzKB9C9XL4K0sTup8nbG5lC14kzEteTNuAFh9u5nA0o5TWnSG2r/JNPRXFVcHJIIeRlmqQ== + +"@types/node@20.5.0": + version "20.5.0" + resolved "https://registry.yarnpkg.com/@types/node/-/node-20.5.0.tgz#7fc8636d5f1aaa3b21e6245e97d56b7f56702313" + integrity sha512-Mgq7eCtoTjT89FqNoTzzXg2XvCi5VMhRV6+I2aYanc6kQCBImeNaAYRs/DyoVqk1YEUJK5gN9VO7HRIdz4Wo3Q== + +"@types/node@^16.0.0": + version "16.18.48" + resolved "https://registry.yarnpkg.com/@types/node/-/node-16.18.48.tgz#3bc872236cdb31cb51024d8875d655e25db489a4" + integrity sha512-mlaecDKQ7rIZrYD7iiKNdzFb6e/qD5I9U1rAhq+Fd+DWvYVs+G2kv74UFHmSOlg5+i/vF3XxuR522V4u8BqO+Q== + +"@types/normalize-package-data@^2.4.0": + version "2.4.1" + resolved "https://registry.yarnpkg.com/@types/normalize-package-data/-/normalize-package-data-2.4.1.tgz#d3357479a0fdfdd5907fe67e17e0a85c906e1301" + integrity sha512-Gj7cI7z+98M282Tqmp2K5EIsoouUEzbBJhQQzDE3jSIRk6r9gsz0oUokqIUR4u1R3dMHo0pDHM7sNOHyhulypw== + +"@types/parse-json@^4.0.0": + version "4.0.0" + resolved "https://registry.yarnpkg.com/@types/parse-json/-/parse-json-4.0.0.tgz#2f8bb441434d163b35fb8ffdccd7138927ffb8c0" + integrity sha512-//oorEZjL6sbPcKUaCdIGlIUeH26mgzimjBB77G6XRgnDl/L5wOnpyBGRe/Mmf5CVW3PwEBE1NjiMZ/ssFh4wA== + +"@types/pretty-hrtime@^1.0.0": + version "1.0.1" + resolved "https://registry.yarnpkg.com/@types/pretty-hrtime/-/pretty-hrtime-1.0.1.tgz#72a26101dc567b0d68fd956cf42314556e42d601" + integrity sha512-VjID5MJb1eGKthz2qUerWT8+R4b9N+CHvGCzg9fn4kWZgaF9AhdYikQio3R7wV8YY1NsQKPaCwKz1Yff+aHNUQ== + +"@types/prop-types@*": + version "15.7.5" + resolved "https://registry.yarnpkg.com/@types/prop-types/-/prop-types-15.7.5.tgz#5f19d2b85a98e9558036f6a3cacc8819420f05cf" + integrity sha512-JCB8C6SnDoQf0cNycqd/35A7MjcnK+ZTqE7judS6o7utxUCg6imJg3QK2qzHKszlTjcj2cn+NwMB2i96ubpj7w== + +"@types/qs@*", "@types/qs@^6.9.5": + version "6.9.8" + resolved "https://registry.yarnpkg.com/@types/qs/-/qs-6.9.8.tgz#f2a7de3c107b89b441e071d5472e6b726b4adf45" + integrity sha512-u95svzDlTysU5xecFNTgfFG5RUWu1A9P0VzgpcIiGZA9iraHOdSzcxMxQ55DyeRaGCSxQi7LxXDI4rzq/MYfdg== + +"@types/range-parser@*": + version "1.2.4" + resolved "https://registry.yarnpkg.com/@types/range-parser/-/range-parser-1.2.4.tgz#cd667bcfdd025213aafb7ca5915a932590acdcdc" + integrity sha512-EEhsLsD6UsDM1yFhAvy0Cjr6VwmpMWqFBCb9w07wVugF7w9nfajxLuVmngTIpgS6svCnm6Vaw+MZhoDCKnOfsw== + +"@types/react-dom@18.2.7": + version "18.2.7" + resolved "https://registry.yarnpkg.com/@types/react-dom/-/react-dom-18.2.7.tgz#67222a08c0a6ae0a0da33c3532348277c70abb63" + integrity sha512-GRaAEriuT4zp9N4p1i8BDBYmEyfo+xQ3yHjJU4eiK5NDa1RmUZG+unZABUTK4/Ox/M+GaHwb6Ow8rUITrtjszA== + dependencies: + "@types/react" "*" + +"@types/react@*", "@types/react@>=16": + version "18.2.21" + resolved "https://registry.yarnpkg.com/@types/react/-/react-18.2.21.tgz#774c37fd01b522d0b91aed04811b58e4e0514ed9" + integrity sha512-neFKG/sBAwGxHgXiIxnbm3/AAVQ/cMRS93hvBpg8xYRbeQSPVABp9U2bRnPf0iI4+Ucdv3plSxKK+3CW2ENJxA== + dependencies: + "@types/prop-types" "*" + "@types/scheduler" "*" + csstype "^3.0.2" + +"@types/react@18.2.20": + version "18.2.20" + resolved "https://registry.yarnpkg.com/@types/react/-/react-18.2.20.tgz#1605557a83df5c8a2cc4eeb743b3dfc0eb6aaeb2" + integrity sha512-WKNtmsLWJM/3D5mG4U84cysVY31ivmyw85dE84fOCk5Hx78wezB/XEjVPWl2JTZ5FkEeaTJf+VgUAUn3PE7Isw== + dependencies: + "@types/prop-types" "*" + "@types/scheduler" "*" + csstype "^3.0.2" + +"@types/react@^16.14.34": + version "16.14.46" + resolved "https://registry.yarnpkg.com/@types/react/-/react-16.14.46.tgz#42ac91aece416176e6b6127cd9ec9e381ea67e16" + integrity sha512-Am4pyXMrr6cWWw/TN3oqHtEZl0j+G6Up/O8m65+xF/3ZaUgkv1GAtTPWw4yNRmH0HJXmur6xKCKoMo3rBGynuw== + dependencies: + "@types/prop-types" "*" + "@types/scheduler" "*" + csstype "^3.0.2" + +"@types/scheduler@*": + version "0.16.3" + resolved "https://registry.yarnpkg.com/@types/scheduler/-/scheduler-0.16.3.tgz#cef09e3ec9af1d63d2a6cc5b383a737e24e6dcf5" + integrity sha512-5cJ8CB4yAx7BH1oMvdU0Jh9lrEXyPkar6F9G/ERswkCuvP4KQZfZkSjcMbAICCpQTN4OuZn8tz0HiKv9TGZgrQ== + +"@types/semver@^7.3.12", "@types/semver@^7.3.4": + version "7.5.1" + resolved "https://registry.yarnpkg.com/@types/semver/-/semver-7.5.1.tgz#0480eeb7221eb9bc398ad7432c9d7e14b1a5a367" + integrity sha512-cJRQXpObxfNKkFAZbJl2yjWtJCqELQIdShsogr1d2MilP8dKD9TE/nEKHkJgUNHdGKCQaf9HbIynuV2csLGVLg== + +"@types/send@*": + version "0.17.1" + resolved "https://registry.yarnpkg.com/@types/send/-/send-0.17.1.tgz#ed4932b8a2a805f1fe362a70f4e62d0ac994e301" + integrity sha512-Cwo8LE/0rnvX7kIIa3QHCkcuF21c05Ayb0ZfxPiv0W8VRiZiNW/WuRupHKpqqGVGf7SUA44QSOUKaEd9lIrd/Q== + dependencies: + "@types/mime" "^1" + "@types/node" "*" + +"@types/serve-static@*": + version "1.15.2" + resolved "https://registry.yarnpkg.com/@types/serve-static/-/serve-static-1.15.2.tgz#3e5419ecd1e40e7405d34093f10befb43f63381a" + integrity sha512-J2LqtvFYCzaj8pVYKw8klQXrLLk7TBZmQ4ShlcdkELFKGwGMfevMLneMMRkMgZxotOD9wg497LpC7O8PcvAmfw== + dependencies: + "@types/http-errors" "*" + "@types/mime" "*" + "@types/node" "*" + +"@types/stylis@^4.0.2": + version "4.2.0" + resolved "https://registry.yarnpkg.com/@types/stylis/-/stylis-4.2.0.tgz#199a3f473f0c3a6f6e4e1b17cdbc967f274bdc6b" + integrity sha512-n4sx2bqL0mW1tvDf/loQ+aMX7GQD3lc3fkCMC55VFNDu/vBOabO+LTIeXKM14xK0ppk5TUGcWRjiSpIlUpghKw== + +"@types/unist@^2.0.0": + version "2.0.8" + resolved "https://registry.yarnpkg.com/@types/unist/-/unist-2.0.8.tgz#bb197b9639aa1a04cf464a617fe800cccd92ad5c" + integrity sha512-d0XxK3YTObnWVp6rZuev3c49+j4Lo8g4L1ZRm9z5L0xpoZycUPshHgczK5gsUMaZOstjVYYi09p5gYvUtfChYw== + +"@types/yargs-parser@*": + version "21.0.0" + resolved "https://registry.yarnpkg.com/@types/yargs-parser/-/yargs-parser-21.0.0.tgz#0c60e537fa790f5f9472ed2776c2b71ec117351b" + integrity sha512-iO9ZQHkZxHn4mSakYV0vFHAVDyEOIJQrV2uZ06HxEPcx+mt8swXoZHIbaaJ2crJYFfErySgktuTZ3BeLz+XmFA== + +"@types/yargs@^16.0.0": + version "16.0.5" + resolved "https://registry.yarnpkg.com/@types/yargs/-/yargs-16.0.5.tgz#12cc86393985735a283e387936398c2f9e5f88e3" + integrity sha512-AxO/ADJOBFJScHbWhq2xAhlWP24rY4aCEG/NFaMvbT3X2MgRsLjhjQwsn0Zi5zn0LG9jUhCCZMeX9Dkuw6k+vQ== + dependencies: + "@types/yargs-parser" "*" + +"@types/yargs@^17.0.8": + version "17.0.24" + resolved "https://registry.yarnpkg.com/@types/yargs/-/yargs-17.0.24.tgz#b3ef8d50ad4aa6aecf6ddc97c580a00f5aa11902" + integrity sha512-6i0aC7jV6QzQB8ne1joVZ0eSFIstHsCrobmOtghM11yGlH0j43FKL2UhWdELkyps0zuf7qVTUVCCR+tgSlyLLw== + dependencies: + "@types/yargs-parser" "*" + +"@typescript-eslint/parser@^5.4.2 || ^6.0.0": + version "6.6.0" + resolved "https://registry.yarnpkg.com/@typescript-eslint/parser/-/parser-6.6.0.tgz#fe323a7b4eafb6d5ea82b96216561810394a739e" + integrity sha512-setq5aJgUwtzGrhW177/i+DMLqBaJbdwGj2CPIVFFLE0NCliy5ujIdLHd2D1ysmlmsjdL2GWW+hR85neEfc12w== + dependencies: + "@typescript-eslint/scope-manager" "6.6.0" + "@typescript-eslint/types" "6.6.0" + "@typescript-eslint/typescript-estree" "6.6.0" + "@typescript-eslint/visitor-keys" "6.6.0" + debug "^4.3.4" + +"@typescript-eslint/scope-manager@5.62.0": + version "5.62.0" + resolved "https://registry.yarnpkg.com/@typescript-eslint/scope-manager/-/scope-manager-5.62.0.tgz#d9457ccc6a0b8d6b37d0eb252a23022478c5460c" + integrity sha512-VXuvVvZeQCQb5Zgf4HAxc04q5j+WrNAtNh9OwCsCgpKqESMTu3tF/jhZ3xG6T4NZwWl65Bg8KuS2uEvhSfLl0w== + dependencies: + "@typescript-eslint/types" "5.62.0" + "@typescript-eslint/visitor-keys" "5.62.0" + +"@typescript-eslint/scope-manager@6.6.0": + version "6.6.0" + resolved "https://registry.yarnpkg.com/@typescript-eslint/scope-manager/-/scope-manager-6.6.0.tgz#57105d4419d6de971f7d2c30a2ff4ac40003f61a" + integrity sha512-pT08u5W/GT4KjPUmEtc2kSYvrH8x89cVzkA0Sy2aaOUIw6YxOIjA8ilwLr/1fLjOedX1QAuBpG9XggWqIIfERw== + dependencies: + "@typescript-eslint/types" "6.6.0" + "@typescript-eslint/visitor-keys" "6.6.0" + +"@typescript-eslint/types@5.62.0": + version "5.62.0" + resolved "https://registry.yarnpkg.com/@typescript-eslint/types/-/types-5.62.0.tgz#258607e60effa309f067608931c3df6fed41fd2f" + integrity sha512-87NVngcbVXUahrRTqIK27gD2t5Cu1yuCXxbLcFtCzZGlfyVWWh8mLHkoxzjsB6DDNnvdL+fW8MiwPEJyGJQDgQ== + +"@typescript-eslint/types@6.6.0": + version "6.6.0" + resolved "https://registry.yarnpkg.com/@typescript-eslint/types/-/types-6.6.0.tgz#95e7ea650a2b28bc5af5ea8907114a48f54618c2" + integrity sha512-CB6QpJQ6BAHlJXdwUmiaXDBmTqIE2bzGTDLADgvqtHWuhfNP3rAOK7kAgRMAET5rDRr9Utt+qAzRBdu3AhR3sg== + +"@typescript-eslint/typescript-estree@5.62.0": + version "5.62.0" + resolved "https://registry.yarnpkg.com/@typescript-eslint/typescript-estree/-/typescript-estree-5.62.0.tgz#7d17794b77fabcac615d6a48fb143330d962eb9b" + integrity sha512-CmcQ6uY7b9y694lKdRB8FEel7JbU/40iSAPomu++SjLMntB+2Leay2LO6i8VnJk58MtE9/nQSFIH6jpyRWyYzA== + dependencies: + "@typescript-eslint/types" "5.62.0" + "@typescript-eslint/visitor-keys" "5.62.0" + debug "^4.3.4" + globby "^11.1.0" + is-glob "^4.0.3" + semver "^7.3.7" + tsutils "^3.21.0" + +"@typescript-eslint/typescript-estree@6.6.0": + version "6.6.0" + resolved "https://registry.yarnpkg.com/@typescript-eslint/typescript-estree/-/typescript-estree-6.6.0.tgz#373c420d2e12c28220f4a83352280a04823a91b7" + integrity sha512-hMcTQ6Al8MP2E6JKBAaSxSVw5bDhdmbCEhGW/V8QXkb9oNsFkA4SBuOMYVPxD3jbtQ4R/vSODBsr76R6fP3tbA== + dependencies: + "@typescript-eslint/types" "6.6.0" + "@typescript-eslint/visitor-keys" "6.6.0" + debug "^4.3.4" + globby "^11.1.0" + is-glob "^4.0.3" + semver "^7.5.4" + ts-api-utils "^1.0.1" + +"@typescript-eslint/utils@^5.45.0": + version "5.62.0" + resolved "https://registry.yarnpkg.com/@typescript-eslint/utils/-/utils-5.62.0.tgz#141e809c71636e4a75daa39faed2fb5f4b10df86" + integrity sha512-n8oxjeb5aIbPFEtmQxQYOLI0i9n5ySBEY/ZEHHZqKQSFnxio1rv6dthascc9dLuwrL0RC5mPCxB7vnAVGAYWAQ== + dependencies: + "@eslint-community/eslint-utils" "^4.2.0" + "@types/json-schema" "^7.0.9" + "@types/semver" "^7.3.12" + "@typescript-eslint/scope-manager" "5.62.0" + "@typescript-eslint/types" "5.62.0" + "@typescript-eslint/typescript-estree" "5.62.0" + eslint-scope "^5.1.1" + semver "^7.3.7" + +"@typescript-eslint/visitor-keys@5.62.0": + version "5.62.0" + resolved "https://registry.yarnpkg.com/@typescript-eslint/visitor-keys/-/visitor-keys-5.62.0.tgz#2174011917ce582875954ffe2f6912d5931e353e" + integrity sha512-07ny+LHRzQXepkGg6w0mFY41fVUNBrL2Roj/++7V1txKugfjm/Ci/qSND03r2RhlJhJYMcTn9AhhSSqQp0Ysyw== + dependencies: + "@typescript-eslint/types" "5.62.0" + eslint-visitor-keys "^3.3.0" + +"@typescript-eslint/visitor-keys@6.6.0": + version "6.6.0" + resolved "https://registry.yarnpkg.com/@typescript-eslint/visitor-keys/-/visitor-keys-6.6.0.tgz#1109088b4346c8b2446f3845db526374d9a3bafc" + integrity sha512-L61uJT26cMOfFQ+lMZKoJNbAEckLe539VhTxiGHrWl5XSKQgA0RTBZJW2HFPy5T0ZvPVSD93QsrTKDkfNwJGyQ== + dependencies: + "@typescript-eslint/types" "6.6.0" + eslint-visitor-keys "^3.4.1" + +"@webassemblyjs/ast@1.11.6", "@webassemblyjs/ast@^1.11.5": + version "1.11.6" + resolved "https://registry.yarnpkg.com/@webassemblyjs/ast/-/ast-1.11.6.tgz#db046555d3c413f8966ca50a95176a0e2c642e24" + integrity sha512-IN1xI7PwOvLPgjcf180gC1bqn3q/QaOCwYUahIOhbYUu8KA/3tw2RT/T0Gidi1l7Hhj5D/INhJxiICObqpMu4Q== + dependencies: + "@webassemblyjs/helper-numbers" "1.11.6" + "@webassemblyjs/helper-wasm-bytecode" "1.11.6" + +"@webassemblyjs/floating-point-hex-parser@1.11.6": + version "1.11.6" + resolved "https://registry.yarnpkg.com/@webassemblyjs/floating-point-hex-parser/-/floating-point-hex-parser-1.11.6.tgz#dacbcb95aff135c8260f77fa3b4c5fea600a6431" + integrity sha512-ejAj9hfRJ2XMsNHk/v6Fu2dGS+i4UaXBXGemOfQ/JfQ6mdQg/WXtwleQRLLS4OvfDhv8rYnVwH27YJLMyYsxhw== + +"@webassemblyjs/helper-api-error@1.11.6": + version "1.11.6" + resolved "https://registry.yarnpkg.com/@webassemblyjs/helper-api-error/-/helper-api-error-1.11.6.tgz#6132f68c4acd59dcd141c44b18cbebbd9f2fa768" + integrity sha512-o0YkoP4pVu4rN8aTJgAyj9hC2Sv5UlkzCHhxqWj8butaLvnpdc2jOwh4ewE6CX0txSfLn/UYaV/pheS2Txg//Q== + +"@webassemblyjs/helper-buffer@1.11.6": + version "1.11.6" + resolved "https://registry.yarnpkg.com/@webassemblyjs/helper-buffer/-/helper-buffer-1.11.6.tgz#b66d73c43e296fd5e88006f18524feb0f2c7c093" + integrity sha512-z3nFzdcp1mb8nEOFFk8DrYLpHvhKC3grJD2ardfKOzmbmJvEf/tPIqCY+sNcwZIY8ZD7IkB2l7/pqhUhqm7hLA== + +"@webassemblyjs/helper-numbers@1.11.6": + version "1.11.6" + resolved "https://registry.yarnpkg.com/@webassemblyjs/helper-numbers/-/helper-numbers-1.11.6.tgz#cbce5e7e0c1bd32cf4905ae444ef64cea919f1b5" + integrity sha512-vUIhZ8LZoIWHBohiEObxVm6hwP034jwmc9kuq5GdHZH0wiLVLIPcMCdpJzG4C11cHoQ25TFIQj9kaVADVX7N3g== + dependencies: + "@webassemblyjs/floating-point-hex-parser" "1.11.6" + "@webassemblyjs/helper-api-error" "1.11.6" + "@xtuc/long" "4.2.2" + +"@webassemblyjs/helper-wasm-bytecode@1.11.6": + version "1.11.6" + resolved "https://registry.yarnpkg.com/@webassemblyjs/helper-wasm-bytecode/-/helper-wasm-bytecode-1.11.6.tgz#bb2ebdb3b83aa26d9baad4c46d4315283acd51e9" + integrity sha512-sFFHKwcmBprO9e7Icf0+gddyWYDViL8bpPjJJl0WHxCdETktXdmtWLGVzoHbqUcY4Be1LkNfwTmXOJUFZYSJdA== + +"@webassemblyjs/helper-wasm-section@1.11.6": + version "1.11.6" + resolved "https://registry.yarnpkg.com/@webassemblyjs/helper-wasm-section/-/helper-wasm-section-1.11.6.tgz#ff97f3863c55ee7f580fd5c41a381e9def4aa577" + integrity sha512-LPpZbSOwTpEC2cgn4hTydySy1Ke+XEu+ETXuoyvuyezHO3Kjdu90KK95Sh9xTbmjrCsUwvWwCOQQNta37VrS9g== + dependencies: + "@webassemblyjs/ast" "1.11.6" + "@webassemblyjs/helper-buffer" "1.11.6" + "@webassemblyjs/helper-wasm-bytecode" "1.11.6" + "@webassemblyjs/wasm-gen" "1.11.6" + +"@webassemblyjs/ieee754@1.11.6": + version "1.11.6" + resolved "https://registry.yarnpkg.com/@webassemblyjs/ieee754/-/ieee754-1.11.6.tgz#bb665c91d0b14fffceb0e38298c329af043c6e3a" + integrity sha512-LM4p2csPNvbij6U1f19v6WR56QZ8JcHg3QIJTlSwzFcmx6WSORicYj6I63f9yU1kEUtrpG+kjkiIAkevHpDXrg== + dependencies: + "@xtuc/ieee754" "^1.2.0" + +"@webassemblyjs/leb128@1.11.6": + version "1.11.6" + resolved "https://registry.yarnpkg.com/@webassemblyjs/leb128/-/leb128-1.11.6.tgz#70e60e5e82f9ac81118bc25381a0b283893240d7" + integrity sha512-m7a0FhE67DQXgouf1tbN5XQcdWoNgaAuoULHIfGFIEVKA6tu/edls6XnIlkmS6FrXAquJRPni3ZZKjw6FSPjPQ== + dependencies: + "@xtuc/long" "4.2.2" + +"@webassemblyjs/utf8@1.11.6": + version "1.11.6" + resolved "https://registry.yarnpkg.com/@webassemblyjs/utf8/-/utf8-1.11.6.tgz#90f8bc34c561595fe156603be7253cdbcd0fab5a" + integrity sha512-vtXf2wTQ3+up9Zsg8sa2yWiQpzSsMyXj0qViVP6xKGCUT8p8YJ6HqI7l5eCnWx1T/FYdsv07HQs2wTFbbof/RA== + +"@webassemblyjs/wasm-edit@^1.11.5": + version "1.11.6" + resolved "https://registry.yarnpkg.com/@webassemblyjs/wasm-edit/-/wasm-edit-1.11.6.tgz#c72fa8220524c9b416249f3d94c2958dfe70ceab" + integrity sha512-Ybn2I6fnfIGuCR+Faaz7YcvtBKxvoLV3Lebn1tM4o/IAJzmi9AWYIPWpyBfU8cC+JxAO57bk4+zdsTjJR+VTOw== + dependencies: + "@webassemblyjs/ast" "1.11.6" + "@webassemblyjs/helper-buffer" "1.11.6" + "@webassemblyjs/helper-wasm-bytecode" "1.11.6" + "@webassemblyjs/helper-wasm-section" "1.11.6" + "@webassemblyjs/wasm-gen" "1.11.6" + "@webassemblyjs/wasm-opt" "1.11.6" + "@webassemblyjs/wasm-parser" "1.11.6" + "@webassemblyjs/wast-printer" "1.11.6" + +"@webassemblyjs/wasm-gen@1.11.6": + version "1.11.6" + resolved "https://registry.yarnpkg.com/@webassemblyjs/wasm-gen/-/wasm-gen-1.11.6.tgz#fb5283e0e8b4551cc4e9c3c0d7184a65faf7c268" + integrity sha512-3XOqkZP/y6B4F0PBAXvI1/bky7GryoogUtfwExeP/v7Nzwo1QLcq5oQmpKlftZLbT+ERUOAZVQjuNVak6UXjPA== + dependencies: + "@webassemblyjs/ast" "1.11.6" + "@webassemblyjs/helper-wasm-bytecode" "1.11.6" + "@webassemblyjs/ieee754" "1.11.6" + "@webassemblyjs/leb128" "1.11.6" + "@webassemblyjs/utf8" "1.11.6" + +"@webassemblyjs/wasm-opt@1.11.6": + version "1.11.6" + resolved "https://registry.yarnpkg.com/@webassemblyjs/wasm-opt/-/wasm-opt-1.11.6.tgz#d9a22d651248422ca498b09aa3232a81041487c2" + integrity sha512-cOrKuLRE7PCe6AsOVl7WasYf3wbSo4CeOk6PkrjS7g57MFfVUF9u6ysQBBODX0LdgSvQqRiGz3CXvIDKcPNy4g== + dependencies: + "@webassemblyjs/ast" "1.11.6" + "@webassemblyjs/helper-buffer" "1.11.6" + "@webassemblyjs/wasm-gen" "1.11.6" + "@webassemblyjs/wasm-parser" "1.11.6" + +"@webassemblyjs/wasm-parser@1.11.6", "@webassemblyjs/wasm-parser@^1.11.5": + version "1.11.6" + resolved "https://registry.yarnpkg.com/@webassemblyjs/wasm-parser/-/wasm-parser-1.11.6.tgz#bb85378c527df824004812bbdb784eea539174a1" + integrity sha512-6ZwPeGzMJM3Dqp3hCsLgESxBGtT/OeCvCZ4TA1JUPYgmhAx38tTPR9JaKy0S5H3evQpO/h2uWs2j6Yc/fjkpTQ== + dependencies: + "@webassemblyjs/ast" "1.11.6" + "@webassemblyjs/helper-api-error" "1.11.6" + "@webassemblyjs/helper-wasm-bytecode" "1.11.6" + "@webassemblyjs/ieee754" "1.11.6" + "@webassemblyjs/leb128" "1.11.6" + "@webassemblyjs/utf8" "1.11.6" + +"@webassemblyjs/wast-printer@1.11.6": + version "1.11.6" + resolved "https://registry.yarnpkg.com/@webassemblyjs/wast-printer/-/wast-printer-1.11.6.tgz#a7bf8dd7e362aeb1668ff43f35cb849f188eff20" + integrity sha512-JM7AhRcE+yW2GWYaKeHL5vt4xqee5N2WcezptmgyhNS+ScggqcT1OtXykhAb13Sn5Yas0j2uv9tHgrjwvzAP4A== + dependencies: + "@webassemblyjs/ast" "1.11.6" + "@xtuc/long" "4.2.2" + +"@xtuc/ieee754@^1.2.0": + version "1.2.0" + resolved "https://registry.yarnpkg.com/@xtuc/ieee754/-/ieee754-1.2.0.tgz#eef014a3145ae477a1cbc00cd1e552336dceb790" + integrity sha512-DX8nKgqcGwsc0eJSqYt5lwP4DH5FlHnmuWWBRy7X0NcaGR0ZtuyeESgMwTYVEtxmsNGY+qit4QYT/MIYTOTPeA== + +"@xtuc/long@4.2.2": + version "4.2.2" + resolved "https://registry.yarnpkg.com/@xtuc/long/-/long-4.2.2.tgz#d291c6a4e97989b5c61d9acf396ae4fe133a718d" + integrity sha512-NuHqBY1PB/D8xU6s/thBgOAiAP7HOYDQ32+BFZILJ8ivkUkAHQnWfn6WhL79Owj1qmUnoN/YPhktdIoucipkAQ== + +"@yarnpkg/esbuild-plugin-pnp@^3.0.0-rc.10": + version "3.0.0-rc.15" + resolved "https://registry.yarnpkg.com/@yarnpkg/esbuild-plugin-pnp/-/esbuild-plugin-pnp-3.0.0-rc.15.tgz#4e40e7d2eb28825c9a35ab9d04c363931d7c0e67" + integrity sha512-kYzDJO5CA9sy+on/s2aIW0411AklfCi8Ck/4QDivOqsMKpStZA2SsR+X27VTggGwpStWaLrjJcDcdDMowtG8MA== + dependencies: + tslib "^2.4.0" + +"@yarnpkg/fslib@2.10.3": + version "2.10.3" + resolved "https://registry.yarnpkg.com/@yarnpkg/fslib/-/fslib-2.10.3.tgz#a8c9893df5d183cf6362680b9f1c6d7504dd5717" + integrity sha512-41H+Ga78xT9sHvWLlFOZLIhtU6mTGZ20pZ29EiZa97vnxdohJD2AF42rCoAoWfqUz486xY6fhjMH+DYEM9r14A== + dependencies: + "@yarnpkg/libzip" "^2.3.0" + tslib "^1.13.0" + +"@yarnpkg/libzip@2.3.0", "@yarnpkg/libzip@^2.3.0": + version "2.3.0" + resolved "https://registry.yarnpkg.com/@yarnpkg/libzip/-/libzip-2.3.0.tgz#fe1e762e47669f6e2c960fc118436608d834e3be" + integrity sha512-6xm38yGVIa6mKm/DUCF2zFFJhERh/QWp1ufm4cNUvxsONBmfPg8uZ9pZBdOmF6qFGr/HlT6ABBkCSx/dlEtvWg== + dependencies: + "@types/emscripten" "^1.39.6" + tslib "^1.13.0" + +abort-controller@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/abort-controller/-/abort-controller-3.0.0.tgz#eaf54d53b62bae4138e809ca225c8439a6efb392" + integrity sha512-h8lQ8tacZYnR3vNQTgibj+tODHI5/+l06Au2Pcriv/Gmet0eaj4TwWH41sO9wnHDiQsEj19q0drzdWdeAHtweg== + dependencies: + event-target-shim "^5.0.0" + +accepts@~1.3.5, accepts@~1.3.8: + version "1.3.8" + resolved "https://registry.yarnpkg.com/accepts/-/accepts-1.3.8.tgz#0bf0be125b67014adcb0b0921e62db7bffe16b2e" + integrity sha512-PYAthTa2m2VKxuvSD3DPC/Gy+U+sOA1LAuT8mkmRuvw+NACSaeXEQ+NHcVF7rONl6qcaxV3Uuemwawk+7+SJLw== + dependencies: + mime-types "~2.1.34" + negotiator "0.6.3" + +acorn-import-assertions@^1.9.0: + version "1.9.0" + resolved "https://registry.yarnpkg.com/acorn-import-assertions/-/acorn-import-assertions-1.9.0.tgz#507276249d684797c84e0734ef84860334cfb1ac" + integrity sha512-cmMwop9x+8KFhxvKrKfPYmN6/pKTYYHBqLa0DfvVZcKMJWNyWLnaqND7dx/qn66R7ewM1UX5XMaDVP5wlVTaVA== + +acorn-jsx@^5.3.1, acorn-jsx@^5.3.2: + version "5.3.2" + resolved "https://registry.yarnpkg.com/acorn-jsx/-/acorn-jsx-5.3.2.tgz#7ed5bb55908b3b2f1bc55c6af1653bada7f07937" + integrity sha512-rq9s+JNhf0IChjtDXxllJ7g41oZk5SlXtp0LHwyA5cejwn7vKmKp4pPri6YEePv2PU65sAsegbXtIinmDFDXgQ== + +acorn-walk@^7.2.0: + version "7.2.0" + resolved "https://registry.yarnpkg.com/acorn-walk/-/acorn-walk-7.2.0.tgz#0de889a601203909b0fbe07b8938dc21d2e967bc" + integrity sha512-OPdCF6GsMIP+Az+aWfAAOEt2/+iVDKE7oy6lJ098aoe59oAmK76qV6Gw60SbZ8jHuG2wH058GF4pLFbYamYrVA== + +acorn@^7.4.1: + version "7.4.1" + resolved "https://registry.yarnpkg.com/acorn/-/acorn-7.4.1.tgz#feaed255973d2e77555b83dbc08851a6c63520fa" + integrity sha512-nQyp0o1/mNdbTO1PO6kHkwSrmgZ0MT/jCCpNiwbUjGoRN4dlBhqJtoQuCnEOKzgTVwg0ZWiCoQy6SxMebQVh8A== + +acorn@^8.7.1, acorn@^8.8.2, acorn@^8.9.0: + version "8.10.0" + resolved "https://registry.yarnpkg.com/acorn/-/acorn-8.10.0.tgz#8be5b3907a67221a81ab23c7889c4c5526b62ec5" + integrity sha512-F0SAmZ8iUtS//m8DmCTA0jlh6TDKkHQyK6xc6V4KDTyZKA9dnvX9/3sRTVQrWm79glUAZbnmmNcdYwUIHWVybw== + +address@^1.0.1: + version "1.2.2" + resolved "https://registry.yarnpkg.com/address/-/address-1.2.2.tgz#2b5248dac5485a6390532c6a517fda2e3faac89e" + integrity sha512-4B/qKCfeE/ODUaAUpSwfzazo5x29WD4r3vXiWsB7I2mSDAihwEqKO+g8GELZUQSSAo5e1XTYh3ZVfLyxBc12nA== + +adjust-sourcemap-loader@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/adjust-sourcemap-loader/-/adjust-sourcemap-loader-4.0.0.tgz#fc4a0fd080f7d10471f30a7320f25560ade28c99" + integrity sha512-OXwN5b9pCUXNQHJpwwD2qP40byEmSgzj8B4ydSN0uMNYWiFmJ6x6KwUllMmfk8Rwu/HJDFR7U8ubsWBoN0Xp0A== + dependencies: + loader-utils "^2.0.0" + regex-parser "^2.2.11" + +agent-base@5: + version "5.1.1" + resolved "https://registry.yarnpkg.com/agent-base/-/agent-base-5.1.1.tgz#e8fb3f242959db44d63be665db7a8e739537a32c" + integrity sha512-TMeqbNl2fMW0nMjTEPOwe3J/PRFP4vqeoNuQMG0HlMrtm5QxKqdvAkZ1pRBQ/ulIyDD5Yq0nJ7YbdD8ey0TO3g== + +agent-base@6: + version "6.0.2" + resolved "https://registry.yarnpkg.com/agent-base/-/agent-base-6.0.2.tgz#49fff58577cfee3f37176feab4c22e00f86d7f77" + integrity sha512-RZNwNclF7+MS/8bDg70amg32dyeZGZxiDuQmZxKLAlQjr3jGyLx+4Kkk58UO7D2QdgFIQCovuSuZESne6RG6XQ== + dependencies: + debug "4" + +aggregate-error@^3.0.0: + version "3.1.0" + resolved "https://registry.yarnpkg.com/aggregate-error/-/aggregate-error-3.1.0.tgz#92670ff50f5359bdb7a3e0d40d0ec30c5737687a" + integrity sha512-4I7Td01quW/RpocfNayFdFVk1qSuoh0E7JrbRJ16nH01HhKFQ88INq9Sd+nd72zqRySlr9BmDA8xlEJ6vJMrYA== + dependencies: + clean-stack "^2.0.0" + indent-string "^4.0.0" + +ajv-formats@^2.1.1: + version "2.1.1" + resolved "https://registry.yarnpkg.com/ajv-formats/-/ajv-formats-2.1.1.tgz#6e669400659eb74973bbf2e33327180a0996b520" + integrity sha512-Wx0Kx52hxE7C18hkMEggYlEifqWZtYaRgouJor+WMdPnQyEK13vgEWyVNup7SoeeoLMsr4kf5h6dOW11I15MUA== + dependencies: + ajv "^8.0.0" + +ajv-keywords@^3.5.2: + version "3.5.2" + resolved "https://registry.yarnpkg.com/ajv-keywords/-/ajv-keywords-3.5.2.tgz#31f29da5ab6e00d1c2d329acf7b5929614d5014d" + integrity sha512-5p6WTN0DdTGVQk6VjcEju19IgaHudalcfabD7yhDGeA6bcQnmL+CpveLJq/3hvfwd1aof6L386Ougkx6RfyMIQ== + +ajv-keywords@^5.1.0: + version "5.1.0" + resolved "https://registry.yarnpkg.com/ajv-keywords/-/ajv-keywords-5.1.0.tgz#69d4d385a4733cdbeab44964a1170a88f87f0e16" + integrity sha512-YCS/JNFAUyr5vAuhk1DWm1CBxRHW9LbJ2ozWeemrIqpbsqKjHVxYPyi5GC0rjZIT5JxJ3virVTS8wk4i/Z+krw== + dependencies: + fast-deep-equal "^3.1.3" + +ajv@^6.12.4, ajv@^6.12.5: + version "6.12.6" + resolved "https://registry.yarnpkg.com/ajv/-/ajv-6.12.6.tgz#baf5a62e802b07d977034586f8c3baf5adf26df4" + integrity sha512-j3fVLgvTo527anyYyJOGTYJbG+vnnQYvE0m5mmkc1TK+nxAppkCLMIL0aZ4dblVCNoGShhm+kzE4ZUykBoMg4g== + dependencies: + fast-deep-equal "^3.1.1" + fast-json-stable-stringify "^2.0.0" + json-schema-traverse "^0.4.1" + uri-js "^4.2.2" + +ajv@^8.0.0, ajv@^8.9.0: + version "8.12.0" + resolved "https://registry.yarnpkg.com/ajv/-/ajv-8.12.0.tgz#d1a0527323e22f53562c567c00991577dfbe19d1" + integrity sha512-sRu1kpcO9yLtYxBKvqfTeh9KzZEwO3STyX1HT+4CaDzC6HpTGYhIhPIzj9XuKU7KYDwnaeh5hcOwjy1QuJzBPA== + dependencies: + fast-deep-equal "^3.1.1" + json-schema-traverse "^1.0.0" + require-from-string "^2.0.2" + uri-js "^4.2.2" + +ansi-html-community@0.0.8, ansi-html-community@^0.0.8: + version "0.0.8" + resolved "https://registry.yarnpkg.com/ansi-html-community/-/ansi-html-community-0.0.8.tgz#69fbc4d6ccbe383f9736934ae34c3f8290f1bf41" + integrity sha512-1APHAyr3+PCamwNw3bXCPp4HFLONZt/yIH0sZp0/469KWNTEy+qN5jQ3GVX6DMZ1UXAi34yVwtTeaG/HpBuuzw== + +ansi-regex@^5.0.1: + version "5.0.1" + resolved "https://registry.yarnpkg.com/ansi-regex/-/ansi-regex-5.0.1.tgz#082cb2c89c9fe8659a311a53bd6a4dc5301db304" + integrity sha512-quJQXlTSUGL2LH9SUXo8VwsY4soanhgo6LNSm84E1LBcE8s3O0wpdiRzyR9z/ZZJMlMWv37qOOb9pdJlMUEKFQ== + +ansi-regex@^6.0.1: + version "6.0.1" + resolved "https://registry.yarnpkg.com/ansi-regex/-/ansi-regex-6.0.1.tgz#3183e38fae9a65d7cb5e53945cd5897d0260a06a" + integrity sha512-n5M855fKb2SsfMIiFFoVrABHJC8QtHwVx+mHWP3QcEqBHYienj5dHSgjbxtC0WEZXYt4wcD6zrQElDPhFuZgfA== + +ansi-styles@^3.2.1: + version "3.2.1" + resolved "https://registry.yarnpkg.com/ansi-styles/-/ansi-styles-3.2.1.tgz#41fbb20243e50b12be0f04b8dedbf07520ce841d" + integrity sha512-VT0ZI6kZRdTh8YyJw3SMbYm/u+NqfsAxEpWO0Pf9sq8/e94WxxOpPKx9FR1FlyCtOVDNOQ+8ntlqFxiRc+r5qA== + dependencies: + color-convert "^1.9.0" + +ansi-styles@^4.0.0, ansi-styles@^4.1.0: + version "4.3.0" + resolved "https://registry.yarnpkg.com/ansi-styles/-/ansi-styles-4.3.0.tgz#edd803628ae71c04c85ae7a0906edad34b648937" + integrity sha512-zbB9rCJAT1rbjiVDb2hqKFHNYLxgtk8NURxZ3IZwD3F6NtxbXZQCnnSi1Lkx+IDohdPlFp222wVALIheZJQSEg== + dependencies: + color-convert "^2.0.1" + +ansi-styles@^5.0.0: + version "5.2.0" + resolved "https://registry.yarnpkg.com/ansi-styles/-/ansi-styles-5.2.0.tgz#07449690ad45777d1924ac2abb2fc8895dba836b" + integrity sha512-Cxwpt2SfTzTtXcfOlzGEee8O+c+MmUgGrNiBcXnuWxuFJHe6a5Hz7qwhwe5OgaSYI0IJvkLqWX1ASG+cJOkEiA== + +ansi-styles@^6.1.0: + version "6.2.1" + resolved "https://registry.yarnpkg.com/ansi-styles/-/ansi-styles-6.2.1.tgz#0e62320cf99c21afff3b3012192546aacbfb05c5" + integrity sha512-bN798gFfQX+viw3R7yrGWRqnrN2oRkEkUjjl4JNn4E8GxxbjtG3FbrEIIY3l8/hrwUwIeCZvi4QuOTP4MErVug== + +any-promise@^1.0.0: + version "1.3.0" + resolved "https://registry.yarnpkg.com/any-promise/-/any-promise-1.3.0.tgz#abc6afeedcea52e809cdc0376aed3ce39635d17f" + integrity sha512-7UvmKalWRt1wgjL1RrGxoSJW/0QZFIegpeGvZG9kjp8vrRu55XTHbwnqq2GpXm9uLbcuhxm3IqX9OB4MZR1b2A== + +anymatch@^3.0.3, anymatch@~3.1.2: + version "3.1.3" + resolved "https://registry.yarnpkg.com/anymatch/-/anymatch-3.1.3.tgz#790c58b19ba1720a84205b57c618d5ad8524973e" + integrity sha512-KMReFUr0B4t+D+OBkjR3KYqvocp2XaSzO55UcB6mgQMd3KbcE+mWTyvVV7D/zsdEbNnV6acZUutkiHQXvTr1Rw== + dependencies: + normalize-path "^3.0.0" + picomatch "^2.0.4" + +app-root-dir@^1.0.2: + version "1.0.2" + resolved "https://registry.yarnpkg.com/app-root-dir/-/app-root-dir-1.0.2.tgz#38187ec2dea7577fff033ffcb12172692ff6e118" + integrity sha512-jlpIfsOoNoafl92Sz//64uQHGSyMrD2vYG5d8o2a4qGvyNCvXur7bzIsWtAC/6flI2RYAp3kv8rsfBtaLm7w0g== + +arg@^5.0.2: + version "5.0.2" + resolved "https://registry.yarnpkg.com/arg/-/arg-5.0.2.tgz#c81433cc427c92c4dcf4865142dbca6f15acd59c" + integrity sha512-PYjyFOLKQ9y57JvQ6QLo8dAgNqswh8M1RMJYdQduT6xbWSgK36P/Z/v+p888pM69jMMfS8Xd8F6I1kQ/I9HUGg== + +argparse@^1.0.7: + version "1.0.10" + resolved "https://registry.yarnpkg.com/argparse/-/argparse-1.0.10.tgz#bcd6791ea5ae09725e17e5ad988134cd40b3d911" + integrity sha512-o5Roy6tNG4SL/FOkCAN6RzjiakZS25RLYFrcMttJqbdd8BWrnA+fGz57iN5Pb06pvBGvl5gQ0B48dJlslXvoTg== + dependencies: + sprintf-js "~1.0.2" + +argparse@^2.0.1: + version "2.0.1" + resolved "https://registry.yarnpkg.com/argparse/-/argparse-2.0.1.tgz#246f50f3ca78a3240f6c997e8a9bd1eac49e4b38" + integrity sha512-8+9WqebbFzpX9OR+Wa6O29asIogeRMzcGtAINdpMHHyAg10f05aSFVBbcEqGf/PXw1EjAZ+q2/bEBg3DvurK3Q== + +aria-hidden@^1.1.1, aria-hidden@^1.1.3: + version "1.2.3" + resolved "https://registry.yarnpkg.com/aria-hidden/-/aria-hidden-1.2.3.tgz#14aeb7fb692bbb72d69bebfa47279c1fd725e954" + integrity sha512-xcLxITLe2HYa1cnYnwCjkOO1PqUHQpozB8x9AR0OgWN2woOBi5kSDVxKfd0b7sb1hw5qFeJhXm9H1nu3xSfLeQ== + dependencies: + tslib "^2.0.0" + +aria-query@5.1.3: + version "5.1.3" + resolved "https://registry.yarnpkg.com/aria-query/-/aria-query-5.1.3.tgz#19db27cd101152773631396f7a95a3b58c22c35e" + integrity sha512-R5iJ5lkuHybztUfuOAznmboyjWq8O6sqNqtK7CLOqdydi54VNbORp49mb14KbWgG1QD3JFO9hJdZ+y4KutfdOQ== + dependencies: + deep-equal "^2.0.5" + +aria-query@^5.1.3: + version "5.3.0" + resolved "https://registry.yarnpkg.com/aria-query/-/aria-query-5.3.0.tgz#650c569e41ad90b51b3d7df5e5eed1c7549c103e" + integrity sha512-b0P0sZPKtyu8HkeRAfCq0IfURZK+SuwMjY1UXGBU27wpAiTwQAIlq56IbIO+ytk/JjS1fMR14ee5WBBfKi5J6A== + dependencies: + dequal "^2.0.3" + +array-buffer-byte-length@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/array-buffer-byte-length/-/array-buffer-byte-length-1.0.0.tgz#fabe8bc193fea865f317fe7807085ee0dee5aead" + integrity sha512-LPuwb2P+NrQw3XhxGc36+XSvuBPopovXYTR9Ew++Du9Yb/bx5AzBfrIsBoj0EZUifjQU+sHL21sseZ3jerWO/A== + dependencies: + call-bind "^1.0.2" + is-array-buffer "^3.0.1" + +array-flatten@1.1.1: + version "1.1.1" + resolved "https://registry.yarnpkg.com/array-flatten/-/array-flatten-1.1.1.tgz#9a5f699051b1e7073328f2a008968b64ea2955d2" + integrity sha512-PCVAQswWemu6UdxsDFFX/+gVeYqKAod3D3UVm91jHwynguOwAvYPhx8nNlM++NqRcK6CxxpUafjmhIdKiHibqg== + +array-includes@^3.1.6: + version "3.1.7" + resolved "https://registry.yarnpkg.com/array-includes/-/array-includes-3.1.7.tgz#8cd2e01b26f7a3086cbc87271593fe921c62abda" + integrity sha512-dlcsNBIiWhPkHdOEEKnehA+RNUWDc4UqFtnIXU4uuYDPtA4LDkr7qip2p0VvFAEXNDr0yWZ9PJyIRiGjRLQzwQ== + dependencies: + call-bind "^1.0.2" + define-properties "^1.2.0" + es-abstract "^1.22.1" + get-intrinsic "^1.2.1" + is-string "^1.0.7" + +array-union@^1.0.1: + version "1.0.2" + resolved "https://registry.yarnpkg.com/array-union/-/array-union-1.0.2.tgz#9a34410e4f4e3da23dea375be5be70f24778ec39" + integrity sha512-Dxr6QJj/RdU/hCaBjOfxW+q6lyuVE6JFWIrAUpuOOhoJJoQ99cUn3igRaHVB5P9WrgFVN0FfArM3x0cueOU8ng== + dependencies: + array-uniq "^1.0.1" + +array-union@^2.1.0: + version "2.1.0" + resolved "https://registry.yarnpkg.com/array-union/-/array-union-2.1.0.tgz#b798420adbeb1de828d84acd8a2e23d3efe85e8d" + integrity sha512-HGyxoOTYUyCM6stUe6EJgnd4EoewAI7zMdfqO+kGjnlZmBDz/cR5pf8r/cR4Wq60sL/p0IkcjUEEPwS3GFrIyw== + +array-uniq@^1.0.1: + version "1.0.3" + resolved "https://registry.yarnpkg.com/array-uniq/-/array-uniq-1.0.3.tgz#af6ac877a25cc7f74e058894753858dfdb24fdb6" + integrity sha512-MNha4BWQ6JbwhFhj03YK552f7cb3AzoE8SzeljgChvL1dl3IcvggXVz1DilzySZkCja+CXuZbdW7yATchWn8/Q== + +array.prototype.findlastindex@^1.2.2: + version "1.2.3" + resolved "https://registry.yarnpkg.com/array.prototype.findlastindex/-/array.prototype.findlastindex-1.2.3.tgz#b37598438f97b579166940814e2c0493a4f50207" + integrity sha512-LzLoiOMAxvy+Gd3BAq3B7VeIgPdo+Q8hthvKtXybMvRV0jrXfJM/t8mw7nNlpEcVlVUnCnM2KSX4XU5HmpodOA== + dependencies: + call-bind "^1.0.2" + define-properties "^1.2.0" + es-abstract "^1.22.1" + es-shim-unscopables "^1.0.0" + get-intrinsic "^1.2.1" + +array.prototype.flat@^1.3.1: + version "1.3.1" + resolved "https://registry.yarnpkg.com/array.prototype.flat/-/array.prototype.flat-1.3.1.tgz#ffc6576a7ca3efc2f46a143b9d1dda9b4b3cf5e2" + integrity sha512-roTU0KWIOmJ4DRLmwKd19Otg0/mT3qPNt0Qb3GWW8iObuZXxrjB/pzn0R3hqpRSWg4HCwqx+0vwOnWnvlOyeIA== + dependencies: + call-bind "^1.0.2" + define-properties "^1.1.4" + es-abstract "^1.20.4" + es-shim-unscopables "^1.0.0" + +array.prototype.flatmap@^1.3.1: + version "1.3.1" + resolved "https://registry.yarnpkg.com/array.prototype.flatmap/-/array.prototype.flatmap-1.3.1.tgz#1aae7903c2100433cb8261cd4ed310aab5c4a183" + integrity sha512-8UGn9O1FDVvMNB0UlLv4voxRMze7+FpHyF5mSMRjWHUMlpoDViniy05870VlxhfgTnLbpuwTzvD76MTtWxB/mQ== + dependencies: + call-bind "^1.0.2" + define-properties "^1.1.4" + es-abstract "^1.20.4" + es-shim-unscopables "^1.0.0" + +array.prototype.tosorted@^1.1.1: + version "1.1.1" + resolved "https://registry.yarnpkg.com/array.prototype.tosorted/-/array.prototype.tosorted-1.1.1.tgz#ccf44738aa2b5ac56578ffda97c03fd3e23dd532" + integrity sha512-pZYPXPRl2PqWcsUs6LOMn+1f1532nEoPTYowBtqLwAW+W8vSVhkIGnmOX1t/UQjD6YGI0vcD2B1U7ZFGQH9jnQ== + dependencies: + call-bind "^1.0.2" + define-properties "^1.1.4" + es-abstract "^1.20.4" + es-shim-unscopables "^1.0.0" + get-intrinsic "^1.1.3" + +arraybuffer.prototype.slice@^1.0.1: + version "1.0.1" + resolved "https://registry.yarnpkg.com/arraybuffer.prototype.slice/-/arraybuffer.prototype.slice-1.0.1.tgz#9b5ea3868a6eebc30273da577eb888381c0044bb" + integrity sha512-09x0ZWFEjj4WD8PDbykUwo3t9arLn8NIzmmYEJFpYekOAQjpkGSyrQhNoRTcwwcFRu+ycWF78QZ63oWTqSjBcw== + dependencies: + array-buffer-byte-length "^1.0.0" + call-bind "^1.0.2" + define-properties "^1.2.0" + get-intrinsic "^1.2.1" + is-array-buffer "^3.0.2" + is-shared-array-buffer "^1.0.2" + +asn1.js@^5.2.0: + version "5.4.1" + resolved "https://registry.yarnpkg.com/asn1.js/-/asn1.js-5.4.1.tgz#11a980b84ebb91781ce35b0fdc2ee294e3783f07" + integrity sha512-+I//4cYPccV8LdmBLiX8CYvf9Sp3vQsrqu2QNXRcrbiWvcx/UdlFiqUJJzxRQxgsZmvhXhn4cSKeSmoFjVdupA== + dependencies: + bn.js "^4.0.0" + inherits "^2.0.1" + minimalistic-assert "^1.0.0" + safer-buffer "^2.1.0" + +assert@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/assert/-/assert-2.0.0.tgz#95fc1c616d48713510680f2eaf2d10dd22e02d32" + integrity sha512-se5Cd+js9dXJnu6Ag2JFc00t+HmHOen+8Q+L7O9zI0PqQXr20uk2J0XQqMxZEeo5U50o8Nvmmx7dZrl+Ufr35A== + dependencies: + es6-object-assign "^1.1.0" + is-nan "^1.2.1" + object-is "^1.0.1" + util "^0.12.0" + +ast-types-flow@^0.0.7: + version "0.0.7" + resolved "https://registry.yarnpkg.com/ast-types-flow/-/ast-types-flow-0.0.7.tgz#f70b735c6bca1a5c9c22d982c3e39e7feba3bdad" + integrity sha512-eBvWn1lvIApYMhzQMsu9ciLfkBY499mFZlNqG+/9WR7PVlroQw0vG30cOQQbaKz3sCEc44TAOu2ykzqXSNnwag== + +ast-types@0.15.2: + version "0.15.2" + resolved "https://registry.yarnpkg.com/ast-types/-/ast-types-0.15.2.tgz#39ae4809393c4b16df751ee563411423e85fb49d" + integrity sha512-c27loCv9QkZinsa5ProX751khO9DJl/AcB5c2KNtA6NRvHKS0PgLfcftz72KVq504vB0Gku5s2kUZzDBvQWvHg== + dependencies: + tslib "^2.0.1" + +ast-types@^0.14.2: + version "0.14.2" + resolved "https://registry.yarnpkg.com/ast-types/-/ast-types-0.14.2.tgz#600b882df8583e3cd4f2df5fa20fa83759d4bdfd" + integrity sha512-O0yuUDnZeQDL+ncNGlJ78BiO4jnYI3bvMsD5prT0/nsgijG/LpNBIr63gTjVTNsiGkgQhiyCShTgxt8oXOrklA== + dependencies: + tslib "^2.0.1" + +ast-types@^0.16.1: + version "0.16.1" + resolved "https://registry.yarnpkg.com/ast-types/-/ast-types-0.16.1.tgz#7a9da1617c9081bc121faafe91711b4c8bb81da2" + integrity sha512-6t10qk83GOG8p0vKmaCr8eiilZwO171AvbROMtvvNiwrTly62t+7XkA8RdIIVbpMhCASAsxgAzdRSwh6nw/5Dg== + dependencies: + tslib "^2.0.1" + +async-limiter@~1.0.0: + version "1.0.1" + resolved "https://registry.yarnpkg.com/async-limiter/-/async-limiter-1.0.1.tgz#dd379e94f0db8310b08291f9d64c3209766617fd" + integrity sha512-csOlWGAcRFJaI6m+F2WKdnMKr4HhdhFVBk0H/QbJFMCr+uO2kwohwXQPxw/9OCxp05r5ghVBFSyioixx3gfkNQ== + +async@^3.2.3, async@^3.2.4: + version "3.2.4" + resolved "https://registry.yarnpkg.com/async/-/async-3.2.4.tgz#2d22e00f8cddeb5fde5dd33522b56d1cf569a81c" + integrity sha512-iAB+JbDEGXhyIUavoDl9WP/Jj106Kz9DEn1DPgYw5ruDn0e3Wgi3sKFm55sASdGBNOQB8F59d9qQ7deqrHA8wQ== + +asynciterator.prototype@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/asynciterator.prototype/-/asynciterator.prototype-1.0.0.tgz#8c5df0514936cdd133604dfcc9d3fb93f09b2b62" + integrity sha512-wwHYEIS0Q80f5mosx3L/dfG5t5rjEa9Ft51GTaNt862EnpyGHpgz2RkZvLPp1oF5TnAiTohkEKVEu8pQPJI7Vg== + dependencies: + has-symbols "^1.0.3" + +asynckit@^0.4.0: + version "0.4.0" + resolved "https://registry.yarnpkg.com/asynckit/-/asynckit-0.4.0.tgz#c79ed97f7f34cb8f2ba1bc9790bcc366474b4b79" + integrity sha512-Oei9OH4tRh0YqU3GxhX79dM/mwVgvbZJaSNaRk+bshkj0S5cfHcgYakreBjrHwatXKbz+IoIdYLxrKim2MjW0Q== + +autoprefixer@^10.4.15: + version "10.4.15" + resolved "https://registry.yarnpkg.com/autoprefixer/-/autoprefixer-10.4.15.tgz#a1230f4aeb3636b89120b34a1f513e2f6834d530" + integrity sha512-KCuPB8ZCIqFdA4HwKXsvz7j6gvSDNhDP7WnUjBleRkKjPdvCmHFuQ77ocavI8FT6NdvlBnE2UFr2H4Mycn8Vew== + dependencies: + browserslist "^4.21.10" + caniuse-lite "^1.0.30001520" + fraction.js "^4.2.0" + normalize-range "^0.1.2" + picocolors "^1.0.0" + postcss-value-parser "^4.2.0" + +available-typed-arrays@^1.0.5: + version "1.0.5" + resolved "https://registry.yarnpkg.com/available-typed-arrays/-/available-typed-arrays-1.0.5.tgz#92f95616501069d07d10edb2fc37d3e1c65123b7" + integrity sha512-DMD0KiN46eipeziST1LPP/STfDU0sufISXmjSgvVsoU2tqxctQeASejWcfNtxYKqETM1UxQ8sp2OrSBWpHY6sw== + +axe-core@^4.6.2: + version "4.7.2" + resolved "https://registry.yarnpkg.com/axe-core/-/axe-core-4.7.2.tgz#040a7342b20765cb18bb50b628394c21bccc17a0" + integrity sha512-zIURGIS1E1Q4pcrMjp+nnEh+16G56eG/MUllJH8yEvw7asDo7Ac9uhC9KIH5jzpITueEZolfYglnCGIuSBz39g== + +axobject-query@^3.1.1: + version "3.2.1" + resolved "https://registry.yarnpkg.com/axobject-query/-/axobject-query-3.2.1.tgz#39c378a6e3b06ca679f29138151e45b2b32da62a" + integrity sha512-jsyHu61e6N4Vbz/v18DHwWYKK0bSWLqn47eeDSKPB7m8tqMHF9YJ+mhIk2lVteyZrY8tnSj/jHOv4YiTCuCJgg== + dependencies: + dequal "^2.0.3" + +babel-core@^7.0.0-bridge.0: + version "7.0.0-bridge.0" + resolved "https://registry.yarnpkg.com/babel-core/-/babel-core-7.0.0-bridge.0.tgz#95a492ddd90f9b4e9a4a1da14eb335b87b634ece" + integrity sha512-poPX9mZH/5CSanm50Q+1toVci6pv5KSRv/5TWCwtzQS5XEwn40BcCrgIeMFWP9CKKIniKXNxoIOnOq4VVlGXhg== + +babel-loader@^9.0.0: + version "9.1.3" + resolved "https://registry.yarnpkg.com/babel-loader/-/babel-loader-9.1.3.tgz#3d0e01b4e69760cc694ee306fe16d358aa1c6f9a" + integrity sha512-xG3ST4DglodGf8qSwv0MdeWLhrDsw/32QMdTO5T1ZIp9gQur0HkCyFs7Awskr10JKXFXwpAhiCuYX5oGXnRGbw== + dependencies: + find-cache-dir "^4.0.0" + schema-utils "^4.0.0" + +babel-plugin-add-react-displayname@^0.0.5: + version "0.0.5" + resolved "https://registry.yarnpkg.com/babel-plugin-add-react-displayname/-/babel-plugin-add-react-displayname-0.0.5.tgz#339d4cddb7b65fd62d1df9db9fe04de134122bd5" + integrity sha512-LY3+Y0XVDYcShHHorshrDbt4KFWL4bSeniCtl4SYZbask+Syngk1uMPCeN9+nSiZo6zX5s0RTq/J9Pnaaf/KHw== + +babel-plugin-istanbul@^6.1.1: + version "6.1.1" + resolved "https://registry.yarnpkg.com/babel-plugin-istanbul/-/babel-plugin-istanbul-6.1.1.tgz#fa88ec59232fd9b4e36dbbc540a8ec9a9b47da73" + integrity sha512-Y1IQok9821cC9onCx5otgFfRm7Lm+I+wwxOx738M/WLPZ9Q42m4IG5W0FNX8WLL2gYMZo3JkuXIH2DOpWM+qwA== + dependencies: + "@babel/helper-plugin-utils" "^7.0.0" + "@istanbuljs/load-nyc-config" "^1.0.0" + "@istanbuljs/schema" "^0.1.2" + istanbul-lib-instrument "^5.0.4" + test-exclude "^6.0.0" + +babel-plugin-named-exports-order@^0.0.2: + version "0.0.2" + resolved "https://registry.yarnpkg.com/babel-plugin-named-exports-order/-/babel-plugin-named-exports-order-0.0.2.tgz#ae14909521cf9606094a2048239d69847540cb09" + integrity sha512-OgOYHOLoRK+/mvXU9imKHlG6GkPLYrUCvFXG/CM93R/aNNO8pOOF4aS+S8CCHMDQoNSeiOYEZb/G6RwL95Jktw== + +babel-plugin-polyfill-corejs2@^0.4.5: + version "0.4.5" + resolved "https://registry.yarnpkg.com/babel-plugin-polyfill-corejs2/-/babel-plugin-polyfill-corejs2-0.4.5.tgz#8097b4cb4af5b64a1d11332b6fb72ef5e64a054c" + integrity sha512-19hwUH5FKl49JEsvyTcoHakh6BE0wgXLLptIyKZ3PijHc/Ci521wygORCUCCred+E/twuqRyAkE02BAWPmsHOg== + dependencies: + "@babel/compat-data" "^7.22.6" + "@babel/helper-define-polyfill-provider" "^0.4.2" + semver "^6.3.1" + +babel-plugin-polyfill-corejs3@^0.8.3: + version "0.8.3" + resolved "https://registry.yarnpkg.com/babel-plugin-polyfill-corejs3/-/babel-plugin-polyfill-corejs3-0.8.3.tgz#b4f719d0ad9bb8e0c23e3e630c0c8ec6dd7a1c52" + integrity sha512-z41XaniZL26WLrvjy7soabMXrfPWARN25PZoriDEiLMxAp50AUW3t35BGQUMg5xK3UrpVTtagIDklxYa+MhiNA== + dependencies: + "@babel/helper-define-polyfill-provider" "^0.4.2" + core-js-compat "^3.31.0" + +babel-plugin-polyfill-regenerator@^0.5.2: + version "0.5.2" + resolved "https://registry.yarnpkg.com/babel-plugin-polyfill-regenerator/-/babel-plugin-polyfill-regenerator-0.5.2.tgz#80d0f3e1098c080c8b5a65f41e9427af692dc326" + integrity sha512-tAlOptU0Xj34V1Y2PNTL4Y0FOJMDB6bZmoW39FeCQIhigGLkqu3Fj6uiXpxIf6Ij274ENdYx64y6Au+ZKlb1IA== + dependencies: + "@babel/helper-define-polyfill-provider" "^0.4.2" + +babel-plugin-react-docgen@^4.2.1: + version "4.2.1" + resolved "https://registry.yarnpkg.com/babel-plugin-react-docgen/-/babel-plugin-react-docgen-4.2.1.tgz#7cc8e2f94e8dc057a06e953162f0810e4e72257b" + integrity sha512-UQ0NmGHj/HAqi5Bew8WvNfCk8wSsmdgNd8ZdMjBCICtyCJCq9LiqgqvjCYe570/Wg7AQArSq1VQ60Dd/CHN7mQ== + dependencies: + ast-types "^0.14.2" + lodash "^4.17.15" + react-docgen "^5.0.0" + +balanced-match@^1.0.0: + version "1.0.2" + resolved "https://registry.yarnpkg.com/balanced-match/-/balanced-match-1.0.2.tgz#e83e3a7e3f300b34cb9d87f615fa0cbf357690ee" + integrity sha512-3oSeUO0TMV67hN1AmbXsK4yaqU7tjiHlbxRDZOpH0KW9+CeX4bRAaX0Anxt0tx2MrpRpWwQaPwIlISEJhYU5Pw== + +base64-js@^1.3.1: + version "1.5.1" + resolved "https://registry.yarnpkg.com/base64-js/-/base64-js-1.5.1.tgz#1b1b440160a5bf7ad40b650f095963481903930a" + integrity sha512-AKpaYlHn8t4SVbOHCy+b5+KKgvR4vrsD8vbvrbiQJps7fKDTkjkDry6ji0rUJjC0kzbNePLwzxq8iypo41qeWA== + +better-opn@^3.0.2: + version "3.0.2" + resolved "https://registry.yarnpkg.com/better-opn/-/better-opn-3.0.2.tgz#f96f35deaaf8f34144a4102651babcf00d1d8817" + integrity sha512-aVNobHnJqLiUelTaHat9DZ1qM2w0C0Eym4LPI/3JxOnSokGVdsl1T1kN7TFvsEAD8G47A6VKQ0TVHqbBnYMJlQ== + dependencies: + open "^8.0.4" + +big-integer@^1.6.44: + version "1.6.51" + resolved "https://registry.yarnpkg.com/big-integer/-/big-integer-1.6.51.tgz#0df92a5d9880560d3ff2d5fd20245c889d130686" + integrity sha512-GPEid2Y9QU1Exl1rpO9B2IPJGHPSupF5GnVIP0blYvNOMer2bTvSWs1jGOUg04hTmu67nmLsQ9TBo1puaotBHg== + +big.js@^5.2.2: + version "5.2.2" + resolved "https://registry.yarnpkg.com/big.js/-/big.js-5.2.2.tgz#65f0af382f578bcdc742bd9c281e9cb2d7768328" + integrity sha512-vyL2OymJxmarO8gxMr0mhChsO9QGwhynfuu4+MHTAW6czfq9humCB7rKpUjDd9YUiDPU4mzpyupFSvOClAwbmQ== + +binary-extensions@^2.0.0: + version "2.2.0" + resolved "https://registry.yarnpkg.com/binary-extensions/-/binary-extensions-2.2.0.tgz#75f502eeaf9ffde42fc98829645be4ea76bd9e2d" + integrity sha512-jDctJ/IVQbZoJykoeHbhXpOlNBqGNcwXJKJog42E5HDPUwQTSdjCHdihjj0DlnheQ7blbT6dHOafNAiS8ooQKA== + +bl@^4.0.3, bl@^4.1.0: + version "4.1.0" + resolved "https://registry.yarnpkg.com/bl/-/bl-4.1.0.tgz#451535264182bec2fbbc83a62ab98cf11d9f7b3a" + integrity sha512-1W07cM9gS6DcLperZfFSj+bWLtaPGSOHWhPiGzXmvVJbRLdG82sH/Kn8EtW1VqWVA54AKf2h5k5BbnIbwF3h6w== + dependencies: + buffer "^5.5.0" + inherits "^2.0.4" + readable-stream "^3.4.0" + +bn.js@^4.0.0, bn.js@^4.1.0, bn.js@^4.11.9: + version "4.12.0" + resolved "https://registry.yarnpkg.com/bn.js/-/bn.js-4.12.0.tgz#775b3f278efbb9718eec7361f483fb36fbbfea88" + integrity sha512-c98Bf3tPniI+scsdk237ku1Dc3ujXQTSgyiPUDEOe7tRkhrqridvh8klBv0HCEso1OLOYcHuCv/cS6DNxKH+ZA== + +bn.js@^5.0.0, bn.js@^5.1.1: + version "5.2.1" + resolved "https://registry.yarnpkg.com/bn.js/-/bn.js-5.2.1.tgz#0bc527a6a0d18d0aa8d5b0538ce4a77dccfa7b70" + integrity sha512-eXRvHzWyYPBuB4NBy0cmYQjGitUrtqwbvlzP3G6VFnNRbsZQIxQ10PbKKHt8gZ/HW/D/747aDl+QkDqg3KQLMQ== + +body-parser@1.20.1: + version "1.20.1" + resolved "https://registry.yarnpkg.com/body-parser/-/body-parser-1.20.1.tgz#b1812a8912c195cd371a3ee5e66faa2338a5c668" + integrity sha512-jWi7abTbYwajOytWCQc37VulmWiRae5RyTpaCyDcS5/lMdtwSz5lOpDE67srw/HYe35f1z3fDQw+3txg7gNtWw== + dependencies: + bytes "3.1.2" + content-type "~1.0.4" + debug "2.6.9" + depd "2.0.0" + destroy "1.2.0" + http-errors "2.0.0" + iconv-lite "0.4.24" + on-finished "2.4.1" + qs "6.11.0" + raw-body "2.5.1" + type-is "~1.6.18" + unpipe "1.0.0" + +boolbase@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/boolbase/-/boolbase-1.0.0.tgz#68dff5fbe60c51eb37725ea9e3ed310dcc1e776e" + integrity sha512-JZOSA7Mo9sNGB8+UjSgzdLtokWAky1zbztM3WRLCbZ70/3cTANmQmOdR7y2g+J0e2WXywy1yS468tY+IruqEww== + +bplist-parser@^0.2.0: + version "0.2.0" + resolved "https://registry.yarnpkg.com/bplist-parser/-/bplist-parser-0.2.0.tgz#43a9d183e5bf9d545200ceac3e712f79ebbe8d0e" + integrity sha512-z0M+byMThzQmD9NILRniCUXYsYpjwnlO8N5uCFaCqIOpqRsJCrQL9NK3JsD67CN5a08nF5oIL2bD6loTdHOuKw== + dependencies: + big-integer "^1.6.44" + +brace-expansion@^1.1.7: + version "1.1.11" + resolved "https://registry.yarnpkg.com/brace-expansion/-/brace-expansion-1.1.11.tgz#3c7fcbf529d87226f3d2f52b966ff5271eb441dd" + integrity sha512-iCuPHDFgrHX7H2vEI/5xpz07zSHB00TpugqhmYtVmMO6518mCuRMoOYFldEBl0g187ufozdaHgWKcYFb61qGiA== + dependencies: + balanced-match "^1.0.0" + concat-map "0.0.1" + +brace-expansion@^2.0.1: + version "2.0.1" + resolved "https://registry.yarnpkg.com/brace-expansion/-/brace-expansion-2.0.1.tgz#1edc459e0f0c548486ecf9fc99f2221364b9a0ae" + integrity sha512-XnAIvQ8eM+kC6aULx6wuQiwVsnzsi9d3WxzV3FpWTGA19F621kwdbsAcFKXgKUHZWsy+mY6iL1sHTxWEFCytDA== + dependencies: + balanced-match "^1.0.0" + +braces@^3.0.2, braces@~3.0.2: + version "3.0.2" + resolved "https://registry.yarnpkg.com/braces/-/braces-3.0.2.tgz#3454e1a462ee8d599e236df336cd9ea4f8afe107" + integrity sha512-b8um+L1RzM3WDSzvhm6gIz1yfTbBt6YTlcEKAvsmqCZZFw46z626lVj9j1yEPW33H5H+lBQpZMP1k8l+78Ha0A== + dependencies: + fill-range "^7.0.1" + +brorand@^1.0.1, brorand@^1.1.0: + version "1.1.0" + resolved "https://registry.yarnpkg.com/brorand/-/brorand-1.1.0.tgz#12c25efe40a45e3c323eb8675a0a0ce57b22371f" + integrity sha512-cKV8tMCEpQs4hK/ik71d6LrPOnpkpGBR0wzxqr68g2m/LB2GxVYQroAjMJZRVM1Y4BCjCKc3vAamxSzOY2RP+w== + +browser-assert@^1.2.1: + version "1.2.1" + resolved "https://registry.yarnpkg.com/browser-assert/-/browser-assert-1.2.1.tgz#9aaa5a2a8c74685c2ae05bfe46efd606f068c200" + integrity sha512-nfulgvOR6S4gt9UKCeGJOuSGBPGiFT6oQ/2UBnvTY/5aQ1PnksW72fhZkM30DzoRRv2WpwZf1vHHEr3mtuXIWQ== + +browserify-aes@^1.0.0, browserify-aes@^1.0.4: + version "1.2.0" + resolved "https://registry.yarnpkg.com/browserify-aes/-/browserify-aes-1.2.0.tgz#326734642f403dabc3003209853bb70ad428ef48" + integrity sha512-+7CHXqGuspUn/Sl5aO7Ea0xWGAtETPXNSAjHo48JfLdPWcMng33Xe4znFvQweqc/uzk5zSOI3H52CYnjCfb5hA== + dependencies: + buffer-xor "^1.0.3" + cipher-base "^1.0.0" + create-hash "^1.1.0" + evp_bytestokey "^1.0.3" + inherits "^2.0.1" + safe-buffer "^5.0.1" + +browserify-cipher@^1.0.0: + version "1.0.1" + resolved "https://registry.yarnpkg.com/browserify-cipher/-/browserify-cipher-1.0.1.tgz#8d6474c1b870bfdabcd3bcfcc1934a10e94f15f0" + integrity sha512-sPhkz0ARKbf4rRQt2hTpAHqn47X3llLkUGn+xEJzLjwY8LRs2p0v7ljvI5EyoRO/mexrNunNECisZs+gw2zz1w== + dependencies: + browserify-aes "^1.0.4" + browserify-des "^1.0.0" + evp_bytestokey "^1.0.0" + +browserify-des@^1.0.0: + version "1.0.2" + resolved "https://registry.yarnpkg.com/browserify-des/-/browserify-des-1.0.2.tgz#3af4f1f59839403572f1c66204375f7a7f703e9c" + integrity sha512-BioO1xf3hFwz4kc6iBhI3ieDFompMhrMlnDFC4/0/vd5MokpuAc3R+LYbwTA9A5Yc9pq9UYPqffKpW2ObuwX5A== + dependencies: + cipher-base "^1.0.1" + des.js "^1.0.0" + inherits "^2.0.1" + safe-buffer "^5.1.2" + +browserify-rsa@^4.0.0, browserify-rsa@^4.0.1: + version "4.1.0" + resolved "https://registry.yarnpkg.com/browserify-rsa/-/browserify-rsa-4.1.0.tgz#b2fd06b5b75ae297f7ce2dc651f918f5be158c8d" + integrity sha512-AdEER0Hkspgno2aR97SAf6vi0y0k8NuOpGnVH3O99rcA5Q6sh8QxcngtHuJ6uXwnfAXNM4Gn1Gb7/MV1+Ymbog== + dependencies: + bn.js "^5.0.0" + randombytes "^2.0.1" + +browserify-sign@^4.0.0: + version "4.2.1" + resolved "https://registry.yarnpkg.com/browserify-sign/-/browserify-sign-4.2.1.tgz#eaf4add46dd54be3bb3b36c0cf15abbeba7956c3" + integrity sha512-/vrA5fguVAKKAVTNJjgSm1tRQDHUU6DbwO9IROu/0WAzC8PKhucDSh18J0RMvVeHAn5puMd+QHC2erPRNf8lmg== + dependencies: + bn.js "^5.1.1" + browserify-rsa "^4.0.1" + create-hash "^1.2.0" + create-hmac "^1.1.7" + elliptic "^6.5.3" + inherits "^2.0.4" + parse-asn1 "^5.1.5" + readable-stream "^3.6.0" + safe-buffer "^5.2.0" + +browserify-zlib@^0.1.4: + version "0.1.4" + resolved "https://registry.yarnpkg.com/browserify-zlib/-/browserify-zlib-0.1.4.tgz#bb35f8a519f600e0fa6b8485241c979d0141fb2d" + integrity sha512-19OEpq7vWgsH6WkvkBJQDFvJS1uPcbFOQ4v9CU839dO+ZZXUZO6XpE6hNCqvlIIj+4fZvRiJ6DsAQ382GwiyTQ== + dependencies: + pako "~0.2.0" + +browserify-zlib@^0.2.0: + version "0.2.0" + resolved "https://registry.yarnpkg.com/browserify-zlib/-/browserify-zlib-0.2.0.tgz#2869459d9aa3be245fe8fe2ca1f46e2e7f54d73f" + integrity sha512-Z942RysHXmJrhqk88FmKBVq/v5tqmSkDz7p54G/MGyjMnCFFnC79XWNbg+Vta8W6Wb2qtSZTSxIGkJrRpCFEiA== + dependencies: + pako "~1.0.5" + +browserslist@^4.14.5, browserslist@^4.21.10, browserslist@^4.21.9: + version "4.21.10" + resolved "https://registry.yarnpkg.com/browserslist/-/browserslist-4.21.10.tgz#dbbac576628c13d3b2231332cb2ec5a46e015bb0" + integrity sha512-bipEBdZfVH5/pwrvqc+Ub0kUPVfGUhlKxbvfD+z1BDnPEO/X98ruXGA1WP5ASpAFKan7Qr6j736IacbZQuAlKQ== + dependencies: + caniuse-lite "^1.0.30001517" + electron-to-chromium "^1.4.477" + node-releases "^2.0.13" + update-browserslist-db "^1.0.11" + +bser@2.1.1: + version "2.1.1" + resolved "https://registry.yarnpkg.com/bser/-/bser-2.1.1.tgz#e6787da20ece9d07998533cfd9de6f5c38f4bc05" + integrity sha512-gQxTNE/GAfIIrmHLUE3oJyp5FO6HRBfhjnw4/wMmA63ZGDJnWBmgY/lyQBpnDUkGmAhbSe39tx2d/iTOAfglwQ== + dependencies: + node-int64 "^0.4.0" + +buffer-crc32@~0.2.3: + version "0.2.13" + resolved "https://registry.yarnpkg.com/buffer-crc32/-/buffer-crc32-0.2.13.tgz#0d333e3f00eac50aa1454abd30ef8c2a5d9a7242" + integrity sha512-VO9Ht/+p3SN7SKWqcrgEzjGbRSJYTx+Q1pTQC0wrWqHx0vpJraQ6GtHx8tvcg1rlK1byhU5gccxgOgj7B0TDkQ== + +buffer-from@^1.0.0: + version "1.1.2" + resolved "https://registry.yarnpkg.com/buffer-from/-/buffer-from-1.1.2.tgz#2b146a6fd72e80b4f55d255f35ed59a3a9a41bd5" + integrity sha512-E+XQCRwSbaaiChtv6k6Dwgc+bx+Bs6vuKJHHl5kox/BaKbhiXzqQOwK4cO22yElGp2OCmjwVhT3HmxgyPGnJfQ== + +buffer-xor@^1.0.3: + version "1.0.3" + resolved "https://registry.yarnpkg.com/buffer-xor/-/buffer-xor-1.0.3.tgz#26e61ed1422fb70dd42e6e36729ed51d855fe8d9" + integrity sha512-571s0T7nZWK6vB67HI5dyUF7wXiNcfaPPPTl6zYCNApANjIvYJTg7hlud/+cJpdAhS7dVzqMLmfhfHR3rAcOjQ== + +buffer@^5.5.0: + version "5.7.1" + resolved "https://registry.yarnpkg.com/buffer/-/buffer-5.7.1.tgz#ba62e7c13133053582197160851a8f648e99eed0" + integrity sha512-EHcyIPBQ4BSGlvjB16k5KgAJ27CIsHY/2JBmCRReo48y9rQ3MaUzWX3KVlBa4U7MyX02HdVj0K7C3WaB3ju7FQ== + dependencies: + base64-js "^1.3.1" + ieee754 "^1.1.13" + +buffer@^6.0.3: + version "6.0.3" + resolved "https://registry.yarnpkg.com/buffer/-/buffer-6.0.3.tgz#2ace578459cc8fbe2a70aaa8f52ee63b6a74c6c6" + integrity sha512-FTiCpNxtwiZZHEZbcbTIcZjERVICn9yq/pDFkTl95/AxzD1naBctN7YO68riM/gLSDY7sdrMby8hofADYuuqOA== + dependencies: + base64-js "^1.3.1" + ieee754 "^1.2.1" + +builtin-status-codes@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/builtin-status-codes/-/builtin-status-codes-3.0.0.tgz#85982878e21b98e1c66425e03d0174788f569ee8" + integrity sha512-HpGFw18DgFWlncDfjTa2rcQ4W88O1mC8e8yZ2AvQY5KDaktSTwo+KRf6nHK6FRI5FyRyb/5T6+TSxfP7QyGsmQ== + +busboy@1.6.0: + version "1.6.0" + resolved "https://registry.yarnpkg.com/busboy/-/busboy-1.6.0.tgz#966ea36a9502e43cdb9146962523b92f531f6893" + integrity sha512-8SFQbg/0hQ9xy3UNTB0YEnsNBbWfhf7RtnzpL7TkBiTBRfrQ9Fxcnz7VJsleJpyp6rVLvXiuORqjlHi5q+PYuA== + dependencies: + streamsearch "^1.1.0" + +bytes@3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/bytes/-/bytes-3.0.0.tgz#d32815404d689699f85a4ea4fa8755dd13a96048" + integrity sha512-pMhOfFDPiv9t5jjIXkHosWmkSyQbvsgEVNkz0ERHbuLh2T/7j4Mqqpz523Fe8MVY89KC6Sh/QfS2sM+SjgFDcw== + +bytes@3.1.2: + version "3.1.2" + resolved "https://registry.yarnpkg.com/bytes/-/bytes-3.1.2.tgz#8b0beeb98605adf1b128fa4386403c009e0221a5" + integrity sha512-/Nf7TyzTx6S3yRJObOAV7956r8cr2+Oj8AC5dt8wSP3BQAoeX58NoHyCU8P8zGkNXStjTSi6fzO6F0pBdcYbEg== + +c8@^7.6.0: + version "7.14.0" + resolved "https://registry.yarnpkg.com/c8/-/c8-7.14.0.tgz#f368184c73b125a80565e9ab2396ff0be4d732f3" + integrity sha512-i04rtkkcNcCf7zsQcSv/T9EbUn4RXQ6mropeMcjFOsQXQ0iGLAr/xT6TImQg4+U9hmNpN9XdvPkjUL1IzbgxJw== + dependencies: + "@bcoe/v8-coverage" "^0.2.3" + "@istanbuljs/schema" "^0.1.3" + find-up "^5.0.0" + foreground-child "^2.0.0" + istanbul-lib-coverage "^3.2.0" + istanbul-lib-report "^3.0.0" + istanbul-reports "^3.1.4" + rimraf "^3.0.2" + test-exclude "^6.0.0" + v8-to-istanbul "^9.0.0" + yargs "^16.2.0" + yargs-parser "^20.2.9" + +call-bind@^1.0.0, call-bind@^1.0.2: + version "1.0.2" + resolved "https://registry.yarnpkg.com/call-bind/-/call-bind-1.0.2.tgz#b1d4e89e688119c3c9a903ad30abb2f6a919be3c" + integrity sha512-7O+FbCihrB5WGbFYesctwmTKae6rOiIzmz1icreWJ+0aA7LJfuqhEso2T9ncpcFtzMQtzXf2QGGueWJGTYsqrA== + dependencies: + function-bind "^1.1.1" + get-intrinsic "^1.0.2" + +callsites@^3.0.0: + version "3.1.0" + resolved "https://registry.yarnpkg.com/callsites/-/callsites-3.1.0.tgz#b3630abd8943432f54b3f0519238e33cd7df2f73" + integrity sha512-P8BjAsXvZS+VIDUI11hHCQEv74YT67YUi5JJFNWIqL235sBmjX4+qx9Muvls5ivyNENctx46xQLQ3aTuE7ssaQ== + +camel-case@^4.1.2: + version "4.1.2" + resolved "https://registry.yarnpkg.com/camel-case/-/camel-case-4.1.2.tgz#9728072a954f805228225a6deea6b38461e1bd5a" + integrity sha512-gxGWBrTT1JuMx6R+o5PTXMmUnhnVzLQ9SNutD4YqKtI6ap897t3tKECYla6gCWEkplXnlNybEkZg9GEGxKFCgw== + dependencies: + pascal-case "^3.1.2" + tslib "^2.0.3" + +camelcase-css@^2.0.1: + version "2.0.1" + resolved "https://registry.yarnpkg.com/camelcase-css/-/camelcase-css-2.0.1.tgz#ee978f6947914cc30c6b44741b6ed1df7f043fd5" + integrity sha512-QOSvevhslijgYwRx6Rv7zKdMF8lbRmx+uQGx2+vDc+KI/eBnsy9kit5aj23AgGu3pa4t9AgwbnXWqS+iOY+2aA== + +camelcase@^5.3.1: + version "5.3.1" + resolved "https://registry.yarnpkg.com/camelcase/-/camelcase-5.3.1.tgz#e3c9b31569e106811df242f715725a1f4c494320" + integrity sha512-L28STB170nwWS63UjtlEOE3dldQApaJXZkOI1uMFfzf3rRuPegHaHesyee+YxQ+W6SvRDQV6UrdOdRiR153wJg== + +camelize@^1.0.0: + version "1.0.1" + resolved "https://registry.yarnpkg.com/camelize/-/camelize-1.0.1.tgz#89b7e16884056331a35d6b5ad064332c91daa6c3" + integrity sha512-dU+Tx2fsypxTgtLoE36npi3UqcjSSMNYfkqgmoEhtZrraP5VWq0K7FkWVTYa8eMPtnU/G2txVsfdCJTn9uzpuQ== + +caniuse-lite@^1.0.30001406, caniuse-lite@^1.0.30001517, caniuse-lite@^1.0.30001520: + version "1.0.30001527" + resolved "https://registry.yarnpkg.com/caniuse-lite/-/caniuse-lite-1.0.30001527.tgz#813826554828245ccee776c850566dce12bdeaba" + integrity sha512-YkJi7RwPgWtXVSgK4lG9AHH57nSzvvOp9MesgXmw4Q7n0C3H04L0foHqfxcmSAm5AcWb8dW9AYj2tR7/5GnddQ== + +case-sensitive-paths-webpack-plugin@^2.4.0: + version "2.4.0" + resolved "https://registry.yarnpkg.com/case-sensitive-paths-webpack-plugin/-/case-sensitive-paths-webpack-plugin-2.4.0.tgz#db64066c6422eed2e08cc14b986ca43796dbc6d4" + integrity sha512-roIFONhcxog0JSSWbvVAh3OocukmSgpqOH6YpMkCvav/ySIV3JKg4Dc8vYtQjYi/UxpNE36r/9v+VqTQqgkYmw== + +chalk@^2.4.2: + version "2.4.2" + resolved "https://registry.yarnpkg.com/chalk/-/chalk-2.4.2.tgz#cd42541677a54333cf541a49108c1432b44c9424" + integrity sha512-Mti+f9lpJNcwF4tWV8/OrTTtF1gZi+f8FqlyAdouralcFWFQWF2+NgCHShjkCb+IFBLq9buZwE1xckQU4peSuQ== + dependencies: + ansi-styles "^3.2.1" + escape-string-regexp "^1.0.5" + supports-color "^5.3.0" + +chalk@^4.0.0, chalk@^4.0.2, chalk@^4.1.0, chalk@^4.1.2: + version "4.1.2" + resolved "https://registry.yarnpkg.com/chalk/-/chalk-4.1.2.tgz#aac4e2b7734a740867aeb16bf02aad556a1e7a01" + integrity sha512-oKnbhFyRIXpUuez8iBMmyEa4nbj4IOQyuhc/wy9kY7/WVPcwIO9VA668Pu8RkO7+0G76SLROeyw9CpQ061i4mA== + dependencies: + ansi-styles "^4.1.0" + supports-color "^7.1.0" + +chokidar@^3.4.0, chokidar@^3.5.3: + version "3.5.3" + resolved "https://registry.yarnpkg.com/chokidar/-/chokidar-3.5.3.tgz#1cf37c8707b932bd1af1ae22c0432e2acd1903bd" + integrity sha512-Dr3sfKRP6oTcjf2JmUmFJfeVMvXBdegxB0iVQ5eb2V10uFJUCAS8OByZdVAyVb8xXNz3GjjTgj9kLWsZTqE6kw== + dependencies: + anymatch "~3.1.2" + braces "~3.0.2" + glob-parent "~5.1.2" + is-binary-path "~2.1.0" + is-glob "~4.0.1" + normalize-path "~3.0.0" + readdirp "~3.6.0" + optionalDependencies: + fsevents "~2.3.2" + +chownr@^1.1.1: + version "1.1.4" + resolved "https://registry.yarnpkg.com/chownr/-/chownr-1.1.4.tgz#6fc9d7b42d32a583596337666e7d08084da2cc6b" + integrity sha512-jJ0bqzaylmJtVnNgzTeSOs8DPavpbYgEr/b0YL8/2GO3xJEhInFmhKMUnEJQjZumK7KXGFhUy89PrsJWlakBVg== + +chownr@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/chownr/-/chownr-2.0.0.tgz#15bfbe53d2eab4cf70f18a8cd68ebe5b3cb1dece" + integrity sha512-bIomtDF5KGpdogkLd9VspvFzk9KfpyyGlS8YFVZl7TGPBHL5snIOnxeshwVgPteQ9b4Eydl+pVbIyE1DcvCWgQ== + +chrome-trace-event@^1.0.2: + version "1.0.3" + resolved "https://registry.yarnpkg.com/chrome-trace-event/-/chrome-trace-event-1.0.3.tgz#1015eced4741e15d06664a957dbbf50d041e26ac" + integrity sha512-p3KULyQg4S7NIHixdwbGX+nFHkoBiA4YQmyWtjb8XngSKV124nJmRysgAeujbUVb15vh+RvFUfCPqU7rXk+hZg== + +ci-info@^3.2.0: + version "3.8.0" + resolved "https://registry.yarnpkg.com/ci-info/-/ci-info-3.8.0.tgz#81408265a5380c929f0bc665d62256628ce9ef91" + integrity sha512-eXTggHWSooYhq49F2opQhuHWgzucfF2YgODK4e1566GQs5BIfP30B0oenwBJHfWxAs2fyPB1s7Mg949zLf61Yw== + +cipher-base@^1.0.0, cipher-base@^1.0.1, cipher-base@^1.0.3: + version "1.0.4" + resolved "https://registry.yarnpkg.com/cipher-base/-/cipher-base-1.0.4.tgz#8760e4ecc272f4c363532f926d874aae2c1397de" + integrity sha512-Kkht5ye6ZGmwv40uUDZztayT2ThLQGfnj/T71N/XzeZeo3nf8foyW7zGTsPYkEya3m5f3cAypH+qe7YOrM1U2Q== + dependencies: + inherits "^2.0.1" + safe-buffer "^5.0.1" + +classnames@^2.2.5, classnames@^2.3.2: + version "2.3.2" + resolved "https://registry.yarnpkg.com/classnames/-/classnames-2.3.2.tgz#351d813bf0137fcc6a76a16b88208d2560a0d924" + integrity sha512-CSbhY4cFEJRe6/GQzIk5qXZ4Jeg5pcsP7b5peFSDpffpe1cqjASH/n9UTjBwOp6XpMSTwQ8Za2K5V02ueA7Tmw== + +clean-css@^5.2.2: + version "5.3.2" + resolved "https://registry.yarnpkg.com/clean-css/-/clean-css-5.3.2.tgz#70ecc7d4d4114921f5d298349ff86a31a9975224" + integrity sha512-JVJbM+f3d3Q704rF4bqQ5UUyTtuJ0JRKNbTKVEeujCCBoMdkEi+V+e8oktO9qGQNSvHrFTM6JZRXrUvGR1czww== + dependencies: + source-map "~0.6.0" + +clean-stack@^2.0.0: + version "2.2.0" + resolved "https://registry.yarnpkg.com/clean-stack/-/clean-stack-2.2.0.tgz#ee8472dbb129e727b31e8a10a427dee9dfe4008b" + integrity sha512-4diC9HaTE+KRAMWhDhrGOECgWZxoevMc5TlkObMqNSsVU62PYzXZ/SMTjzyGAFF1YusgxGcSWTEXBhp0CPwQ1A== + +cli-cursor@^3.1.0: + version "3.1.0" + resolved "https://registry.yarnpkg.com/cli-cursor/-/cli-cursor-3.1.0.tgz#264305a7ae490d1d03bf0c9ba7c925d1753af307" + integrity sha512-I/zHAwsKf9FqGoXM4WWRACob9+SNukZTd94DWF57E4toouRulbCxcUh6RKUEOQlYTHJnzkPMySvPNaaSLNfLZw== + dependencies: + restore-cursor "^3.1.0" + +cli-spinners@^2.5.0: + version "2.9.0" + resolved "https://registry.yarnpkg.com/cli-spinners/-/cli-spinners-2.9.0.tgz#5881d0ad96381e117bbe07ad91f2008fe6ffd8db" + integrity sha512-4/aL9X3Wh0yiMQlE+eeRhWP6vclO3QRtw1JHKIT0FFUs5FjpFmESqtMvYZ0+lbzBw900b95mS0hohy+qn2VK/g== + +cli-table3@^0.6.1: + version "0.6.3" + resolved "https://registry.yarnpkg.com/cli-table3/-/cli-table3-0.6.3.tgz#61ab765aac156b52f222954ffc607a6f01dbeeb2" + integrity sha512-w5Jac5SykAeZJKntOxJCrm63Eg5/4dhMWIcuTbo9rpE+brgaSZo0RuNJZeOyMgsUdhDeojvgyQLmjI+K50ZGyg== + dependencies: + string-width "^4.2.0" + optionalDependencies: + "@colors/colors" "1.5.0" + +client-only@0.0.1, client-only@^0.0.1: + version "0.0.1" + resolved "https://registry.yarnpkg.com/client-only/-/client-only-0.0.1.tgz#38bba5d403c41ab150bff64a95c85013cf73bca1" + integrity sha512-IV3Ou0jSMzZrd3pZ48nLkT9DA7Ag1pnPzaiQhpW7c3RbcqqzvzzVu+L8gfqMp/8IM2MQtSiqaCxrrcfu8I8rMA== + +cliui@^7.0.2: + version "7.0.4" + resolved "https://registry.yarnpkg.com/cliui/-/cliui-7.0.4.tgz#a0265ee655476fc807aea9df3df8df7783808b4f" + integrity sha512-OcRE68cOsVMXp1Yvonl/fzkQOyjLSu/8bhPDfQt0e0/Eb283TKP20Fs2MqoPsr9SwA595rRCA+QMzYc9nBP+JQ== + dependencies: + string-width "^4.2.0" + strip-ansi "^6.0.0" + wrap-ansi "^7.0.0" + +cliui@^8.0.1: + version "8.0.1" + resolved "https://registry.yarnpkg.com/cliui/-/cliui-8.0.1.tgz#0c04b075db02cbfe60dc8e6cf2f5486b1a3608aa" + integrity sha512-BSeNnyus75C4//NQ9gQt1/csTXyo/8Sb+afLAkzAptFuMsod9HFokGNudZpi/oQV73hnVK+sR+5PVRMd+Dr7YQ== + dependencies: + string-width "^4.2.0" + strip-ansi "^6.0.1" + wrap-ansi "^7.0.0" + +clone-deep@^4.0.1: + version "4.0.1" + resolved "https://registry.yarnpkg.com/clone-deep/-/clone-deep-4.0.1.tgz#c19fd9bdbbf85942b4fd979c84dcf7d5f07c2387" + integrity sha512-neHB9xuzh/wk0dIHweyAXv2aPGZIVk3pLMe+/RNzINf17fe0OG96QroktYAUm7SM1PBnzTabaLboqqxDyMU+SQ== + dependencies: + is-plain-object "^2.0.4" + kind-of "^6.0.2" + shallow-clone "^3.0.0" + +clone@^1.0.2: + version "1.0.4" + resolved "https://registry.yarnpkg.com/clone/-/clone-1.0.4.tgz#da309cc263df15994c688ca902179ca3c7cd7c7e" + integrity sha512-JQHZ2QMW6l3aH/j6xCqQThY/9OH4D/9ls34cgkUBiEeocRTU04tHfKPBsUK1PqZCUQM7GiA0IIXJSuXHI64Kbg== + +color-convert@^1.9.0: + version "1.9.3" + resolved "https://registry.yarnpkg.com/color-convert/-/color-convert-1.9.3.tgz#bb71850690e1f136567de629d2d5471deda4c1e8" + integrity sha512-QfAUtd+vFdAtFQcC8CCyYt1fYWxSqAiK2cSD6zDB8N3cpsEBAvRxp9zOGg6G/SHHJYAT88/az/IuDGALsNVbGg== + dependencies: + color-name "1.1.3" + +color-convert@^2.0.1: + version "2.0.1" + resolved "https://registry.yarnpkg.com/color-convert/-/color-convert-2.0.1.tgz#72d3a68d598c9bdb3af2ad1e84f21d896abd4de3" + integrity sha512-RRECPsj7iu/xb5oKYcsFHSppFNnsj/52OVTRKb4zP5onXwVF3zVmmToNcOfGC+CRDpfK/U584fMg38ZHCaElKQ== + dependencies: + color-name "~1.1.4" + +color-name@1.1.3: + version "1.1.3" + resolved "https://registry.yarnpkg.com/color-name/-/color-name-1.1.3.tgz#a7d0558bd89c42f795dd42328f740831ca53bc25" + integrity sha512-72fSenhMw2HZMTVHeCA9KCmpEIbzWiQsjN+BHcBbS9vr1mtt+vJjPdksIBNUmKAW8TFUDPJK5SUU3QhE9NEXDw== + +color-name@~1.1.4: + version "1.1.4" + resolved "https://registry.yarnpkg.com/color-name/-/color-name-1.1.4.tgz#c2a09a87acbde69543de6f63fa3995c826c536a2" + integrity sha512-dOy+3AuW3a2wNbZHIuMZpTcgjGuLU/uBL/ubcZF9OXbDo8ff4O8yVp5Bf0efS8uEoYo5q4Fx7dY9OgQGXgAsQA== + +colorette@^2.0.10, colorette@^2.0.19: + version "2.0.20" + resolved "https://registry.yarnpkg.com/colorette/-/colorette-2.0.20.tgz#9eb793e6833067f7235902fcd3b09917a000a95a" + integrity sha512-IfEDxwoWIjkeXL1eXcDiow4UbKjhLdq6/EuSVR9GMN7KVH3r9gQ83e73hsz1Nd1T3ijd5xv1wcWRYO+D6kCI2w== + +combined-stream@^1.0.8: + version "1.0.8" + resolved "https://registry.yarnpkg.com/combined-stream/-/combined-stream-1.0.8.tgz#c3d45a8b34fd730631a110a8a2520682b31d5a7f" + integrity sha512-FQN4MRfuJeHf7cBbBMJFXhKSDq+2kAArBlmRBvcvFE5BB1HZKXtSFASDhdlz9zOYwxh8lDdnvmMOe/+5cdoEdg== + dependencies: + delayed-stream "~1.0.0" + +commander@^11.0.0: + version "11.0.0" + resolved "https://registry.yarnpkg.com/commander/-/commander-11.0.0.tgz#43e19c25dbedc8256203538e8d7e9346877a6f67" + integrity sha512-9HMlXtt/BNoYr8ooyjjNRdIilOTkVJXB+GhxMTtOKwk0R4j4lS4NpjuqmRxroBfnfTSHQIHQB7wryHhXarNjmQ== + +commander@^2.19.0, commander@^2.20.0: + version "2.20.3" + resolved "https://registry.yarnpkg.com/commander/-/commander-2.20.3.tgz#fd485e84c03eb4881c20722ba48035e8531aeb33" + integrity sha512-GpVkmM8vF2vQUkj2LvZmD35JxeJOLCwJ9cUkugyk2nuhbv3+mJvpLYYt+0+USMxE+oj+ey/lJEnhZw75x/OMcQ== + +commander@^4.0.0, commander@^4.0.1: + version "4.1.1" + resolved "https://registry.yarnpkg.com/commander/-/commander-4.1.1.tgz#9fd602bd936294e9e9ef46a3f4d6964044b18068" + integrity sha512-NOKm8xhkzAjzFx8B2v5OAHT+u5pRQc2UCa2Vq9jYL/31o2wi9mxBA7LIFs3sV5VSC49z6pEhfbMULvShKj26WA== + +commander@^6.2.1: + version "6.2.1" + resolved "https://registry.yarnpkg.com/commander/-/commander-6.2.1.tgz#0792eb682dfbc325999bb2b84fddddba110ac73c" + integrity sha512-U7VdrJFnJgo4xjrHpTzu0yrHPGImdsmD95ZlgYSEajAn2JKzDhDTPG9kBTefmObL2w/ngeZnilk+OV9CG3d7UA== + +commander@^8.3.0: + version "8.3.0" + resolved "https://registry.yarnpkg.com/commander/-/commander-8.3.0.tgz#4837ea1b2da67b9c616a67afbb0fafee567bca66" + integrity sha512-OkTL9umf+He2DZkUq8f8J9of7yL6RJKI24dVITBmNfZBmri9zYZQrKkuXiKhyfPSu8tUhnVBB1iKXevvnlR4Ww== + +common-path-prefix@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/common-path-prefix/-/common-path-prefix-3.0.0.tgz#7d007a7e07c58c4b4d5f433131a19141b29f11e0" + integrity sha512-QE33hToZseCH3jS0qN96O/bSh3kaw/h+Tq7ngyY9eWDUnTlTNUyqfqvCXioLe5Na5jFsL78ra/wuBU4iuEgd4w== + +commondir@^1.0.1: + version "1.0.1" + resolved "https://registry.yarnpkg.com/commondir/-/commondir-1.0.1.tgz#ddd800da0c66127393cca5950ea968a3aaf1253b" + integrity sha512-W9pAhw0ja1Edb5GVdIF1mjZw/ASI0AlShXM83UUGe2DVr5TdAPEA1OA8m/g8zWp9x6On7gqufY+FatDbC3MDQg== + +compressible@~2.0.16: + version "2.0.18" + resolved "https://registry.yarnpkg.com/compressible/-/compressible-2.0.18.tgz#af53cca6b070d4c3c0750fbd77286a6d7cc46fba" + integrity sha512-AF3r7P5dWxL8MxyITRMlORQNaOA2IkAFaTr4k7BUumjPtRpGDTZpl0Pb1XCO6JeDCBdp126Cgs9sMxqSjgYyRg== + dependencies: + mime-db ">= 1.43.0 < 2" + +compression@^1.7.4: + version "1.7.4" + resolved "https://registry.yarnpkg.com/compression/-/compression-1.7.4.tgz#95523eff170ca57c29a0ca41e6fe131f41e5bb8f" + integrity sha512-jaSIDzP9pZVS4ZfQ+TzvtiWhdpFhE2RDHz8QJkpX9SIpLq88VueF5jJw6t+6CUQcAoA6t+x89MLrWAqpfDE8iQ== + dependencies: + accepts "~1.3.5" + bytes "3.0.0" + compressible "~2.0.16" + debug "2.6.9" + on-headers "~1.0.2" + safe-buffer "5.1.2" + vary "~1.1.2" + +concat-map@0.0.1: + version "0.0.1" + resolved "https://registry.yarnpkg.com/concat-map/-/concat-map-0.0.1.tgz#d8a96bd77fd68df7793a73036a3ba0d5405d477b" + integrity sha512-/Srv4dswyQNBfohGpz9o6Yb3Gz3SrUDqBH5rTuhGR7ahtlbYKnVxw2bCFMRljaA7EXHaXZ8wsHdodFvbkhKmqg== + +concat-stream@^1.6.2: + version "1.6.2" + resolved "https://registry.yarnpkg.com/concat-stream/-/concat-stream-1.6.2.tgz#904bdf194cd3122fc675c77fc4ac3d4ff0fd1a34" + integrity sha512-27HBghJxjiZtIk3Ycvn/4kbJk/1uZuJFfuPEns6LaEvpvG1f0hTea8lilrouyo9mVc2GWdcEZ8OLoGmSADlrCw== + dependencies: + buffer-from "^1.0.0" + inherits "^2.0.3" + readable-stream "^2.2.2" + typedarray "^0.0.6" + +console-browserify@^1.2.0: + version "1.2.0" + resolved "https://registry.yarnpkg.com/console-browserify/-/console-browserify-1.2.0.tgz#67063cef57ceb6cf4993a2ab3a55840ae8c49336" + integrity sha512-ZMkYO/LkF17QvCPqM0gxw8yUzigAOZOSWSHg91FH6orS7vcEj5dVZTidN2fQ14yBSdg97RqhSNwLUXInd52OTA== + +constants-browserify@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/constants-browserify/-/constants-browserify-1.0.0.tgz#c20b96d8c617748aaf1c16021760cd27fcb8cb75" + integrity sha512-xFxOwqIzR/e1k1gLiWEophSCMqXcwVHIH7akf7b/vxcUeGunlj3hvZaaqxwHsTgn+IndtkQJgSztIDWeumWJDQ== + +content-disposition@0.5.4: + version "0.5.4" + resolved "https://registry.yarnpkg.com/content-disposition/-/content-disposition-0.5.4.tgz#8b82b4efac82512a02bb0b1dcec9d2c5e8eb5bfe" + integrity sha512-FveZTNuGw04cxlAiWbzi6zTAL/lhehaWbTtgluJh4/E95DqMwTmha3KZN1aAWA8cFIhHzMZUvLevkw5Rqk+tSQ== + dependencies: + safe-buffer "5.2.1" + +content-type@~1.0.4: + version "1.0.5" + resolved "https://registry.yarnpkg.com/content-type/-/content-type-1.0.5.tgz#8b773162656d1d1086784c8f23a54ce6d73d7918" + integrity sha512-nTjqfcBFEipKdXCv4YDQWCfmcLZKm81ldF0pAopTvyrFGVbcR6P/VAAd5G7N+0tTr8QqiU0tFadD6FK4NtJwOA== + +convert-source-map@^1.1.0, convert-source-map@^1.6.0, convert-source-map@^1.7.0: + version "1.9.0" + resolved "https://registry.yarnpkg.com/convert-source-map/-/convert-source-map-1.9.0.tgz#7faae62353fb4213366d0ca98358d22e8368b05f" + integrity sha512-ASFBup0Mz1uyiIjANan1jzLQami9z1PoYSZCiiYW2FczPbenXc45FZdBZLzOT+r6+iciuEModtmCti+hjaAk0A== + +convert-source-map@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/convert-source-map/-/convert-source-map-2.0.0.tgz#4b560f649fc4e918dd0ab75cf4961e8bc882d82a" + integrity sha512-Kvp459HrV2FEJ1CAsi1Ku+MY3kasH19TFykTz2xWmMeq6bk2NU3XXvfJ+Q61m0xktWwt+1HSYf3JZsTms3aRJg== + +cookie-signature@1.0.6: + version "1.0.6" + resolved "https://registry.yarnpkg.com/cookie-signature/-/cookie-signature-1.0.6.tgz#e303a882b342cc3ee8ca513a79999734dab3ae2c" + integrity sha512-QADzlaHc8icV8I7vbaJXJwod9HWYp8uCqf1xa4OfNu1T7JVxQIrUgOWtHdNDtPiywmFbiS12VjotIXLrKM3orQ== + +cookie@0.5.0: + version "0.5.0" + resolved "https://registry.yarnpkg.com/cookie/-/cookie-0.5.0.tgz#d1f5d71adec6558c58f389987c366aa47e994f8b" + integrity sha512-YZ3GUyn/o8gfKJlnlX7g7xq4gyO6OSuhGPKaaGssGB2qgDUS0gPgtTvoyZLTt9Ab6dC4hfc9dV5arkvc/OCmrw== + +copy-anything@^2.0.1: + version "2.0.6" + resolved "https://registry.yarnpkg.com/copy-anything/-/copy-anything-2.0.6.tgz#092454ea9584a7b7ad5573062b2a87f5900fc480" + integrity sha512-1j20GZTsvKNkc4BY3NpMOM8tt///wY3FpIzozTOFO2ffuZcV61nojHXVKIy3WM+7ADCy5FVhdZYHYDdgTU0yJw== + dependencies: + is-what "^3.14.1" + +copy-webpack-plugin@^11.0.0: + version "11.0.0" + resolved "https://registry.yarnpkg.com/copy-webpack-plugin/-/copy-webpack-plugin-11.0.0.tgz#96d4dbdb5f73d02dd72d0528d1958721ab72e04a" + integrity sha512-fX2MWpamkW0hZxMEg0+mYnA40LTosOSa5TqZ9GYIBzyJa9C3QUaMPSE2xAi/buNr8u89SfD9wHSQVBzrRa/SOQ== + dependencies: + fast-glob "^3.2.11" + glob-parent "^6.0.1" + globby "^13.1.1" + normalize-path "^3.0.0" + schema-utils "^4.0.0" + serialize-javascript "^6.0.0" + +core-js-compat@^3.31.0: + version "3.32.1" + resolved "https://registry.yarnpkg.com/core-js-compat/-/core-js-compat-3.32.1.tgz#55f9a7d297c0761a8eb1d31b593e0f5b6ffae964" + integrity sha512-GSvKDv4wE0bPnQtjklV101juQ85g6H3rm5PDP20mqlS5j0kXF3pP97YvAu5hl+uFHqMictp3b2VxOHljWMAtuA== + dependencies: + browserslist "^4.21.10" + +core-js-pure@^3.23.3: + version "3.32.1" + resolved "https://registry.yarnpkg.com/core-js-pure/-/core-js-pure-3.32.1.tgz#5775b88f9062885f67b6d7edce59984e89d276f3" + integrity sha512-f52QZwkFVDPf7UEQZGHKx6NYxsxmVGJe5DIvbzOdRMJlmT6yv0KDjR8rmy3ngr/t5wU54c7Sp/qIJH0ppbhVpQ== + +core-util-is@~1.0.0: + version "1.0.3" + resolved "https://registry.yarnpkg.com/core-util-is/-/core-util-is-1.0.3.tgz#a6042d3634c2b27e9328f837b965fac83808db85" + integrity sha512-ZQBvi1DcpJ4GDqanjucZ2Hj3wEO5pZDS89BWbkcrvdxksJorwUDDZamX9ldFkp9aw2lmBDLgkObEA4DWNJ9FYQ== + +cosmiconfig@^7.0.1: + version "7.1.0" + resolved "https://registry.yarnpkg.com/cosmiconfig/-/cosmiconfig-7.1.0.tgz#1443b9afa596b670082ea46cbd8f6a62b84635f6" + integrity sha512-AdmX6xUzdNASswsFtmwSt7Vj8po9IuqXm0UXz7QKPuEUmPB4XyjGfaAr2PSuELMwkRMVH1EpIkX5bTZGRB3eCA== + dependencies: + "@types/parse-json" "^4.0.0" + import-fresh "^3.2.1" + parse-json "^5.0.0" + path-type "^4.0.0" + yaml "^1.10.0" + +cosmiconfig@^8.2.0: + version "8.3.4" + resolved "https://registry.yarnpkg.com/cosmiconfig/-/cosmiconfig-8.3.4.tgz#ee1356e7f24e248a6bb34ec5d438c3dcebeb410c" + integrity sha512-SF+2P8+o/PTV05rgsAjDzL4OFdVXAulSfC/L19VaeVT7+tpOOSscCt2QLxDZ+CLxF2WOiq6y1K5asvs8qUJT/Q== + dependencies: + import-fresh "^3.3.0" + js-yaml "^4.1.0" + parse-json "^5.2.0" + path-type "^4.0.0" + +create-ecdh@^4.0.0: + version "4.0.4" + resolved "https://registry.yarnpkg.com/create-ecdh/-/create-ecdh-4.0.4.tgz#d6e7f4bffa66736085a0762fd3a632684dabcc4e" + integrity sha512-mf+TCx8wWc9VpuxfP2ht0iSISLZnt0JgWlrOKZiNqyUZWnjIaCIVNQArMHnCZKfEYRg6IM7A+NeJoN8gf/Ws0A== + dependencies: + bn.js "^4.1.0" + elliptic "^6.5.3" + +create-hash@^1.1.0, create-hash@^1.1.2, create-hash@^1.2.0: + version "1.2.0" + resolved "https://registry.yarnpkg.com/create-hash/-/create-hash-1.2.0.tgz#889078af11a63756bcfb59bd221996be3a9ef196" + integrity sha512-z00bCGNHDG8mHAkP7CtT1qVu+bFQUPjYq/4Iv3C3kWjTFV10zIjfSoeqXo9Asws8gwSHDGj/hl2u4OGIjapeCg== + dependencies: + cipher-base "^1.0.1" + inherits "^2.0.1" + md5.js "^1.3.4" + ripemd160 "^2.0.1" + sha.js "^2.4.0" + +create-hmac@^1.1.0, create-hmac@^1.1.4, create-hmac@^1.1.7: + version "1.1.7" + resolved "https://registry.yarnpkg.com/create-hmac/-/create-hmac-1.1.7.tgz#69170c78b3ab957147b2b8b04572e47ead2243ff" + integrity sha512-MJG9liiZ+ogc4TzUwuvbER1JRdgvUFSB5+VR/g5h82fGaIRWMWddtKBHi7/sVhfjQZ6SehlyhvQYrcYkaUIpLg== + dependencies: + cipher-base "^1.0.3" + create-hash "^1.1.0" + inherits "^2.0.1" + ripemd160 "^2.0.0" + safe-buffer "^5.0.1" + sha.js "^2.4.8" + +cross-spawn@^7.0.0, cross-spawn@^7.0.2, cross-spawn@^7.0.3: + version "7.0.3" + resolved "https://registry.yarnpkg.com/cross-spawn/-/cross-spawn-7.0.3.tgz#f73a85b9d5d41d045551c177e2882d4ac85728a6" + integrity sha512-iRDPJKUPVEND7dHPO8rkbOnPpyDygcDFtWjpeWNCgy8WP2rXcxXL8TskReQl6OrB2G7+UJrags1q15Fudc7G6w== + dependencies: + path-key "^3.1.0" + shebang-command "^2.0.0" + which "^2.0.1" + +crypto-browserify@^3.12.0: + version "3.12.0" + resolved "https://registry.yarnpkg.com/crypto-browserify/-/crypto-browserify-3.12.0.tgz#396cf9f3137f03e4b8e532c58f698254e00f80ec" + integrity sha512-fz4spIh+znjO2VjL+IdhEpRJ3YN6sMzITSBijk6FK2UvTqruSQW+/cCZTSNsMiZNvUeq0CqurF+dAbyiGOY6Wg== + dependencies: + browserify-cipher "^1.0.0" + browserify-sign "^4.0.0" + create-ecdh "^4.0.0" + create-hash "^1.1.0" + create-hmac "^1.1.0" + diffie-hellman "^5.0.0" + inherits "^2.0.1" + pbkdf2 "^3.0.3" + public-encrypt "^4.0.0" + randombytes "^2.0.0" + randomfill "^1.0.3" + +crypto-random-string@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/crypto-random-string/-/crypto-random-string-2.0.0.tgz#ef2a7a966ec11083388369baa02ebead229b30d5" + integrity sha512-v1plID3y9r/lPhviJ1wrXpLeyUIGAZ2SHNYTEapm7/8A9nLPoyvVp3RK/EPFqn5kEznyWgYZNsRtYYIWbuG8KA== + +css-color-keywords@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/css-color-keywords/-/css-color-keywords-1.0.0.tgz#fea2616dc676b2962686b3af8dbdbe180b244e05" + integrity sha512-FyyrDHZKEjXDpNJYvVsV960FiqQyXc/LlYmsxl2BcdMb2WPx0OGRVgTg55rPSyLSNMqP52R9r8geSp7apN3Ofg== + +css-loader@^6.7.1, css-loader@^6.7.3: + version "6.8.1" + resolved "https://registry.yarnpkg.com/css-loader/-/css-loader-6.8.1.tgz#0f8f52699f60f5e679eab4ec0fcd68b8e8a50a88" + integrity sha512-xDAXtEVGlD0gJ07iclwWVkLoZOpEvAWaSyf6W18S2pOC//K8+qUDIx8IIT3D+HjnmkJPQeesOPv5aiUaJsCM2g== + dependencies: + icss-utils "^5.1.0" + postcss "^8.4.21" + postcss-modules-extract-imports "^3.0.0" + postcss-modules-local-by-default "^4.0.3" + postcss-modules-scope "^3.0.0" + postcss-modules-values "^4.0.0" + postcss-value-parser "^4.2.0" + semver "^7.3.8" + +css-select@^4.1.3: + version "4.3.0" + resolved "https://registry.yarnpkg.com/css-select/-/css-select-4.3.0.tgz#db7129b2846662fd8628cfc496abb2b59e41529b" + integrity sha512-wPpOYtnsVontu2mODhA19JrqWxNsfdatRKd64kmpRbQgh1KtItko5sTnEpPdpSaJszTOhEMlF/RPz28qj4HqhQ== + dependencies: + boolbase "^1.0.0" + css-what "^6.0.1" + domhandler "^4.3.1" + domutils "^2.8.0" + nth-check "^2.0.1" + +css-to-react-native@^3.2.0: + version "3.2.0" + resolved "https://registry.yarnpkg.com/css-to-react-native/-/css-to-react-native-3.2.0.tgz#cdd8099f71024e149e4f6fe17a7d46ecd55f1e32" + integrity sha512-e8RKaLXMOFii+02mOlqwjbD00KSEKqblnpO9e++1aXS1fPQOpS1YoqdVHBqPjHNoxeF2mimzVqawm2KCbEdtHQ== + dependencies: + camelize "^1.0.0" + css-color-keywords "^1.0.0" + postcss-value-parser "^4.0.2" + +css-unit-converter@^1.1.1: + version "1.1.2" + resolved "https://registry.yarnpkg.com/css-unit-converter/-/css-unit-converter-1.1.2.tgz#4c77f5a1954e6dbff60695ecb214e3270436ab21" + integrity sha512-IiJwMC8rdZE0+xiEZHeru6YoONC4rfPMqGm2W85jMIbkFvv5nFTwJVFHam2eFrN6txmoUYFAFXiv8ICVeTO0MA== + +css-what@^6.0.1: + version "6.1.0" + resolved "https://registry.yarnpkg.com/css-what/-/css-what-6.1.0.tgz#fb5effcf76f1ddea2c81bdfaa4de44e79bac70f4" + integrity sha512-HTUrgRJ7r4dsZKU6GjmpfRK1O76h97Z8MfS1G0FozR+oF2kG6Vfe8JE6zwrkbxigziPHinCJ+gCPjA9EaBDtRw== + +cssesc@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/cssesc/-/cssesc-3.0.0.tgz#37741919903b868565e1c09ea747445cd18983ee" + integrity sha512-/Tb/JcjK111nNScGob5MNtsntNM1aCNUDipB/TkwZFhyDrrE47SOx/18wF2bbjgc3ZzCSKW1T5nt5EbFoAz/Vg== + +csstype@^3.0.2, csstype@^3.1.2: + version "3.1.2" + resolved "https://registry.yarnpkg.com/csstype/-/csstype-3.1.2.tgz#1d4bf9d572f11c14031f0436e1c10bc1f571f50b" + integrity sha512-I7K1Uu0MBPzaFKg4nI5Q7Vs2t+3gWWW648spaF+Rg7pI9ds18Ugn+lvg4SHczUdKlHI5LWBXyqfS8+DufyBsgQ== + +"d3-array@2 - 3", "d3-array@2.10.0 - 3", d3-array@^3.1.6: + version "3.2.4" + resolved "https://registry.yarnpkg.com/d3-array/-/d3-array-3.2.4.tgz#15fec33b237f97ac5d7c986dc77da273a8ed0bb5" + integrity sha512-tdQAmyA18i4J7wprpYq8ClcxZy3SC31QMeByyCFyRt7BVHdREQZ5lpzoe5mFEYZUWe+oq8HBvk9JjpibyEV4Jg== + dependencies: + internmap "1 - 2" + +"d3-color@1 - 3": + version "3.1.0" + resolved "https://registry.yarnpkg.com/d3-color/-/d3-color-3.1.0.tgz#395b2833dfac71507f12ac2f7af23bf819de24e2" + integrity sha512-zg/chbXyeBtMQ1LbD/WSoW2DpC3I0mpmPdW+ynRTj/x2DAWYrIY7qeZIHidozwV24m4iavr15lNwIwLxRmOxhA== + +d3-ease@^3.0.1: + version "3.0.1" + resolved "https://registry.yarnpkg.com/d3-ease/-/d3-ease-3.0.1.tgz#9658ac38a2140d59d346160f1f6c30fda0bd12f4" + integrity sha512-wR/XK3D3XcLIZwpbvQwQ5fK+8Ykds1ip7A2Txe0yxncXSdq1L9skcG7blcedkOX+ZcgxGAmLX1FrRGbADwzi0w== + +"d3-format@1 - 3": + version "3.1.0" + resolved "https://registry.yarnpkg.com/d3-format/-/d3-format-3.1.0.tgz#9260e23a28ea5cb109e93b21a06e24e2ebd55641" + integrity sha512-YyUI6AEuY/Wpt8KWLgZHsIU86atmikuoOmCfommt0LYHiQSPjvX2AcFc38PX0CBpr2RCyZhjex+NS/LPOv6YqA== + +"d3-interpolate@1.2.0 - 3", d3-interpolate@^3.0.1: + version "3.0.1" + resolved "https://registry.yarnpkg.com/d3-interpolate/-/d3-interpolate-3.0.1.tgz#3c47aa5b32c5b3dfb56ef3fd4342078a632b400d" + integrity sha512-3bYs1rOD33uo8aqJfKP3JWPAibgw8Zm2+L9vBKEHJ2Rg+viTR7o5Mmv5mZcieN+FRYaAOWX5SJATX6k1PWz72g== + dependencies: + d3-color "1 - 3" + +d3-path@^3.1.0: + version "3.1.0" + resolved "https://registry.yarnpkg.com/d3-path/-/d3-path-3.1.0.tgz#22df939032fb5a71ae8b1800d61ddb7851c42526" + integrity sha512-p3KP5HCf/bvjBSSKuXid6Zqijx7wIfNW+J/maPs+iwR35at5JCbLUT0LzF1cnjbCHWhqzQTIN2Jpe8pRebIEFQ== + +d3-scale@^4.0.2: + version "4.0.2" + resolved "https://registry.yarnpkg.com/d3-scale/-/d3-scale-4.0.2.tgz#82b38e8e8ff7080764f8dcec77bd4be393689396" + integrity sha512-GZW464g1SH7ag3Y7hXjf8RoUuAFIqklOAq3MRl4OaWabTFJY9PN/E1YklhXLh+OQ3fM9yS2nOkCoS+WLZ6kvxQ== + dependencies: + d3-array "2.10.0 - 3" + d3-format "1 - 3" + d3-interpolate "1.2.0 - 3" + d3-time "2.1.1 - 3" + d3-time-format "2 - 4" + +d3-shape@^3.1.0: + version "3.2.0" + resolved "https://registry.yarnpkg.com/d3-shape/-/d3-shape-3.2.0.tgz#a1a839cbd9ba45f28674c69d7f855bcf91dfc6a5" + integrity sha512-SaLBuwGm3MOViRq2ABk3eLoxwZELpH6zhl3FbAoJ7Vm1gofKx6El1Ib5z23NUEhF9AsGl7y+dzLe5Cw2AArGTA== + dependencies: + d3-path "^3.1.0" + +"d3-time-format@2 - 4": + version "4.1.0" + resolved "https://registry.yarnpkg.com/d3-time-format/-/d3-time-format-4.1.0.tgz#7ab5257a5041d11ecb4fe70a5c7d16a195bb408a" + integrity sha512-dJxPBlzC7NugB2PDLwo9Q8JiTR3M3e4/XANkreKSUxF8vvXKqm1Yfq4Q5dl8budlunRVlUUaDUgFt7eA8D6NLg== + dependencies: + d3-time "1 - 3" + +"d3-time@1 - 3", "d3-time@2.1.1 - 3", d3-time@^3.0.0: + version "3.1.0" + resolved "https://registry.yarnpkg.com/d3-time/-/d3-time-3.1.0.tgz#9310db56e992e3c0175e1ef385e545e48a9bb5c7" + integrity sha512-VqKjzBLejbSMT4IgbmVgDjpkYrNWUYJnbCGo874u7MMKIWsILRX+OpX/gTk8MqjpT1A/c6HY2dCA77ZN0lkQ2Q== + dependencies: + d3-array "2 - 3" + +d3-timer@^3.0.1: + version "3.0.1" + resolved "https://registry.yarnpkg.com/d3-timer/-/d3-timer-3.0.1.tgz#6284d2a2708285b1abb7e201eda4380af35e63b0" + integrity sha512-ndfJ/JxxMd3nw31uyKoY2naivF+r29V+Lc0svZxe1JvvIRmi8hUsrMvdOwgS1o6uBHmiz91geQ0ylPP0aj1VUA== + +damerau-levenshtein@^1.0.8: + version "1.0.8" + resolved "https://registry.yarnpkg.com/damerau-levenshtein/-/damerau-levenshtein-1.0.8.tgz#b43d286ccbd36bc5b2f7ed41caf2d0aba1f8a6e7" + integrity sha512-sdQSFB7+llfUcQHUQO3+B8ERRj0Oa4w9POWMI/puGtuf7gFywGmkaLCElnudfTiKZV+NvHqL0ifzdrI8Ro7ESA== + +date-fns@^2.28.0: + version "2.30.0" + resolved "https://registry.yarnpkg.com/date-fns/-/date-fns-2.30.0.tgz#f367e644839ff57894ec6ac480de40cae4b0f4d0" + integrity sha512-fnULvOpxnC5/Vg3NCiWelDsLiUc9bRwAPs/+LfTLNvetFCtCTN+yQz15C/fs4AwX1R9K5GLtLfn8QW+dWisaAw== + dependencies: + "@babel/runtime" "^7.21.0" + +debug@2.6.9, debug@^2.6.9: + version "2.6.9" + resolved "https://registry.yarnpkg.com/debug/-/debug-2.6.9.tgz#5d128515df134ff327e90a4c93f4e077a536341f" + integrity sha512-bC7ElrdJaJnPbAP+1EotYvqZsb3ecl5wi6Bfi6BJTUcNowp6cvspg0jXznRTKDjm/E7AdgFBVeAPVMNcKGsHMA== + dependencies: + ms "2.0.0" + +debug@4, debug@^4.1.0, debug@^4.1.1, debug@^4.3.2, debug@^4.3.4: + version "4.3.4" + resolved "https://registry.yarnpkg.com/debug/-/debug-4.3.4.tgz#1319f6579357f2338d3337d2cdd4914bb5dcc865" + integrity sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ== + dependencies: + ms "2.1.2" + +debug@^3.2.6, debug@^3.2.7: + version "3.2.7" + resolved "https://registry.yarnpkg.com/debug/-/debug-3.2.7.tgz#72580b7e9145fb39b6676f9c5e5fb100b934179a" + integrity sha512-CFjzYYAi4ThfiQvizrFQevTTXHtnCqWfe7x1AhgEscTz6ZbLbfoLRLPugTQyBth6f8ZERVUSyWHFD/7Wu4t1XQ== + dependencies: + ms "^2.1.1" + +decimal.js-light@^2.4.1: + version "2.5.1" + resolved "https://registry.yarnpkg.com/decimal.js-light/-/decimal.js-light-2.5.1.tgz#134fd32508f19e208f4fb2f8dac0d2626a867934" + integrity sha512-qIMFpTMZmny+MMIitAB6D7iVPEorVw6YQRWkvarTkT4tBeSLLiHzcwj6q0MmYSFCiVpiqPJTJEYIrpcPzVEIvg== + +dedent@^0.7.0: + version "0.7.0" + resolved "https://registry.yarnpkg.com/dedent/-/dedent-0.7.0.tgz#2495ddbaf6eb874abb0e1be9df22d2e5a544326c" + integrity sha512-Q6fKUPqnAHAyhiUgFU7BUzLiv0kd8saH9al7tnu5Q/okj6dnupxyTgFIBjVzJATdfIAm9NAsvXNzjaKa+bxVyA== + +deep-equal@^2.0.5: + version "2.2.2" + resolved "https://registry.yarnpkg.com/deep-equal/-/deep-equal-2.2.2.tgz#9b2635da569a13ba8e1cc159c2f744071b115daa" + integrity sha512-xjVyBf0w5vH0I42jdAZzOKVldmPgSulmiyPRywoyq7HXC9qdgo17kxJE+rdnif5Tz6+pIrpJI8dCpMNLIGkUiA== + dependencies: + array-buffer-byte-length "^1.0.0" + call-bind "^1.0.2" + es-get-iterator "^1.1.3" + get-intrinsic "^1.2.1" + is-arguments "^1.1.1" + is-array-buffer "^3.0.2" + is-date-object "^1.0.5" + is-regex "^1.1.4" + is-shared-array-buffer "^1.0.2" + isarray "^2.0.5" + object-is "^1.1.5" + object-keys "^1.1.1" + object.assign "^4.1.4" + regexp.prototype.flags "^1.5.0" + side-channel "^1.0.4" + which-boxed-primitive "^1.0.2" + which-collection "^1.0.1" + which-typed-array "^1.1.9" + +deep-is@^0.1.3: + version "0.1.4" + resolved "https://registry.yarnpkg.com/deep-is/-/deep-is-0.1.4.tgz#a6f2dce612fadd2ef1f519b73551f17e85199831" + integrity sha512-oIPzksmTg4/MriiaYGO+okXDT7ztn/w3Eptv/+gSIdMdKsJo0u4CfYNFJPy+4SKMuCqGw2wxnA+URMg3t8a/bQ== + +deepmerge@^4.2.2: + version "4.3.1" + resolved "https://registry.yarnpkg.com/deepmerge/-/deepmerge-4.3.1.tgz#44b5f2147cd3b00d4b56137685966f26fd25dd4a" + integrity sha512-3sUqbMEc77XqpdNO7FRyRog+eW3ph+GYCbj+rK+uYyRMuwsVy0rMiVtPn+QJlKFvWP/1PYpapqYn0Me2knFn+A== + +default-browser-id@3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/default-browser-id/-/default-browser-id-3.0.0.tgz#bee7bbbef1f4e75d31f98f4d3f1556a14cea790c" + integrity sha512-OZ1y3y0SqSICtE8DE4S8YOE9UZOJ8wO16fKWVP5J1Qz42kV9jcnMVFrEE/noXb/ss3Q4pZIH79kxofzyNNtUNA== + dependencies: + bplist-parser "^0.2.0" + untildify "^4.0.0" + +defaults@^1.0.3: + version "1.0.4" + resolved "https://registry.yarnpkg.com/defaults/-/defaults-1.0.4.tgz#b0b02062c1e2aa62ff5d9528f0f98baa90978d7a" + integrity sha512-eFuaLoy/Rxalv2kr+lqMlUnrDWV+3j4pljOIJgLIhI058IQfWJ7vXhyEIHu+HtC738klGALYxOKDO0bQP3tg8A== + dependencies: + clone "^1.0.2" + +define-lazy-prop@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/define-lazy-prop/-/define-lazy-prop-2.0.0.tgz#3f7ae421129bcaaac9bc74905c98a0009ec9ee7f" + integrity sha512-Ds09qNh8yw3khSjiJjiUInaGX9xlqZDY7JVryGxdxV7NPeuqQfplOpQ66yJFZut3jLa5zOwkXw1g9EI2uKh4Og== + +define-properties@^1.1.3, define-properties@^1.1.4, define-properties@^1.2.0: + version "1.2.0" + resolved "https://registry.yarnpkg.com/define-properties/-/define-properties-1.2.0.tgz#52988570670c9eacedd8064f4a990f2405849bd5" + integrity sha512-xvqAVKGfT1+UAvPwKTVw/njhdQ8ZhXK4lI0bCIuCMrp2up9nPnaDftrLtmpTazqd1o+UY4zgzU+avtMbDP+ldA== + dependencies: + has-property-descriptors "^1.0.0" + object-keys "^1.1.1" + +defu@^6.1.2: + version "6.1.2" + resolved "https://registry.yarnpkg.com/defu/-/defu-6.1.2.tgz#1217cba167410a1765ba93893c6dbac9ed9d9e5c" + integrity sha512-+uO4+qr7msjNNWKYPHqN/3+Dx3NFkmIzayk2L1MyZQlvgZb/J1A0fo410dpKrN2SnqFjt8n4JL8fDJE0wIgjFQ== + +del@^6.0.0: + version "6.1.1" + resolved "https://registry.yarnpkg.com/del/-/del-6.1.1.tgz#3b70314f1ec0aa325c6b14eb36b95786671edb7a" + integrity sha512-ua8BhapfP0JUJKC/zV9yHHDW/rDoDxP4Zhn3AkA6/xT6gY7jYXJiaeyBZznYVujhZZET+UgcbZiQ7sN3WqcImg== + dependencies: + globby "^11.0.1" + graceful-fs "^4.2.4" + is-glob "^4.0.1" + is-path-cwd "^2.2.0" + is-path-inside "^3.0.2" + p-map "^4.0.0" + rimraf "^3.0.2" + slash "^3.0.0" + +delayed-stream@~1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/delayed-stream/-/delayed-stream-1.0.0.tgz#df3ae199acadfb7d440aaae0b29e2272b24ec619" + integrity sha512-ZySD7Nf91aLB0RxL4KGrKHBXl7Eds1DAmEdcoVawXnLD7SDhpNgtuII2aAkg7a7QS41jxPSZ17p4VdGnMHk3MQ== + +depd@2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/depd/-/depd-2.0.0.tgz#b696163cc757560d09cf22cc8fad1571b79e76df" + integrity sha512-g7nH6P6dyDioJogAAGprGpCtVImJhpPk/roCzdb3fIh61/s/nPsfR6onyMwkCAR/OlC3yBC0lESvUoQEAssIrw== + +dequal@^2.0.2, dequal@^2.0.3: + version "2.0.3" + resolved "https://registry.yarnpkg.com/dequal/-/dequal-2.0.3.tgz#2644214f1997d39ed0ee0ece72335490a7ac67be" + integrity sha512-0je+qPKHEMohvfRTCEo3CrPG6cAzAYgmzKyxRiYSSDkS6eGJdyVJm7WaYA5ECaAD9wLB2T4EEeymA5aFVcYXCA== + +des.js@^1.0.0: + version "1.1.0" + resolved "https://registry.yarnpkg.com/des.js/-/des.js-1.1.0.tgz#1d37f5766f3bbff4ee9638e871a8768c173b81da" + integrity sha512-r17GxjhUCjSRy8aiJpr8/UadFIzMzJGexI3Nmz4ADi9LYSFx4gTBp80+NaX/YsXWWLhpZ7v/v/ubEc/bCNfKwg== + dependencies: + inherits "^2.0.1" + minimalistic-assert "^1.0.0" + +destroy@1.2.0: + version "1.2.0" + resolved "https://registry.yarnpkg.com/destroy/-/destroy-1.2.0.tgz#4803735509ad8be552934c67df614f94e66fa015" + integrity sha512-2sJGJTaXIIaR1w4iJSNoN0hnMY7Gpc/n8D4qSCJw8QqFWXf7cuAgnEHxBpweaVcPevC2l3KpjYCx3NypQQgaJg== + +detect-indent@^6.1.0: + version "6.1.0" + resolved "https://registry.yarnpkg.com/detect-indent/-/detect-indent-6.1.0.tgz#592485ebbbf6b3b1ab2be175c8393d04ca0d57e6" + integrity sha512-reYkTUJAZb9gUuZ2RvVCNhVHdg62RHnJ7WJl8ftMi4diZ6NWlciOzQN88pUhSELEwflJht4oQDv0F0BMlwaYtA== + +detect-node-es@^1.1.0: + version "1.1.0" + resolved "https://registry.yarnpkg.com/detect-node-es/-/detect-node-es-1.1.0.tgz#163acdf643330caa0b4cd7c21e7ee7755d6fa493" + integrity sha512-ypdmJU/TbBby2Dxibuv7ZLW3Bs1QEmM7nHjEANfohJLvE0XVujisn1qPJcZxg+qDucsr+bP6fLD1rPS3AhJ7EQ== + +detect-package-manager@^2.0.1: + version "2.0.1" + resolved "https://registry.yarnpkg.com/detect-package-manager/-/detect-package-manager-2.0.1.tgz#6b182e3ae5e1826752bfef1de9a7b828cffa50d8" + integrity sha512-j/lJHyoLlWi6G1LDdLgvUtz60Zo5GEj+sVYtTVXnYLDPuzgC3llMxonXym9zIwhhUII8vjdw0LXxavpLqTbl1A== + dependencies: + execa "^5.1.1" + +detect-port@^1.3.0: + version "1.5.1" + resolved "https://registry.yarnpkg.com/detect-port/-/detect-port-1.5.1.tgz#451ca9b6eaf20451acb0799b8ab40dff7718727b" + integrity sha512-aBzdj76lueB6uUst5iAs7+0H/oOjqI5D16XUWxlWMIMROhcM0rfsNVk93zTngq1dDNpoXRr++Sus7ETAExppAQ== + dependencies: + address "^1.0.1" + debug "4" + +didyoumean@^1.2.2: + version "1.2.2" + resolved "https://registry.yarnpkg.com/didyoumean/-/didyoumean-1.2.2.tgz#989346ffe9e839b4555ecf5666edea0d3e8ad037" + integrity sha512-gxtyfqMg7GKyhQmb056K7M3xszy/myH8w+B4RT+QXBQsvAOdc3XymqDDPHx1BgPgsdAA5SIifona89YtRATDzw== + +diffie-hellman@^5.0.0: + version "5.0.3" + resolved "https://registry.yarnpkg.com/diffie-hellman/-/diffie-hellman-5.0.3.tgz#40e8ee98f55a2149607146921c63e1ae5f3d2875" + integrity sha512-kqag/Nl+f3GwyK25fhUMYj81BUOrZ9IuJsjIcDE5icNM9FJHAVm3VcUDxdLPoQtTuUylWm6ZIknYJwwaPxsUzg== + dependencies: + bn.js "^4.1.0" + miller-rabin "^4.0.0" + randombytes "^2.0.0" + +dir-glob@^3.0.1: + version "3.0.1" + resolved "https://registry.yarnpkg.com/dir-glob/-/dir-glob-3.0.1.tgz#56dbf73d992a4a93ba1584f4534063fd2e41717f" + integrity sha512-WkrWp9GR4KXfKGYzOLmTuGVi1UWFfws377n9cc55/tb6DuqyF6pcQ5AbiHEshaDpY9v6oaSr2XCDidGmMwdzIA== + dependencies: + path-type "^4.0.0" + +dlv@^1.1.3: + version "1.1.3" + resolved "https://registry.yarnpkg.com/dlv/-/dlv-1.1.3.tgz#5c198a8a11453596e751494d49874bc7732f2e79" + integrity sha512-+HlytyjlPKnIG8XuRG8WvmBP8xs8P71y+SKKS6ZXWoEgLuePxtDoUEiH7WkdePWrQ5JBpE6aoVqfZfJUQkjXwA== + +doctrine@^2.1.0: + version "2.1.0" + resolved "https://registry.yarnpkg.com/doctrine/-/doctrine-2.1.0.tgz#5cd01fc101621b42c4cd7f5d1a66243716d3f39d" + integrity sha512-35mSku4ZXK0vfCuHEDAwt55dg2jNajHZ1odvF+8SSr82EsZY4QmXfuWso8oEd8zRhVObSN18aM0CjSdoBX7zIw== + dependencies: + esutils "^2.0.2" + +doctrine@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/doctrine/-/doctrine-3.0.0.tgz#addebead72a6574db783639dc87a121773973961" + integrity sha512-yS+Q5i3hBf7GBkd4KG8a7eBNNWNGLTaEwwYWUijIYM7zrlYDM0BFXHjjPWlWZ1Rg7UaddZeIDmi9jF3HmqiQ2w== + dependencies: + esutils "^2.0.2" + +dom-accessibility-api@^0.5.9: + version "0.5.16" + resolved "https://registry.yarnpkg.com/dom-accessibility-api/-/dom-accessibility-api-0.5.16.tgz#5a7429e6066eb3664d911e33fb0e45de8eb08453" + integrity sha512-X7BJ2yElsnOJ30pZF4uIIDfBEVgF4XEBxL9Bxhy6dnrm5hkzqmsWHGTiHqRiITNhMyFLyAiWndIJP7Z1NTteDg== + +dom-converter@^0.2.0: + version "0.2.0" + resolved "https://registry.yarnpkg.com/dom-converter/-/dom-converter-0.2.0.tgz#6721a9daee2e293682955b6afe416771627bb768" + integrity sha512-gd3ypIPfOMr9h5jIKq8E3sHOTCjeirnl0WK5ZdS1AW0Odt0b1PaWaHdJ4Qk4klv+YB9aJBS7mESXjFoDQPu6DA== + dependencies: + utila "~0.4" + +dom-helpers@^3.4.0: + version "3.4.0" + resolved "https://registry.yarnpkg.com/dom-helpers/-/dom-helpers-3.4.0.tgz#e9b369700f959f62ecde5a6babde4bccd9169af8" + integrity sha512-LnuPJ+dwqKDIyotW1VzmOZ5TONUN7CwkCR5hrgawTUbkBGYdeoNLZo6nNfGkCrjtE1nXXaj7iMMpDa8/d9WoIA== + dependencies: + "@babel/runtime" "^7.1.2" + +dom-helpers@^5.0.1: + version "5.2.1" + resolved "https://registry.yarnpkg.com/dom-helpers/-/dom-helpers-5.2.1.tgz#d9400536b2bf8225ad98fe052e029451ac40e902" + integrity sha512-nRCa7CK3VTrM2NmGkIy4cbK7IZlgBE/PYMn55rrXefr5xXDP0LdtfPnblFDoVdcAfslJ7or6iqAUnx0CCGIWQA== + dependencies: + "@babel/runtime" "^7.8.7" + csstype "^3.0.2" + +dom-serializer@^1.0.1: + version "1.4.1" + resolved "https://registry.yarnpkg.com/dom-serializer/-/dom-serializer-1.4.1.tgz#de5d41b1aea290215dc45a6dae8adcf1d32e2d30" + integrity sha512-VHwB3KfrcOOkelEG2ZOfxqLZdfkil8PtJi4P8N2MMXucZq2yLp75ClViUlOVwyoHEDjYU433Aq+5zWP61+RGag== + dependencies: + domelementtype "^2.0.1" + domhandler "^4.2.0" + entities "^2.0.0" + +domain-browser@^4.22.0: + version "4.22.0" + resolved "https://registry.yarnpkg.com/domain-browser/-/domain-browser-4.22.0.tgz#6ddd34220ec281f9a65d3386d267ddd35c491f9f" + integrity sha512-IGBwjF7tNk3cwypFNH/7bfzBcgSCbaMOD3GsaY1AU/JRrnHnYgEM0+9kQt52iZxjNsjBtJYtao146V+f8jFZNw== + +domelementtype@^2.0.1, domelementtype@^2.2.0: + version "2.3.0" + resolved "https://registry.yarnpkg.com/domelementtype/-/domelementtype-2.3.0.tgz#5c45e8e869952626331d7aab326d01daf65d589d" + integrity sha512-OLETBj6w0OsagBwdXnPdN0cnMfF9opN69co+7ZrbfPGrdpPVNBUj02spi6B1N7wChLQiPn4CSH/zJvXw56gmHw== + +domhandler@^4.0.0, domhandler@^4.2.0, domhandler@^4.3.1: + version "4.3.1" + resolved "https://registry.yarnpkg.com/domhandler/-/domhandler-4.3.1.tgz#8d792033416f59d68bc03a5aa7b018c1ca89279c" + integrity sha512-GrwoxYN+uWlzO8uhUXRl0P+kHE4GtVPfYzVLcUxPL7KNdHKj66vvlhiweIHqYYXWlw+T8iLMp42Lm67ghw4WMQ== + dependencies: + domelementtype "^2.2.0" + +domutils@^2.5.2, domutils@^2.8.0: + version "2.8.0" + resolved "https://registry.yarnpkg.com/domutils/-/domutils-2.8.0.tgz#4437def5db6e2d1f5d6ee859bd95ca7d02048135" + integrity sha512-w96Cjofp72M5IIhpjgobBimYEfoPjx1Vx0BSX9P30WBdZW2WIKU0T1Bd0kz2eNZ9ikjKgHbEyKx8BB6H1L3h3A== + dependencies: + dom-serializer "^1.0.1" + domelementtype "^2.2.0" + domhandler "^4.2.0" + +dot-case@^3.0.4: + version "3.0.4" + resolved "https://registry.yarnpkg.com/dot-case/-/dot-case-3.0.4.tgz#9b2b670d00a431667a8a75ba29cd1b98809ce751" + integrity sha512-Kv5nKlh6yRrdrGvxeJ2e5y2eRUpkUosIW4A2AS38zwSz27zu7ufDwQPi5Jhs3XAlGNetl3bmnGhQsMtkKJnj3w== + dependencies: + no-case "^3.0.4" + tslib "^2.0.3" + +dotenv-expand@^10.0.0: + version "10.0.0" + resolved "https://registry.yarnpkg.com/dotenv-expand/-/dotenv-expand-10.0.0.tgz#12605d00fb0af6d0a592e6558585784032e4ef37" + integrity sha512-GopVGCpVS1UKH75VKHGuQFqS1Gusej0z4FyQkPdwjil2gNIv+LNsqBlboOzpJFZKVT95GkCyWJbBSdFEFUWI2A== + +dotenv@^16.0.0: + version "16.3.1" + resolved "https://registry.yarnpkg.com/dotenv/-/dotenv-16.3.1.tgz#369034de7d7e5b120972693352a3bf112172cc3e" + integrity sha512-IPzF4w4/Rd94bA9imS68tZBaYyBWSCE47V1RGuMrB94iyTOIEwRmVL2x/4An+6mETpLrKJ5hQkB8W4kFAadeIQ== + +duplexify@^3.5.0, duplexify@^3.6.0: + version "3.7.1" + resolved "https://registry.yarnpkg.com/duplexify/-/duplexify-3.7.1.tgz#2a4df5317f6ccfd91f86d6fd25d8d8a103b88309" + integrity sha512-07z8uv2wMyS51kKhD1KsdXJg5WQ6t93RneqRxUHnskXVtlYYkLqM0gqStQZ3pj073g687jPCHrqNfCzawLYh5g== + dependencies: + end-of-stream "^1.0.0" + inherits "^2.0.1" + readable-stream "^2.0.0" + stream-shift "^1.0.0" + +eastasianwidth@^0.2.0: + version "0.2.0" + resolved "https://registry.yarnpkg.com/eastasianwidth/-/eastasianwidth-0.2.0.tgz#696ce2ec0aa0e6ea93a397ffcf24aa7840c827cb" + integrity sha512-I88TYZWc9XiYHRQ4/3c5rjjfgkjhLyW2luGIheGERbNQ6OY7yTybanSpDXZa8y7VUP9YmDcYa+eyq4ca7iLqWA== + +ee-first@1.1.1: + version "1.1.1" + resolved "https://registry.yarnpkg.com/ee-first/-/ee-first-1.1.1.tgz#590c61156b0ae2f4f0255732a158b266bc56b21d" + integrity sha512-WMwm9LhRUo+WUaRN+vRuETqG89IgZphVSNkdFgeb6sS/E4OrDIN7t48CAewSHXc6C8lefD8KKfr5vY61brQlow== + +ejs@^3.1.8: + version "3.1.9" + resolved "https://registry.yarnpkg.com/ejs/-/ejs-3.1.9.tgz#03c9e8777fe12686a9effcef22303ca3d8eeb361" + integrity sha512-rC+QVNMJWv+MtPgkt0y+0rVEIdbtxVADApW9JXrUVlzHetgcyczP/E7DJmWJ4fJCZF2cPcBk0laWO9ZHMG3DmQ== + dependencies: + jake "^10.8.5" + +electron-to-chromium@^1.4.477: + version "1.4.508" + resolved "https://registry.yarnpkg.com/electron-to-chromium/-/electron-to-chromium-1.4.508.tgz#5641ff2f5ba11df4bd960fe6a2f9f70aa8b9af96" + integrity sha512-FFa8QKjQK/A5QuFr2167myhMesGrhlOBD+3cYNxO9/S4XzHEXesyTD/1/xF644gC8buFPz3ca6G1LOQD0tZrrg== + +elliptic@^6.5.3: + version "6.5.4" + resolved "https://registry.yarnpkg.com/elliptic/-/elliptic-6.5.4.tgz#da37cebd31e79a1367e941b592ed1fbebd58abbb" + integrity sha512-iLhC6ULemrljPZb+QutR5TQGB+pdW6KGD5RSegS+8sorOZT+rdQFbsQFJgvN3eRqNALqJer4oQ16YvJHlU8hzQ== + dependencies: + bn.js "^4.11.9" + brorand "^1.1.0" + hash.js "^1.0.0" + hmac-drbg "^1.0.1" + inherits "^2.0.4" + minimalistic-assert "^1.0.1" + minimalistic-crypto-utils "^1.0.1" + +email-addresses@^5.0.0: + version "5.0.0" + resolved "https://registry.yarnpkg.com/email-addresses/-/email-addresses-5.0.0.tgz#7ae9e7f58eef7d5e3e2c2c2d3ea49b78dc854fa6" + integrity sha512-4OIPYlA6JXqtVn8zpHpGiI7vE6EQOAg16aGnDMIAlZVinnoZ8208tW1hAbjWydgN/4PLTT9q+O1K6AH/vALJGw== + +emoji-regex@^10.2.1: + version "10.2.1" + resolved "https://registry.yarnpkg.com/emoji-regex/-/emoji-regex-10.2.1.tgz#a41c330d957191efd3d9dfe6e1e8e1e9ab048b3f" + integrity sha512-97g6QgOk8zlDRdgq1WxwgTMgEWGVAQvB5Fdpgc1MkNy56la5SKP9GsMXKDOdqwn90/41a8yPwIGk1Y6WVbeMQA== + +emoji-regex@^8.0.0: + version "8.0.0" + resolved "https://registry.yarnpkg.com/emoji-regex/-/emoji-regex-8.0.0.tgz#e818fd69ce5ccfcb404594f842963bf53164cc37" + integrity sha512-MSjYzcWNOA0ewAHpz0MxpYFvwg6yjy1NG3xteoqz644VCo/RPgnr1/GGt+ic3iJTzQ8Eu3TdM14SawnVUmGE6A== + +emoji-regex@^9.2.2: + version "9.2.2" + resolved "https://registry.yarnpkg.com/emoji-regex/-/emoji-regex-9.2.2.tgz#840c8803b0d8047f4ff0cf963176b32d4ef3ed72" + integrity sha512-L18DaJsXSUk2+42pv8mLs5jJT2hqFkFE4j21wOmgbUqsZ2hL72NsUU785g9RXgo3s0ZNgVl42TiHp3ZtOv/Vyg== + +emojis-list@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/emojis-list/-/emojis-list-3.0.0.tgz#5570662046ad29e2e916e71aae260abdff4f6a78" + integrity sha512-/kyM18EfinwXZbno9FyUGeFh87KC8HRQBQGildHZbEuRyWFOmv1U10o9BBp8XVZDVNNuQKyIGIu5ZYAAXJ0V2Q== + +encodeurl@~1.0.2: + version "1.0.2" + resolved "https://registry.yarnpkg.com/encodeurl/-/encodeurl-1.0.2.tgz#ad3ff4c86ec2d029322f5a02c3a9a606c95b3f59" + integrity sha512-TPJXq8JqFaVYm2CWmPvnP2Iyo4ZSM7/QKcSmuMLDObfpH5fi7RUGmd/rTDf+rut/saiDiQEeVTNgAmJEdAOx0w== + +end-of-stream@^1.0.0, end-of-stream@^1.1.0, end-of-stream@^1.4.1: + version "1.4.4" + resolved "https://registry.yarnpkg.com/end-of-stream/-/end-of-stream-1.4.4.tgz#5ae64a5f45057baf3626ec14da0ca5e4b2431eb0" + integrity sha512-+uw1inIHVPQoaVuHzRyXd21icM+cnt4CzD5rW+NC1wjOUSTOs+Te7FOv7AhN7vS9x/oIyhLP5PR1H+phQAHu5Q== + dependencies: + once "^1.4.0" + +endent@^2.0.1: + version "2.1.0" + resolved "https://registry.yarnpkg.com/endent/-/endent-2.1.0.tgz#5aaba698fb569e5e18e69e1ff7a28ff35373cd88" + integrity sha512-r8VyPX7XL8U01Xgnb1CjZ3XV+z90cXIJ9JPE/R9SEC9vpw2P6CfsRPJmp20DppC5N7ZAMCmjYkJIa744Iyg96w== + dependencies: + dedent "^0.7.0" + fast-json-parse "^1.0.3" + objectorarray "^1.0.5" + +enhanced-resolve@^5.12.0, enhanced-resolve@^5.15.0, enhanced-resolve@^5.7.0: + version "5.15.0" + resolved "https://registry.yarnpkg.com/enhanced-resolve/-/enhanced-resolve-5.15.0.tgz#1af946c7d93603eb88e9896cee4904dc012e9c35" + integrity sha512-LXYT42KJ7lpIKECr2mAXIaMldcNCh/7E0KBKOu4KSfkHmP+mZmSs+8V5gBAqisWBy0OO4W5Oyys0GO1Y8KtdKg== + dependencies: + graceful-fs "^4.2.4" + tapable "^2.2.0" + +entities@^2.0.0: + version "2.2.0" + resolved "https://registry.yarnpkg.com/entities/-/entities-2.2.0.tgz#098dc90ebb83d8dffa089d55256b351d34c4da55" + integrity sha512-p92if5Nz619I0w+akJrLZH0MX0Pb5DX39XOwQTtXSdQQOaYH03S1uIQp4mhOZtAXrxq4ViO67YTiLBo2638o9A== + +envinfo@^7.7.3: + version "7.10.0" + resolved "https://registry.yarnpkg.com/envinfo/-/envinfo-7.10.0.tgz#55146e3909cc5fe63c22da63fb15b05aeac35b13" + integrity sha512-ZtUjZO6l5mwTHvc1L9+1q5p/R3wTopcfqMW8r5t8SJSKqeVI/LtajORwRFEKpEFuekjD0VBjwu1HMxL4UalIRw== + +errno@^0.1.1: + version "0.1.8" + resolved "https://registry.yarnpkg.com/errno/-/errno-0.1.8.tgz#8bb3e9c7d463be4976ff888f76b4809ebc2e811f" + integrity sha512-dJ6oBr5SQ1VSd9qkk7ByRgb/1SH4JZjCHSW/mr63/QcXO9zLVxvJ6Oy13nio03rxpSnVDDjFor75SjVeZWPW/A== + dependencies: + prr "~1.0.1" + +error-ex@^1.3.1: + version "1.3.2" + resolved "https://registry.yarnpkg.com/error-ex/-/error-ex-1.3.2.tgz#b4ac40648107fdcdcfae242f428bea8a14d4f1bf" + integrity sha512-7dFHNmqeFSEt2ZBsCriorKnn3Z2pj+fd9kmI6QoWw4//DL+icEBfc0U7qJCisqrTsKTjw4fNFy2pW9OqStD84g== + dependencies: + is-arrayish "^0.2.1" + +error-stack-parser@^2.0.6: + version "2.1.4" + resolved "https://registry.yarnpkg.com/error-stack-parser/-/error-stack-parser-2.1.4.tgz#229cb01cdbfa84440bfa91876285b94680188286" + integrity sha512-Sk5V6wVazPhq5MhpO+AUxJn5x7XSXGl1R93Vn7i+zS15KDVxQijejNCrz8340/2bgLBjR9GtEG8ZVKONDjcqGQ== + dependencies: + stackframe "^1.3.4" + +es-abstract@^1.20.4, es-abstract@^1.22.1: + version "1.22.1" + resolved "https://registry.yarnpkg.com/es-abstract/-/es-abstract-1.22.1.tgz#8b4e5fc5cefd7f1660f0f8e1a52900dfbc9d9ccc" + integrity sha512-ioRRcXMO6OFyRpyzV3kE1IIBd4WG5/kltnzdxSCqoP8CMGs/Li+M1uF5o7lOkZVFjDs+NLesthnF66Pg/0q0Lw== + dependencies: + array-buffer-byte-length "^1.0.0" + arraybuffer.prototype.slice "^1.0.1" + available-typed-arrays "^1.0.5" + call-bind "^1.0.2" + es-set-tostringtag "^2.0.1" + es-to-primitive "^1.2.1" + function.prototype.name "^1.1.5" + get-intrinsic "^1.2.1" + get-symbol-description "^1.0.0" + globalthis "^1.0.3" + gopd "^1.0.1" + has "^1.0.3" + has-property-descriptors "^1.0.0" + has-proto "^1.0.1" + has-symbols "^1.0.3" + internal-slot "^1.0.5" + is-array-buffer "^3.0.2" + is-callable "^1.2.7" + is-negative-zero "^2.0.2" + is-regex "^1.1.4" + is-shared-array-buffer "^1.0.2" + is-string "^1.0.7" + is-typed-array "^1.1.10" + is-weakref "^1.0.2" + object-inspect "^1.12.3" + object-keys "^1.1.1" + object.assign "^4.1.4" + regexp.prototype.flags "^1.5.0" + safe-array-concat "^1.0.0" + safe-regex-test "^1.0.0" + string.prototype.trim "^1.2.7" + string.prototype.trimend "^1.0.6" + string.prototype.trimstart "^1.0.6" + typed-array-buffer "^1.0.0" + typed-array-byte-length "^1.0.0" + typed-array-byte-offset "^1.0.0" + typed-array-length "^1.0.4" + unbox-primitive "^1.0.2" + which-typed-array "^1.1.10" + +es-get-iterator@^1.1.3: + version "1.1.3" + resolved "https://registry.yarnpkg.com/es-get-iterator/-/es-get-iterator-1.1.3.tgz#3ef87523c5d464d41084b2c3c9c214f1199763d6" + integrity sha512-sPZmqHBe6JIiTfN5q2pEi//TwxmAFHwj/XEuYjTuse78i8KxaqMTTzxPoFKuzRpDpTJ+0NAbpfenkmH2rePtuw== + dependencies: + call-bind "^1.0.2" + get-intrinsic "^1.1.3" + has-symbols "^1.0.3" + is-arguments "^1.1.1" + is-map "^2.0.2" + is-set "^2.0.2" + is-string "^1.0.7" + isarray "^2.0.5" + stop-iteration-iterator "^1.0.0" + +es-iterator-helpers@^1.0.12: + version "1.0.14" + resolved "https://registry.yarnpkg.com/es-iterator-helpers/-/es-iterator-helpers-1.0.14.tgz#19cd7903697d97e21198f3293b55e8985791c365" + integrity sha512-JgtVnwiuoRuzLvqelrvN3Xu7H9bu2ap/kQ2CrM62iidP8SKuD99rWU3CJy++s7IVL2qb/AjXPGR/E7i9ngd/Cw== + dependencies: + asynciterator.prototype "^1.0.0" + call-bind "^1.0.2" + define-properties "^1.2.0" + es-abstract "^1.22.1" + es-set-tostringtag "^2.0.1" + function-bind "^1.1.1" + get-intrinsic "^1.2.1" + globalthis "^1.0.3" + has-property-descriptors "^1.0.0" + has-proto "^1.0.1" + has-symbols "^1.0.3" + internal-slot "^1.0.5" + iterator.prototype "^1.1.0" + safe-array-concat "^1.0.0" + +es-module-lexer@^1.2.1: + version "1.3.0" + resolved "https://registry.yarnpkg.com/es-module-lexer/-/es-module-lexer-1.3.0.tgz#6be9c9e0b4543a60cd166ff6f8b4e9dae0b0c16f" + integrity sha512-vZK7T0N2CBmBOixhmjdqx2gWVbFZ4DXZ/NyRMZVlJXPa7CyFS+/a4QQsDGDQy9ZfEzxFuNEsMLeQJnKP2p5/JA== + +es-set-tostringtag@^2.0.1: + version "2.0.1" + resolved "https://registry.yarnpkg.com/es-set-tostringtag/-/es-set-tostringtag-2.0.1.tgz#338d502f6f674301d710b80c8592de8a15f09cd8" + integrity sha512-g3OMbtlwY3QewlqAiMLI47KywjWZoEytKr8pf6iTC8uJq5bIAH52Z9pnQ8pVL6whrCto53JZDuUIsifGeLorTg== + dependencies: + get-intrinsic "^1.1.3" + has "^1.0.3" + has-tostringtag "^1.0.0" + +es-shim-unscopables@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/es-shim-unscopables/-/es-shim-unscopables-1.0.0.tgz#702e632193201e3edf8713635d083d378e510241" + integrity sha512-Jm6GPcCdC30eMLbZ2x8z2WuRwAws3zTBBKuusffYVUrNj/GVSUAZ+xKMaUpfNDR5IbyNA5LJbaecoUVbmUcB1w== + dependencies: + has "^1.0.3" + +es-to-primitive@^1.2.1: + version "1.2.1" + resolved "https://registry.yarnpkg.com/es-to-primitive/-/es-to-primitive-1.2.1.tgz#e55cd4c9cdc188bcefb03b366c736323fc5c898a" + integrity sha512-QCOllgZJtaUo9miYBcLChTUaHNjJF3PYs1VidD7AwiEj1kYxKeQTctLAezAOH5ZKRH0g2IgPn6KwB4IT8iRpvA== + dependencies: + is-callable "^1.1.4" + is-date-object "^1.0.1" + is-symbol "^1.0.2" + +es6-object-assign@^1.1.0: + version "1.1.0" + resolved "https://registry.yarnpkg.com/es6-object-assign/-/es6-object-assign-1.1.0.tgz#c2c3582656247c39ea107cb1e6652b6f9f24523c" + integrity sha512-MEl9uirslVwqQU369iHNWZXsI8yaZYGg/D65aOgZkeyFJwHYSxilf7rQzXKI7DdDuBPrBXbfk3sl9hJhmd5AUw== + +esbuild-plugin-alias@^0.2.1: + version "0.2.1" + resolved "https://registry.yarnpkg.com/esbuild-plugin-alias/-/esbuild-plugin-alias-0.2.1.tgz#45a86cb941e20e7c2bc68a2bea53562172494fcb" + integrity sha512-jyfL/pwPqaFXyKnj8lP8iLk6Z0m099uXR45aSN8Av1XD4vhvQutxxPzgA2bTcAwQpa1zCXDcWOlhFgyP3GKqhQ== + +esbuild-register@^3.4.0: + version "3.4.2" + resolved "https://registry.yarnpkg.com/esbuild-register/-/esbuild-register-3.4.2.tgz#1e39ee0a77e8f320a9790e68c64c3559620b9175" + integrity sha512-kG/XyTDyz6+YDuyfB9ZoSIOOmgyFCH+xPRtsCa8W85HLRV5Csp+o3jWVbOSHgSLfyLc5DmP+KFDNwty4mEjC+Q== + dependencies: + debug "^4.3.4" + +esbuild@^0.18.0: + version "0.18.20" + resolved "https://registry.yarnpkg.com/esbuild/-/esbuild-0.18.20.tgz#4709f5a34801b43b799ab7d6d82f7284a9b7a7a6" + integrity sha512-ceqxoedUrcayh7Y7ZX6NdbbDzGROiyVBgC4PriJThBKSVPWnnFHZAkfI1lJT8QFkOwH4qOS2SJkS4wvpGl8BpA== + optionalDependencies: + "@esbuild/android-arm" "0.18.20" + "@esbuild/android-arm64" "0.18.20" + "@esbuild/android-x64" "0.18.20" + "@esbuild/darwin-arm64" "0.18.20" + "@esbuild/darwin-x64" "0.18.20" + "@esbuild/freebsd-arm64" "0.18.20" + "@esbuild/freebsd-x64" "0.18.20" + "@esbuild/linux-arm" "0.18.20" + "@esbuild/linux-arm64" "0.18.20" + "@esbuild/linux-ia32" "0.18.20" + "@esbuild/linux-loong64" "0.18.20" + "@esbuild/linux-mips64el" "0.18.20" + "@esbuild/linux-ppc64" "0.18.20" + "@esbuild/linux-riscv64" "0.18.20" + "@esbuild/linux-s390x" "0.18.20" + "@esbuild/linux-x64" "0.18.20" + "@esbuild/netbsd-x64" "0.18.20" + "@esbuild/openbsd-x64" "0.18.20" + "@esbuild/sunos-x64" "0.18.20" + "@esbuild/win32-arm64" "0.18.20" + "@esbuild/win32-ia32" "0.18.20" + "@esbuild/win32-x64" "0.18.20" + +escalade@^3.1.1: + version "3.1.1" + resolved "https://registry.yarnpkg.com/escalade/-/escalade-3.1.1.tgz#d8cfdc7000965c5a0174b4a82eaa5c0552742e40" + integrity sha512-k0er2gUkLf8O0zKJiAhmkTnJlTvINGv7ygDNPbeIsX/TJjGJZHuh9B2UxbsaEkmlEo9MfhrSzmhIlhRlI2GXnw== + +escape-html@~1.0.3: + version "1.0.3" + resolved "https://registry.yarnpkg.com/escape-html/-/escape-html-1.0.3.tgz#0258eae4d3d0c0974de1c169188ef0051d1d1988" + integrity sha512-NiSupZ4OeuGwr68lGIeym/ksIZMJodUGOSCZ/FSnTxcrekbvqrgdUxlJOMpijaKZVjAJrWrGs/6Jy8OMuyj9ow== + +escape-string-regexp@^1.0.2, escape-string-regexp@^1.0.5: + version "1.0.5" + resolved "https://registry.yarnpkg.com/escape-string-regexp/-/escape-string-regexp-1.0.5.tgz#1b61c0562190a8dff6ae3bb2cf0200ca130b86d4" + integrity sha512-vbRorB5FUQWvla16U8R/qgaFIya2qGzwDrNmCZuYKrbdSUMG6I1ZCGQRefkRVhuOkIGVne7BQ35DSfo1qvJqFg== + +escape-string-regexp@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/escape-string-regexp/-/escape-string-regexp-4.0.0.tgz#14ba83a5d373e3d311e5afca29cf5bfad965bf34" + integrity sha512-TtpcNJ3XAzx3Gq8sWRzJaVajRs0uVxA2YAkdb1jm2YkPz4G6egUFAyA3n5vtEIZefPk5Wa4UXbKuS5fKkJWdgA== + +escodegen@^2.1.0: + version "2.1.0" + resolved "https://registry.yarnpkg.com/escodegen/-/escodegen-2.1.0.tgz#ba93bbb7a43986d29d6041f99f5262da773e2e17" + integrity sha512-2NlIDTwUWJN0mRPQOdtQBzbUHvdGY2P1VXSyU83Q3xKxM7WHX2Ql8dKq782Q9TgQUNOLEzEYu9bzLNj1q88I5w== + dependencies: + esprima "^4.0.1" + estraverse "^5.2.0" + esutils "^2.0.2" + optionalDependencies: + source-map "~0.6.1" + +eslint-config-next@13.4.19: + version "13.4.19" + resolved "https://registry.yarnpkg.com/eslint-config-next/-/eslint-config-next-13.4.19.tgz#f46be9d4bd9e52755f846338456132217081d7f8" + integrity sha512-WE8367sqMnjhWHvR5OivmfwENRQ1ixfNE9hZwQqNCsd+iM3KnuMc1V8Pt6ytgjxjf23D+xbesADv9x3xaKfT3g== + dependencies: + "@next/eslint-plugin-next" "13.4.19" + "@rushstack/eslint-patch" "^1.1.3" + "@typescript-eslint/parser" "^5.4.2 || ^6.0.0" + eslint-import-resolver-node "^0.3.6" + eslint-import-resolver-typescript "^3.5.2" + eslint-plugin-import "^2.26.0" + eslint-plugin-jsx-a11y "^6.5.1" + eslint-plugin-react "^7.31.7" + eslint-plugin-react-hooks "^4.5.0 || 5.0.0-canary-7118f5dd7-20230705" + +eslint-config-prettier@^9.0.0: + version "9.0.0" + resolved "https://registry.yarnpkg.com/eslint-config-prettier/-/eslint-config-prettier-9.0.0.tgz#eb25485946dd0c66cd216a46232dc05451518d1f" + integrity sha512-IcJsTkJae2S35pRsRAwoCE+925rJJStOdkKnLVgtE+tEpqU0EVVM7OqrwxqgptKdX29NUwC82I5pXsGFIgSevw== + +eslint-import-resolver-node@^0.3.6, eslint-import-resolver-node@^0.3.7: + version "0.3.9" + resolved "https://registry.yarnpkg.com/eslint-import-resolver-node/-/eslint-import-resolver-node-0.3.9.tgz#d4eaac52b8a2e7c3cd1903eb00f7e053356118ac" + integrity sha512-WFj2isz22JahUv+B788TlO3N6zL3nNJGU8CcZbPZvVEkBPaJdCV4vy5wyghty5ROFbCRnm132v8BScu5/1BQ8g== + dependencies: + debug "^3.2.7" + is-core-module "^2.13.0" + resolve "^1.22.4" + +eslint-import-resolver-typescript@^3.5.2: + version "3.6.0" + resolved "https://registry.yarnpkg.com/eslint-import-resolver-typescript/-/eslint-import-resolver-typescript-3.6.0.tgz#36f93e1eb65a635e688e16cae4bead54552e3bbd" + integrity sha512-QTHR9ddNnn35RTxlaEnx2gCxqFlF2SEN0SE2d17SqwyM7YOSI2GHWRYp5BiRkObTUNYPupC/3Fq2a0PpT+EKpg== + dependencies: + debug "^4.3.4" + enhanced-resolve "^5.12.0" + eslint-module-utils "^2.7.4" + fast-glob "^3.3.1" + get-tsconfig "^4.5.0" + is-core-module "^2.11.0" + is-glob "^4.0.3" + +eslint-module-utils@^2.7.4, eslint-module-utils@^2.8.0: + version "2.8.0" + resolved "https://registry.yarnpkg.com/eslint-module-utils/-/eslint-module-utils-2.8.0.tgz#e439fee65fc33f6bba630ff621efc38ec0375c49" + integrity sha512-aWajIYfsqCKRDgUfjEXNN/JlrzauMuSEy5sbd7WXbtW3EH6A6MpwEh42c7qD+MqQo9QMJ6fWLAeIJynx0g6OAw== + dependencies: + debug "^3.2.7" + +eslint-plugin-import@^2.26.0: + version "2.28.1" + resolved "https://registry.yarnpkg.com/eslint-plugin-import/-/eslint-plugin-import-2.28.1.tgz#63b8b5b3c409bfc75ebaf8fb206b07ab435482c4" + integrity sha512-9I9hFlITvOV55alzoKBI+K9q74kv0iKMeY6av5+umsNwayt59fz692daGyjR+oStBQgx6nwR9rXldDev3Clw+A== + dependencies: + array-includes "^3.1.6" + array.prototype.findlastindex "^1.2.2" + array.prototype.flat "^1.3.1" + array.prototype.flatmap "^1.3.1" + debug "^3.2.7" + doctrine "^2.1.0" + eslint-import-resolver-node "^0.3.7" + eslint-module-utils "^2.8.0" + has "^1.0.3" + is-core-module "^2.13.0" + is-glob "^4.0.3" + minimatch "^3.1.2" + object.fromentries "^2.0.6" + object.groupby "^1.0.0" + object.values "^1.1.6" + semver "^6.3.1" + tsconfig-paths "^3.14.2" + +eslint-plugin-jsx-a11y@^6.5.1: + version "6.7.1" + resolved "https://registry.yarnpkg.com/eslint-plugin-jsx-a11y/-/eslint-plugin-jsx-a11y-6.7.1.tgz#fca5e02d115f48c9a597a6894d5bcec2f7a76976" + integrity sha512-63Bog4iIethyo8smBklORknVjB0T2dwB8Mr/hIC+fBS0uyHdYYpzM/Ed+YC8VxTjlXHEWFOdmgwcDn1U2L9VCA== + dependencies: + "@babel/runtime" "^7.20.7" + aria-query "^5.1.3" + array-includes "^3.1.6" + array.prototype.flatmap "^1.3.1" + ast-types-flow "^0.0.7" + axe-core "^4.6.2" + axobject-query "^3.1.1" + damerau-levenshtein "^1.0.8" + emoji-regex "^9.2.2" + has "^1.0.3" + jsx-ast-utils "^3.3.3" + language-tags "=1.0.5" + minimatch "^3.1.2" + object.entries "^1.1.6" + object.fromentries "^2.0.6" + semver "^6.3.0" + +"eslint-plugin-react-hooks@^4.5.0 || 5.0.0-canary-7118f5dd7-20230705": + version "4.6.0" + resolved "https://registry.yarnpkg.com/eslint-plugin-react-hooks/-/eslint-plugin-react-hooks-4.6.0.tgz#4c3e697ad95b77e93f8646aaa1630c1ba607edd3" + integrity sha512-oFc7Itz9Qxh2x4gNHStv3BqJq54ExXmfC+a1NjAta66IAN87Wu0R/QArgIS9qKzX3dXKPI9H5crl9QchNMY9+g== + +eslint-plugin-react@^7.31.7: + version "7.33.2" + resolved "https://registry.yarnpkg.com/eslint-plugin-react/-/eslint-plugin-react-7.33.2.tgz#69ee09443ffc583927eafe86ffebb470ee737608" + integrity sha512-73QQMKALArI8/7xGLNI/3LylrEYrlKZSb5C9+q3OtOewTnMQi5cT+aE9E41sLCmli3I9PGGmD1yiZydyo4FEPw== + dependencies: + array-includes "^3.1.6" + array.prototype.flatmap "^1.3.1" + array.prototype.tosorted "^1.1.1" + doctrine "^2.1.0" + es-iterator-helpers "^1.0.12" + estraverse "^5.3.0" + jsx-ast-utils "^2.4.1 || ^3.0.0" + minimatch "^3.1.2" + object.entries "^1.1.6" + object.fromentries "^2.0.6" + object.hasown "^1.1.2" + object.values "^1.1.6" + prop-types "^15.8.1" + resolve "^2.0.0-next.4" + semver "^6.3.1" + string.prototype.matchall "^4.0.8" + +eslint-plugin-storybook@^0.6.13: + version "0.6.13" + resolved "https://registry.yarnpkg.com/eslint-plugin-storybook/-/eslint-plugin-storybook-0.6.13.tgz#897a9f6a9bb88c63b02f05850f30c28a9848a3f7" + integrity sha512-smd+CS0WH1jBqUEJ3znGS7DU4ayBE9z6lkQAK2yrSUv1+rq8BT/tiI5C/rKE7rmiqiAfojtNYZRhzo5HrulccQ== + dependencies: + "@storybook/csf" "^0.0.1" + "@typescript-eslint/utils" "^5.45.0" + requireindex "^1.1.0" + ts-dedent "^2.2.0" + +eslint-scope@5.1.1, eslint-scope@^5.1.1: + version "5.1.1" + resolved "https://registry.yarnpkg.com/eslint-scope/-/eslint-scope-5.1.1.tgz#e786e59a66cb92b3f6c1fb0d508aab174848f48c" + integrity sha512-2NxwbF/hZ0KpepYN0cNbo+FN6XoK7GaHlQhgx/hIZl6Va0bF45RQOOwhLIy8lQDbuCiadSLCBnH2CFYquit5bw== + dependencies: + esrecurse "^4.3.0" + estraverse "^4.1.1" + +eslint-scope@^7.2.2: + version "7.2.2" + resolved "https://registry.yarnpkg.com/eslint-scope/-/eslint-scope-7.2.2.tgz#deb4f92563390f32006894af62a22dba1c46423f" + integrity sha512-dOt21O7lTMhDM+X9mB4GX+DZrZtCUJPL/wlcTqxyrx5IvO0IYtILdtrQGQp+8n5S0gwSVmOf9NQrjMOgfQZlIg== + dependencies: + esrecurse "^4.3.0" + estraverse "^5.2.0" + +eslint-visitor-keys@^3.3.0, eslint-visitor-keys@^3.4.1, eslint-visitor-keys@^3.4.3: + version "3.4.3" + resolved "https://registry.yarnpkg.com/eslint-visitor-keys/-/eslint-visitor-keys-3.4.3.tgz#0cd72fe8550e3c2eae156a96a4dddcd1c8ac5800" + integrity sha512-wpc+LXeiyiisxPlEkUzU6svyS1frIO3Mgxj1fdy7Pm8Ygzguax2N3Fa/D/ag1WqbOprdI+uY6wMUl8/a2G+iag== + +eslint@^8.48.0: + version "8.48.0" + resolved "https://registry.yarnpkg.com/eslint/-/eslint-8.48.0.tgz#bf9998ba520063907ba7bfe4c480dc8be03c2155" + integrity sha512-sb6DLeIuRXxeM1YljSe1KEx9/YYeZFQWcV8Rq9HfigmdDEugjLEVEa1ozDjL6YDjBpQHPJxJzze+alxi4T3OLg== + dependencies: + "@eslint-community/eslint-utils" "^4.2.0" + "@eslint-community/regexpp" "^4.6.1" + "@eslint/eslintrc" "^2.1.2" + "@eslint/js" "8.48.0" + "@humanwhocodes/config-array" "^0.11.10" + "@humanwhocodes/module-importer" "^1.0.1" + "@nodelib/fs.walk" "^1.2.8" + ajv "^6.12.4" + chalk "^4.0.0" + cross-spawn "^7.0.2" + debug "^4.3.2" + doctrine "^3.0.0" + escape-string-regexp "^4.0.0" + eslint-scope "^7.2.2" + eslint-visitor-keys "^3.4.3" + espree "^9.6.1" + esquery "^1.4.2" + esutils "^2.0.2" + fast-deep-equal "^3.1.3" + file-entry-cache "^6.0.1" + find-up "^5.0.0" + glob-parent "^6.0.2" + globals "^13.19.0" + graphemer "^1.4.0" + ignore "^5.2.0" + imurmurhash "^0.1.4" + is-glob "^4.0.0" + is-path-inside "^3.0.3" + js-yaml "^4.1.0" + json-stable-stringify-without-jsonify "^1.0.1" + levn "^0.4.1" + lodash.merge "^4.6.2" + minimatch "^3.1.2" + natural-compare "^1.4.0" + optionator "^0.9.3" + strip-ansi "^6.0.1" + text-table "^0.2.0" + +espree@^9.6.0, espree@^9.6.1: + version "9.6.1" + resolved "https://registry.yarnpkg.com/espree/-/espree-9.6.1.tgz#a2a17b8e434690a5432f2f8018ce71d331a48c6f" + integrity sha512-oruZaFkjorTpF32kDSI5/75ViwGeZginGGy2NoOSg3Q9bnwlnmDm4HLnkl0RE3n+njDXR037aY1+x58Z/zFdwQ== + dependencies: + acorn "^8.9.0" + acorn-jsx "^5.3.2" + eslint-visitor-keys "^3.4.1" + +esprima@^4.0.0, esprima@^4.0.1, esprima@~4.0.0: + version "4.0.1" + resolved "https://registry.yarnpkg.com/esprima/-/esprima-4.0.1.tgz#13b04cdb3e6c5d19df91ab6987a8695619b0aa71" + integrity sha512-eGuFFw7Upda+g4p+QHvnW0RyTX/SVeJBDM/gCtMARO0cLuT2HcEKnTPvhjV6aGeqrCB/sbNop0Kszm0jsaWU4A== + +esquery@^1.4.2: + version "1.5.0" + resolved "https://registry.yarnpkg.com/esquery/-/esquery-1.5.0.tgz#6ce17738de8577694edd7361c57182ac8cb0db0b" + integrity sha512-YQLXUplAwJgCydQ78IMJywZCceoqk1oH01OERdSAJc/7U2AylwjhSCLDEtqwg811idIS/9fIU5GjG73IgjKMVg== + dependencies: + estraverse "^5.1.0" + +esrecurse@^4.3.0: + version "4.3.0" + resolved "https://registry.yarnpkg.com/esrecurse/-/esrecurse-4.3.0.tgz#7ad7964d679abb28bee72cec63758b1c5d2c9921" + integrity sha512-KmfKL3b6G+RXvP8N1vr3Tq1kL/oCFgn2NYXEtqP8/L3pKapUA4G8cFVaoF3SU323CD4XypR/ffioHmkti6/Tag== + dependencies: + estraverse "^5.2.0" + +estraverse@^4.1.1: + version "4.3.0" + resolved "https://registry.yarnpkg.com/estraverse/-/estraverse-4.3.0.tgz#398ad3f3c5a24948be7725e83d11a7de28cdbd1d" + integrity sha512-39nnKffWz8xN1BU/2c79n9nB9HDzo0niYUqx6xyqUnyoAnQyyWpOTdZEeiCch8BBu515t4wp9ZmgVfVhn9EBpw== + +estraverse@^5.1.0, estraverse@^5.2.0, estraverse@^5.3.0: + version "5.3.0" + resolved "https://registry.yarnpkg.com/estraverse/-/estraverse-5.3.0.tgz#2eea5290702f26ab8fe5370370ff86c965d21123" + integrity sha512-MMdARuVEQziNTeJD8DgMqmhwR11BRQ/cBP+pLtYdSTnf3MIO8fFeiINEbX36ZdNlfU/7A9f3gUw49B3oQsvwBA== + +estree-to-babel@^3.1.0: + version "3.2.1" + resolved "https://registry.yarnpkg.com/estree-to-babel/-/estree-to-babel-3.2.1.tgz#82e78315275c3ca74475fdc8ac1a5103c8a75bf5" + integrity sha512-YNF+mZ/Wu2FU/gvmzuWtYc8rloubL7wfXCTgouFrnjGVXPA/EeYYA7pupXWrb3Iv1cTBeSSxxJIbK23l4MRNqg== + dependencies: + "@babel/traverse" "^7.1.6" + "@babel/types" "^7.2.0" + c8 "^7.6.0" + +esutils@^2.0.2: + version "2.0.3" + resolved "https://registry.yarnpkg.com/esutils/-/esutils-2.0.3.tgz#74d2eb4de0b8da1293711910d50775b9b710ef64" + integrity sha512-kVscqXk4OCp68SZ0dkgEKVi6/8ij300KBWTJq32P/dYeWTSwK41WyTxalN1eRmA5Z9UU/LX9D7FWSmV9SAYx6g== + +etag@~1.8.1: + version "1.8.1" + resolved "https://registry.yarnpkg.com/etag/-/etag-1.8.1.tgz#41ae2eeb65efa62268aebfea83ac7d79299b0887" + integrity sha512-aIL5Fx7mawVa300al2BnEE4iNvo1qETxLrPI/o05L7z6go7fCw1J6EQmbK4FmJ2AS7kgVF/KEZWufBfdClMcPg== + +event-target-shim@^5.0.0: + version "5.0.1" + resolved "https://registry.yarnpkg.com/event-target-shim/-/event-target-shim-5.0.1.tgz#5d4d3ebdf9583d63a5333ce2deb7480ab2b05789" + integrity sha512-i/2XbnSz/uxRCU6+NdVJgKWDTM427+MqYbkQzD321DuCQJUqOuJKIA0IM2+W2xtYHdKOmZ4dR6fExsd4SXL+WQ== + +eventemitter3@^4.0.1: + version "4.0.7" + resolved "https://registry.yarnpkg.com/eventemitter3/-/eventemitter3-4.0.7.tgz#2de9b68f6528d5644ef5c59526a1b4a07306169f" + integrity sha512-8guHBZCwKnFhYdHr2ysuRWErTwhoN2X8XELRlrRwpmfeY2jjuUN4taQMsULKUVo1K4DvZl+0pgfyoysHxvmvEw== + +events@^3.2.0, events@^3.3.0: + version "3.3.0" + resolved "https://registry.yarnpkg.com/events/-/events-3.3.0.tgz#31a95ad0a924e2d2c419a813aeb2c4e878ea7400" + integrity sha512-mQw+2fkQbALzQ7V0MY0IqdnXNOeTtP4r0lN9z7AAawCXgqea7bDii20AYrIBrFd/Hx0M2Ocz6S111CaFkUcb0Q== + +evp_bytestokey@^1.0.0, evp_bytestokey@^1.0.3: + version "1.0.3" + resolved "https://registry.yarnpkg.com/evp_bytestokey/-/evp_bytestokey-1.0.3.tgz#7fcbdb198dc71959432efe13842684e0525acb02" + integrity sha512-/f2Go4TognH/KvCISP7OUsHn85hT9nUkxxA9BEWxFn+Oj9o8ZNLm/40hdlgSLyuOimsrTKLUMEorQexp/aPQeA== + dependencies: + md5.js "^1.3.4" + safe-buffer "^5.1.1" + +execa@^5.0.0, execa@^5.1.1: + version "5.1.1" + resolved "https://registry.yarnpkg.com/execa/-/execa-5.1.1.tgz#f80ad9cbf4298f7bd1d4c9555c21e93741c411dd" + integrity sha512-8uSpZZocAZRBAPIEINJj3Lo9HyGitllczc27Eh5YYojjMFMn8yHMDMaUHE2Jqfq05D/wucwI4JGURyXt1vchyg== + dependencies: + cross-spawn "^7.0.3" + get-stream "^6.0.0" + human-signals "^2.1.0" + is-stream "^2.0.0" + merge-stream "^2.0.0" + npm-run-path "^4.0.1" + onetime "^5.1.2" + signal-exit "^3.0.3" + strip-final-newline "^2.0.0" + +express@^4.17.3: + version "4.18.2" + resolved "https://registry.yarnpkg.com/express/-/express-4.18.2.tgz#3fabe08296e930c796c19e3c516979386ba9fd59" + integrity sha512-5/PsL6iGPdfQ/lKM1UuielYgv3BUoJfz1aUwU9vHZ+J7gyvwdQXFEBIEIaxeGf0GIcreATNyBExtalisDbuMqQ== + dependencies: + accepts "~1.3.8" + array-flatten "1.1.1" + body-parser "1.20.1" + content-disposition "0.5.4" + content-type "~1.0.4" + cookie "0.5.0" + cookie-signature "1.0.6" + debug "2.6.9" + depd "2.0.0" + encodeurl "~1.0.2" + escape-html "~1.0.3" + etag "~1.8.1" + finalhandler "1.2.0" + fresh "0.5.2" + http-errors "2.0.0" + merge-descriptors "1.0.1" + methods "~1.1.2" + on-finished "2.4.1" + parseurl "~1.3.3" + path-to-regexp "0.1.7" + proxy-addr "~2.0.7" + qs "6.11.0" + range-parser "~1.2.1" + safe-buffer "5.2.1" + send "0.18.0" + serve-static "1.15.0" + setprototypeof "1.2.0" + statuses "2.0.1" + type-is "~1.6.18" + utils-merge "1.0.1" + vary "~1.1.2" + +extend@^3.0.0: + version "3.0.2" + resolved "https://registry.yarnpkg.com/extend/-/extend-3.0.2.tgz#f8b1136b4071fbd8eb140aff858b1019ec2915fa" + integrity sha512-fjquC59cD7CyW6urNXK0FBufkZcoiGG80wTuPujX590cB5Ttln20E2UB4S/WARVqhXffZl2LNgS+gQdPIIim/g== + +extract-zip@^1.6.6: + version "1.7.0" + resolved "https://registry.yarnpkg.com/extract-zip/-/extract-zip-1.7.0.tgz#556cc3ae9df7f452c493a0cfb51cc30277940927" + integrity sha512-xoh5G1W/PB0/27lXgMQyIhP5DSY/LhoCsOyZgb+6iMmRtCwVBo55uKaMoEYrDCKQhWvqEip5ZPKAc6eFNyf/MA== + dependencies: + concat-stream "^1.6.2" + debug "^2.6.9" + mkdirp "^0.5.4" + yauzl "^2.10.0" + +fast-deep-equal@^3.1.1, fast-deep-equal@^3.1.3: + version "3.1.3" + resolved "https://registry.yarnpkg.com/fast-deep-equal/-/fast-deep-equal-3.1.3.tgz#3a7d56b559d6cbc3eb512325244e619a65c6c525" + integrity sha512-f3qQ9oQy9j2AhBe/H9VC91wLmKBCCU/gDOnKNAYG5hswO7BLKj09Hc5HYNz9cGI++xlpDCIgDaitVs03ATR84Q== + +fast-equals@^5.0.0: + version "5.0.1" + resolved "https://registry.yarnpkg.com/fast-equals/-/fast-equals-5.0.1.tgz#a4eefe3c5d1c0d021aeed0bc10ba5e0c12ee405d" + integrity sha512-WF1Wi8PwwSY7/6Kx0vKXtw8RwuSGoM1bvDaJbu7MxDlR1vovZjIAKrnzyrThgAjm6JDTu0fVgWXDlMGspodfoQ== + +fast-glob@^3.2.11, fast-glob@^3.2.12, fast-glob@^3.2.9, fast-glob@^3.3.0, fast-glob@^3.3.1: + version "3.3.1" + resolved "https://registry.yarnpkg.com/fast-glob/-/fast-glob-3.3.1.tgz#784b4e897340f3dbbef17413b3f11acf03c874c4" + integrity sha512-kNFPyjhh5cKjrUltxs+wFx+ZkbRaxxmZ+X0ZU31SOsxCEtP9VPgtq2teZw1DebupL5GmDaNQ6yKMMVcM41iqDg== + dependencies: + "@nodelib/fs.stat" "^2.0.2" + "@nodelib/fs.walk" "^1.2.3" + glob-parent "^5.1.2" + merge2 "^1.3.0" + micromatch "^4.0.4" + +fast-json-parse@^1.0.3: + version "1.0.3" + resolved "https://registry.yarnpkg.com/fast-json-parse/-/fast-json-parse-1.0.3.tgz#43e5c61ee4efa9265633046b770fb682a7577c4d" + integrity sha512-FRWsaZRWEJ1ESVNbDWmsAlqDk96gPQezzLghafp5J4GUKjbCz3OkAHuZs5TuPEtkbVQERysLp9xv6c24fBm8Aw== + +fast-json-stable-stringify@^2.0.0, fast-json-stable-stringify@^2.1.0: + version "2.1.0" + resolved "https://registry.yarnpkg.com/fast-json-stable-stringify/-/fast-json-stable-stringify-2.1.0.tgz#874bf69c6f404c2b5d99c481341399fd55892633" + integrity sha512-lhd/wF+Lk98HZoTCtlVraHtfh5XYijIjalXck7saUtuanSDyLMxnHhSXEDJqHxD7msR8D0uCmqlkwjCV8xvwHw== + +fast-levenshtein@^2.0.6: + version "2.0.6" + resolved "https://registry.yarnpkg.com/fast-levenshtein/-/fast-levenshtein-2.0.6.tgz#3d8a5c66883a16a30ca8643e851f19baa7797917" + integrity sha512-DCXu6Ifhqcks7TZKY3Hxp3y6qphY5SJZmrWMDrKcERSOXWQdMhU9Ig/PYrzyw/ul9jOIyh0N4M0tbC5hodg8dw== + +fastq@^1.6.0: + version "1.15.0" + resolved "https://registry.yarnpkg.com/fastq/-/fastq-1.15.0.tgz#d04d07c6a2a68fe4599fea8d2e103a937fae6b3a" + integrity sha512-wBrocU2LCXXa+lWBt8RoIRD89Fi8OdABODa/kEnyeyjS5aZO5/GNvI5sEINADqP/h8M29UHTHUb53sUu5Ihqdw== + dependencies: + reusify "^1.0.4" + +fb-watchman@^2.0.0: + version "2.0.2" + resolved "https://registry.yarnpkg.com/fb-watchman/-/fb-watchman-2.0.2.tgz#e9524ee6b5c77e9e5001af0f85f3adbb8623255c" + integrity sha512-p5161BqbuCaSnB8jIbzQHOlpgsPmK5rJVDfDKO91Axs5NC1uu3HRQm6wt9cd9/+GtQQIO53JdGXXoyDpTAsgYA== + dependencies: + bser "2.1.1" + +fd-slicer@~1.1.0: + version "1.1.0" + resolved "https://registry.yarnpkg.com/fd-slicer/-/fd-slicer-1.1.0.tgz#25c7c89cb1f9077f8891bbe61d8f390eae256f1e" + integrity sha512-cE1qsB/VwyQozZ+q1dGxR8LBYNZeofhEdUNGSMbQD3Gw2lAzX9Zb3uIU6Ebc/Fmyjo9AWWfnn0AUCHqtevs/8g== + dependencies: + pend "~1.2.0" + +fetch-retry@^5.0.2: + version "5.0.6" + resolved "https://registry.yarnpkg.com/fetch-retry/-/fetch-retry-5.0.6.tgz#17d0bc90423405b7a88b74355bf364acd2a7fa56" + integrity sha512-3yurQZ2hD9VISAhJJP9bpYFNQrHHBXE2JxxjY5aLEcDi46RmAzJE2OC9FAde0yis5ElW0jTTzs0zfg/Cca4XqQ== + +file-entry-cache@^6.0.1: + version "6.0.1" + resolved "https://registry.yarnpkg.com/file-entry-cache/-/file-entry-cache-6.0.1.tgz#211b2dd9659cb0394b073e7323ac3c933d522027" + integrity sha512-7Gps/XWymbLk2QLYK4NzpMOrYjMhdIxXuIvy2QBsLE6ljuodKvdkWs/cpyJJ3CVIVpH0Oi1Hvg1ovbMzLdFBBg== + dependencies: + flat-cache "^3.0.4" + +file-system-cache@2.3.0: + version "2.3.0" + resolved "https://registry.yarnpkg.com/file-system-cache/-/file-system-cache-2.3.0.tgz#201feaf4c8cd97b9d0d608e96861bb6005f46fe6" + integrity sha512-l4DMNdsIPsVnKrgEXbJwDJsA5mB8rGwHYERMgqQx/xAUtChPJMre1bXBzDEqqVbWv9AIbFezXMxeEkZDSrXUOQ== + dependencies: + fs-extra "11.1.1" + ramda "0.29.0" + +filelist@^1.0.4: + version "1.0.4" + resolved "https://registry.yarnpkg.com/filelist/-/filelist-1.0.4.tgz#f78978a1e944775ff9e62e744424f215e58352b5" + integrity sha512-w1cEuf3S+DrLCQL7ET6kz+gmlJdbq9J7yXCSjK/OZCPA+qEN1WyF4ZAf0YYJa4/shHJra2t/d/r8SV4Ji+x+8Q== + dependencies: + minimatch "^5.0.1" + +filename-reserved-regex@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/filename-reserved-regex/-/filename-reserved-regex-2.0.0.tgz#abf73dfab735d045440abfea2d91f389ebbfa229" + integrity sha512-lc1bnsSr4L4Bdif8Xb/qrtokGbq5zlsms/CYH8PP+WtCkGNF65DPiQY8vG3SakEdRn8Dlnm+gW/qWKKjS5sZzQ== + +filenamify@^4.3.0: + version "4.3.0" + resolved "https://registry.yarnpkg.com/filenamify/-/filenamify-4.3.0.tgz#62391cb58f02b09971c9d4f9d63b3cf9aba03106" + integrity sha512-hcFKyUG57yWGAzu1CMt/dPzYZuv+jAJUT85bL8mrXvNe6hWj6yEHEc4EdcgiA6Z3oi1/9wXJdZPXF2dZNgwgOg== + dependencies: + filename-reserved-regex "^2.0.0" + strip-outer "^1.0.1" + trim-repeated "^1.0.0" + +fill-range@^7.0.1: + version "7.0.1" + resolved "https://registry.yarnpkg.com/fill-range/-/fill-range-7.0.1.tgz#1919a6a7c75fe38b2c7c77e5198535da9acdda40" + integrity sha512-qOo9F+dMUmC2Lcb4BbVvnKJxTPjCm+RRpe4gDuGrzkL7mEVl/djYSu2OdQ2Pa302N4oqkSg9ir6jaLWJ2USVpQ== + dependencies: + to-regex-range "^5.0.1" + +filter-obj@^2.0.2: + version "2.0.2" + resolved "https://registry.yarnpkg.com/filter-obj/-/filter-obj-2.0.2.tgz#fff662368e505d69826abb113f0f6a98f56e9d5f" + integrity sha512-lO3ttPjHZRfjMcxWKb1j1eDhTFsu4meeR3lnMcnBFhk6RuLhvEiuALu2TlfL310ph4lCYYwgF/ElIjdP739tdg== + +finalhandler@1.2.0: + version "1.2.0" + resolved "https://registry.yarnpkg.com/finalhandler/-/finalhandler-1.2.0.tgz#7d23fe5731b207b4640e4fcd00aec1f9207a7b32" + integrity sha512-5uXcUVftlQMFnWC9qu/svkWv3GTd2PfUhK/3PLkYNAe7FbqJMt3515HaxE6eRL74GdsriiwujiawdaB1BpEISg== + dependencies: + debug "2.6.9" + encodeurl "~1.0.2" + escape-html "~1.0.3" + on-finished "2.4.1" + parseurl "~1.3.3" + statuses "2.0.1" + unpipe "~1.0.0" + +find-cache-dir@^2.0.0: + version "2.1.0" + resolved "https://registry.yarnpkg.com/find-cache-dir/-/find-cache-dir-2.1.0.tgz#8d0f94cd13fe43c6c7c261a0d86115ca918c05f7" + integrity sha512-Tq6PixE0w/VMFfCgbONnkiQIVol/JJL7nRMi20fqzA4NRs9AfeqMGeRdPi3wIhYkxjeBaWh2rxwapn5Tu3IqOQ== + dependencies: + commondir "^1.0.1" + make-dir "^2.0.0" + pkg-dir "^3.0.0" + +find-cache-dir@^3.0.0, find-cache-dir@^3.3.1: + version "3.3.2" + resolved "https://registry.yarnpkg.com/find-cache-dir/-/find-cache-dir-3.3.2.tgz#b30c5b6eff0730731aea9bbd9dbecbd80256d64b" + integrity sha512-wXZV5emFEjrridIgED11OoUKLxiYjAcqot/NJdAkOhlJ+vGzwhOAfcG5OX1jP+S0PcjEn8bdMJv+g2jwQ3Onig== + dependencies: + commondir "^1.0.1" + make-dir "^3.0.2" + pkg-dir "^4.1.0" + +find-cache-dir@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/find-cache-dir/-/find-cache-dir-4.0.0.tgz#a30ee0448f81a3990708f6453633c733e2f6eec2" + integrity sha512-9ZonPT4ZAK4a+1pUPVPZJapbi7O5qbbJPdYw/NOQWZZbVLdDTYM3A4R9z/DpAM08IDaFGsvPgiGZ82WEwUDWjg== + dependencies: + common-path-prefix "^3.0.0" + pkg-dir "^7.0.0" + +find-up@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/find-up/-/find-up-3.0.0.tgz#49169f1d7993430646da61ecc5ae355c21c97b73" + integrity sha512-1yD6RmLI1XBfxugvORwlck6f75tYL+iR0jqwsOrOxMZyGYqUuDhJ0l4AXdO1iX/FTs9cBAMEk1gWSEx1kSbylg== + dependencies: + locate-path "^3.0.0" + +find-up@^4.0.0, find-up@^4.1.0: + version "4.1.0" + resolved "https://registry.yarnpkg.com/find-up/-/find-up-4.1.0.tgz#97afe7d6cdc0bc5928584b7c8d7b16e8a9aa5d19" + integrity sha512-PpOwAdQ/YlXQ2vj8a3h8IipDuYRi3wceVQQGYWxNINccq40Anw7BlsEXCMbt1Zt+OLA6Fq9suIpIWD0OsnISlw== + dependencies: + locate-path "^5.0.0" + path-exists "^4.0.0" + +find-up@^5.0.0: + version "5.0.0" + resolved "https://registry.yarnpkg.com/find-up/-/find-up-5.0.0.tgz#4c92819ecb7083561e4f4a240a86be5198f536fc" + integrity sha512-78/PXT1wlLLDgTzDs7sjq9hzz0vXD+zn+7wypEe4fXQxCmdmqfGsEPQxmiCSQI3ajFV91bVSsvNtrJRiW6nGng== + dependencies: + locate-path "^6.0.0" + path-exists "^4.0.0" + +find-up@^6.3.0: + version "6.3.0" + resolved "https://registry.yarnpkg.com/find-up/-/find-up-6.3.0.tgz#2abab3d3280b2dc7ac10199ef324c4e002c8c790" + integrity sha512-v2ZsoEuVHYy8ZIlYqwPe/39Cy+cFDzp4dXPaxNvkEuouymu+2Jbz0PxpKarJHYJTmv2HWT3O382qY8l4jMWthw== + dependencies: + locate-path "^7.1.0" + path-exists "^5.0.0" + +flat-cache@^3.0.4: + version "3.1.0" + resolved "https://registry.yarnpkg.com/flat-cache/-/flat-cache-3.1.0.tgz#0e54ab4a1a60fe87e2946b6b00657f1c99e1af3f" + integrity sha512-OHx4Qwrrt0E4jEIcI5/Xb+f+QmJYNj2rrK8wiIdQOIrB9WrrJL8cjZvXdXuBTkkEwEqLycb5BeZDV1o2i9bTew== + dependencies: + flatted "^3.2.7" + keyv "^4.5.3" + rimraf "^3.0.2" + +flatted@^3.2.7: + version "3.2.7" + resolved "https://registry.yarnpkg.com/flatted/-/flatted-3.2.7.tgz#609f39207cb614b89d0765b477cb2d437fbf9787" + integrity sha512-5nqDSxl8nn5BSNxyR3n4I6eDmbolI6WT+QqR547RwxQapgjQBmtktdP+HTBb/a/zLsbzERTONyUB5pefh5TtjQ== + +flow-parser@0.*: + version "0.215.1" + resolved "https://registry.yarnpkg.com/flow-parser/-/flow-parser-0.215.1.tgz#a14007f404db46ac829bb6db3a22a7956d9e298f" + integrity sha512-qq3rdRToqwesrddyXf+Ml8Tuf7TdoJS+EMbJgC6fHAVoBCXjb4mHelNd3J+jD8ts0bSHX81FG3LN7Qn/dcl6pA== + +for-each@^0.3.3: + version "0.3.3" + resolved "https://registry.yarnpkg.com/for-each/-/for-each-0.3.3.tgz#69b447e88a0a5d32c3e7084f3f1710034b21376e" + integrity sha512-jqYfLp7mo9vIyQf8ykW2v7A+2N4QjeCeI5+Dz9XraiO1ign81wjiH7Fb9vSOWvQfNtmSa4H2RoQTrrXivdUZmw== + dependencies: + is-callable "^1.1.3" + +foreground-child@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/foreground-child/-/foreground-child-2.0.0.tgz#71b32800c9f15aa8f2f83f4a6bd9bff35d861a53" + integrity sha512-dCIq9FpEcyQyXKCkyzmlPTFNgrCzPudOe+mhvJU5zAtlBnGVy2yKxtfsxK2tQBThwq225jcvBjpw1Gr40uzZCA== + dependencies: + cross-spawn "^7.0.0" + signal-exit "^3.0.2" + +foreground-child@^3.1.0: + version "3.1.1" + resolved "https://registry.yarnpkg.com/foreground-child/-/foreground-child-3.1.1.tgz#1d173e776d75d2772fed08efe4a0de1ea1b12d0d" + integrity sha512-TMKDUnIte6bfb5nWv7V/caI169OHgvwjb7V4WkeUvbQQdjr5rWKqHFiKWb/fcOwB+CzBT+qbWjvj+DVwRskpIg== + dependencies: + cross-spawn "^7.0.0" + signal-exit "^4.0.1" + +fork-ts-checker-webpack-plugin@^8.0.0: + version "8.0.0" + resolved "https://registry.yarnpkg.com/fork-ts-checker-webpack-plugin/-/fork-ts-checker-webpack-plugin-8.0.0.tgz#dae45dfe7298aa5d553e2580096ced79b6179504" + integrity sha512-mX3qW3idpueT2klaQXBzrIM/pHw+T0B/V9KHEvNrqijTq9NFnMZU6oreVxDYcf33P8a5cW+67PjodNHthGnNVg== + dependencies: + "@babel/code-frame" "^7.16.7" + chalk "^4.1.2" + chokidar "^3.5.3" + cosmiconfig "^7.0.1" + deepmerge "^4.2.2" + fs-extra "^10.0.0" + memfs "^3.4.1" + minimatch "^3.0.4" + node-abort-controller "^3.0.1" + schema-utils "^3.1.1" + semver "^7.3.5" + tapable "^2.2.1" + +form-data@^3.0.0: + version "3.0.1" + resolved "https://registry.yarnpkg.com/form-data/-/form-data-3.0.1.tgz#ebd53791b78356a99af9a300d4282c4d5eb9755f" + integrity sha512-RHkBKtLWUVwd7SqRIvCZMEvAMoGUp0XU+seQiZejj0COz3RI3hWP4sCv3gZWWLjJTd7rGwcsF5eKZGii0r/hbg== + dependencies: + asynckit "^0.4.0" + combined-stream "^1.0.8" + mime-types "^2.1.12" + +forwarded@0.2.0: + version "0.2.0" + resolved "https://registry.yarnpkg.com/forwarded/-/forwarded-0.2.0.tgz#2269936428aad4c15c7ebe9779a84bf0b2a81811" + integrity sha512-buRG0fpBtRHSTCOASe6hD258tEubFoRLb4ZNA6NxMVHNw2gOcwHo9wyablzMzOA5z9xA9L1KNjk/Nt6MT9aYow== + +fraction.js@^4.2.0: + version "4.3.6" + resolved "https://registry.yarnpkg.com/fraction.js/-/fraction.js-4.3.6.tgz#e9e3acec6c9a28cf7bc36cbe35eea4ceb2c5c92d" + integrity sha512-n2aZ9tNfYDwaHhvFTkhFErqOMIb8uyzSQ+vGJBjZyanAKZVbGUQ1sngfk9FdkBw7G26O7AgNjLcecLffD1c7eg== + +fresh@0.5.2: + version "0.5.2" + resolved "https://registry.yarnpkg.com/fresh/-/fresh-0.5.2.tgz#3d8cadd90d976569fa835ab1f8e4b23a105605a7" + integrity sha512-zJ2mQYM18rEFOudeV4GShTGIQ7RbzA7ozbU9I/XBpm7kqgMywgmylMwXHxZJmkVoYkna9d2pVXVXPdYTP9ej8Q== + +fs-constants@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/fs-constants/-/fs-constants-1.0.0.tgz#6be0de9be998ce16af8afc24497b9ee9b7ccd9ad" + integrity sha512-y6OAwoSIf7FyjMIv94u+b5rdheZEjzR63GTyZJm5qh4Bi+2YgwLCcI/fPFZkL5PSixOt6ZNKm+w+Hfp/Bciwow== + +fs-extra@11.1.1, fs-extra@^11.1.0, fs-extra@^11.1.1: + version "11.1.1" + resolved "https://registry.yarnpkg.com/fs-extra/-/fs-extra-11.1.1.tgz#da69f7c39f3b002378b0954bb6ae7efdc0876e2d" + integrity sha512-MGIE4HOvQCeUCzmlHs0vXpih4ysz4wg9qiSAu6cd42lVwPbTM1TjV7RusoyQqMmk/95gdQZX72u+YW+c3eEpFQ== + dependencies: + graceful-fs "^4.2.0" + jsonfile "^6.0.1" + universalify "^2.0.0" + +fs-extra@^10.0.0: + version "10.1.0" + resolved "https://registry.yarnpkg.com/fs-extra/-/fs-extra-10.1.0.tgz#02873cfbc4084dde127eaa5f9905eef2325d1abf" + integrity sha512-oRXApq54ETRj4eMiFzGnHWGy+zo5raudjuxN0b8H7s/RU2oW0Wvsx9O0ACRN/kRq9E8Vu/ReskGB5o3ji+FzHQ== + dependencies: + graceful-fs "^4.2.0" + jsonfile "^6.0.1" + universalify "^2.0.0" + +fs-minipass@^2.0.0: + version "2.1.0" + resolved "https://registry.yarnpkg.com/fs-minipass/-/fs-minipass-2.1.0.tgz#7f5036fdbf12c63c169190cbe4199c852271f9fb" + integrity sha512-V/JgOLFCS+R6Vcq0slCuaeWEdNC3ouDlJMNIsacH2VtALiu9mV4LPrHc5cDl8k5aw6J8jwgWWpiTo5RYhmIzvg== + dependencies: + minipass "^3.0.0" + +fs-monkey@^1.0.4: + version "1.0.4" + resolved "https://registry.yarnpkg.com/fs-monkey/-/fs-monkey-1.0.4.tgz#ee8c1b53d3fe8bb7e5d2c5c5dfc0168afdd2f747" + integrity sha512-INM/fWAxMICjttnD0DX1rBvinKskj5G1w+oy/pnm9u/tSlnBrzFonJMcalKJ30P8RRsPzKcCG7Q8l0jx5Fh9YQ== + +fs-readdir-recursive@^1.1.0: + version "1.1.0" + resolved "https://registry.yarnpkg.com/fs-readdir-recursive/-/fs-readdir-recursive-1.1.0.tgz#e32fc030a2ccee44a6b5371308da54be0b397d27" + integrity sha512-GNanXlVr2pf02+sPN40XN8HG+ePaNcvM0q5mZBd668Obwb0yD5GiUbZOFgwn8kGMY6I3mdyDJzieUy3PTYyTRA== + +fs.realpath@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/fs.realpath/-/fs.realpath-1.0.0.tgz#1504ad2523158caa40db4a2787cb01411994ea4f" + integrity sha512-OO0pH2lK6a0hZnAdau5ItzHPI6pUlvI7jMVnxUQRtw4owF2wk8lOSabtGDCTP4Ggrg2MbGnWO9X8K1t4+fGMDw== + +fsevents@^2.3.2, fsevents@~2.3.2: + version "2.3.3" + resolved "https://registry.yarnpkg.com/fsevents/-/fsevents-2.3.3.tgz#cac6407785d03675a2a5e1a5305c697b347d90d6" + integrity sha512-5xoDfX+fL7faATnagmWPpbFtwh/R77WmMMqqHGS65C3vvB0YHrgF+B1YmZ3441tMj5n63k0212XNoJwzlhffQw== + +function-bind@^1.1.1: + version "1.1.1" + resolved "https://registry.yarnpkg.com/function-bind/-/function-bind-1.1.1.tgz#a56899d3ea3c9bab874bb9773b7c5ede92f4895d" + integrity sha512-yIovAzMX49sF8Yl58fSCWJ5svSLuaibPxXQJFLmBObTuCr0Mf1KiPopGM9NiFjiYBCbfaa2Fh6breQ6ANVTI0A== + +function.prototype.name@^1.1.5: + version "1.1.6" + resolved "https://registry.yarnpkg.com/function.prototype.name/-/function.prototype.name-1.1.6.tgz#cdf315b7d90ee77a4c6ee216c3c3362da07533fd" + integrity sha512-Z5kx79swU5P27WEayXM1tBi5Ze/lbIyiNgU3qyXUOf9b2rgXYyF9Dy9Cx+IQv/Lc8WCG6L82zwUPpSS9hGehIg== + dependencies: + call-bind "^1.0.2" + define-properties "^1.2.0" + es-abstract "^1.22.1" + functions-have-names "^1.2.3" + +functions-have-names@^1.2.3: + version "1.2.3" + resolved "https://registry.yarnpkg.com/functions-have-names/-/functions-have-names-1.2.3.tgz#0404fe4ee2ba2f607f0e0ec3c80bae994133b834" + integrity sha512-xckBUXyTIqT97tq2x2AMb+g163b5JFysYk0x4qxNFwbfQkmNZoiRHb6sPzI9/QV33WeuvVYBUIiD4NzNIyqaRQ== + +gensync@^1.0.0-beta.2: + version "1.0.0-beta.2" + resolved "https://registry.yarnpkg.com/gensync/-/gensync-1.0.0-beta.2.tgz#32a6ee76c3d7f52d46b2b1ae5d93fea8580a25e0" + integrity sha512-3hN7NaskYvMDLQY55gnW3NQ+mesEAepTqlg+VEbj7zzqEMBVNhzcGYYeqFo/TlYz6eQiFcp1HcsCZO+nGgS8zg== + +get-caller-file@^2.0.5: + version "2.0.5" + resolved "https://registry.yarnpkg.com/get-caller-file/-/get-caller-file-2.0.5.tgz#4f94412a82db32f36e3b0b9741f8a97feb031f7e" + integrity sha512-DyFP3BM/3YHTQOCUL/w0OZHR0lpKeGrxotcHWcqNEdnltqFwXVfhEBQ94eIo34AfQpo0rGki4cyIiftY06h2Fg== + +get-intrinsic@^1.0.2, get-intrinsic@^1.1.1, get-intrinsic@^1.1.3, get-intrinsic@^1.2.0, get-intrinsic@^1.2.1: + version "1.2.1" + resolved "https://registry.yarnpkg.com/get-intrinsic/-/get-intrinsic-1.2.1.tgz#d295644fed4505fc9cde952c37ee12b477a83d82" + integrity sha512-2DcsyfABl+gVHEfCOaTrWgyt+tb6MSEGmKq+kI5HwLbIYgjgmMcV8KQ41uaKz1xxUcn9tJtgFbQUEVcEbd0FYw== + dependencies: + function-bind "^1.1.1" + has "^1.0.3" + has-proto "^1.0.1" + has-symbols "^1.0.3" + +get-nonce@^1.0.0: + version "1.0.1" + resolved "https://registry.yarnpkg.com/get-nonce/-/get-nonce-1.0.1.tgz#fdf3f0278073820d2ce9426c18f07481b1e0cdf3" + integrity sha512-FJhYRoDaiatfEkUK8HKlicmu/3SGFD51q3itKDGoSTysQJBnfOcxU5GxnhE1E6soB76MbT0MBtnKJuXyAx+96Q== + +get-npm-tarball-url@^2.0.3: + version "2.0.3" + resolved "https://registry.yarnpkg.com/get-npm-tarball-url/-/get-npm-tarball-url-2.0.3.tgz#67dff908d699e9e2182530ae6e939a93e5f8dfdb" + integrity sha512-R/PW6RqyaBQNWYaSyfrh54/qtcnOp22FHCCiRhSSZj0FP3KQWCsxxt0DzIdVTbwTqe9CtQfvl/FPD4UIPt4pqw== + +get-package-type@^0.1.0: + version "0.1.0" + resolved "https://registry.yarnpkg.com/get-package-type/-/get-package-type-0.1.0.tgz#8de2d803cff44df3bc6c456e6668b36c3926e11a" + integrity sha512-pjzuKtY64GYfWizNAJ0fr9VqttZkNiK2iS430LtIHzjBEr6bX8Am2zm4sW4Ro5wjWW5cAlRL1qAMTcXbjNAO2Q== + +get-port@^5.1.1: + version "5.1.1" + resolved "https://registry.yarnpkg.com/get-port/-/get-port-5.1.1.tgz#0469ed07563479de6efb986baf053dcd7d4e3193" + integrity sha512-g/Q1aTSDOxFpchXC4i8ZWvxA1lnPqx/JHqcpIw0/LX9T8x/GBbi6YnlN5nhaKIFkT8oFsscUKgDJYxfwfS6QsQ== + +get-stream@^6.0.0: + version "6.0.1" + resolved "https://registry.yarnpkg.com/get-stream/-/get-stream-6.0.1.tgz#a262d8eef67aced57c2852ad6167526a43cbf7b7" + integrity sha512-ts6Wi+2j3jQjqi70w5AlN8DFnkSwC+MqmxEzdEALB2qXZYV3X/b1CTfgPLGJNMeAWxdPfU8FO1ms3NUfaHCPYg== + +get-symbol-description@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/get-symbol-description/-/get-symbol-description-1.0.0.tgz#7fdb81c900101fbd564dd5f1a30af5aadc1e58d6" + integrity sha512-2EmdH1YvIQiZpltCNgkuiUnyukzxM/R6NDJX31Ke3BG1Nq5b0S2PhX59UKi9vZpPDQVdqn+1IcaAwnzTT5vCjw== + dependencies: + call-bind "^1.0.2" + get-intrinsic "^1.1.1" + +get-tsconfig@^4.5.0: + version "4.7.0" + resolved "https://registry.yarnpkg.com/get-tsconfig/-/get-tsconfig-4.7.0.tgz#06ce112a1463e93196aa90320c35df5039147e34" + integrity sha512-pmjiZ7xtB8URYm74PlGJozDNyhvsVLUcpBa8DZBG3bWHwaHa9bPiRpiSfovw+fjhwONSCWKRyk+JQHEGZmMrzw== + dependencies: + resolve-pkg-maps "^1.0.0" + +gh-pages@^6.0.0: + version "6.0.0" + resolved "https://registry.yarnpkg.com/gh-pages/-/gh-pages-6.0.0.tgz#3bb46ea13dc7cee306662db0d3f02bf05635cdc1" + integrity sha512-FXZWJRsvP/fK2HJGY+Di6FRNHvqFF6gOIELaopDjXXgjeOYSNURcuYwEO/6bwuq6koP5Lnkvnr5GViXzuOB89g== + dependencies: + async "^3.2.4" + commander "^11.0.0" + email-addresses "^5.0.0" + filenamify "^4.3.0" + find-cache-dir "^3.3.1" + fs-extra "^11.1.1" + globby "^6.1.0" + +giget@^1.0.0: + version "1.1.2" + resolved "https://registry.yarnpkg.com/giget/-/giget-1.1.2.tgz#f99a49cb0ff85479c8c3612cdc7ca27f2066e818" + integrity sha512-HsLoS07HiQ5oqvObOI+Qb2tyZH4Gj5nYGfF9qQcZNrPw+uEFhdXtgJr01aO2pWadGHucajYDLxxbtQkm97ON2A== + dependencies: + colorette "^2.0.19" + defu "^6.1.2" + https-proxy-agent "^5.0.1" + mri "^1.2.0" + node-fetch-native "^1.0.2" + pathe "^1.1.0" + tar "^6.1.13" + +github-slugger@^1.0.0: + version "1.5.0" + resolved "https://registry.yarnpkg.com/github-slugger/-/github-slugger-1.5.0.tgz#17891bbc73232051474d68bd867a34625c955f7d" + integrity sha512-wIh+gKBI9Nshz2o46B0B3f5k/W+WI9ZAv6y5Dn5WJ5SK1t0TnDimB4WE5rmTD05ZAIn8HALCZVmCsvj0w0v0lw== + +glob-parent@^5.1.2, glob-parent@~5.1.2: + version "5.1.2" + resolved "https://registry.yarnpkg.com/glob-parent/-/glob-parent-5.1.2.tgz#869832c58034fe68a4093c17dc15e8340d8401c4" + integrity sha512-AOIgSQCepiJYwP3ARnGx+5VnTu2HBYdzbGP45eLw1vr3zB3vZLeyed1sC9hnbcOc9/SrMyM5RPQrkGz4aS9Zow== + dependencies: + is-glob "^4.0.1" + +glob-parent@^6.0.1, glob-parent@^6.0.2: + version "6.0.2" + resolved "https://registry.yarnpkg.com/glob-parent/-/glob-parent-6.0.2.tgz#6d237d99083950c79290f24c7642a3de9a28f9e3" + integrity sha512-XxwI8EOhVQgWp6iDL+3b0r86f4d6AX6zSU55HfB4ydCEuXLXc5FcYeOu+nnGftS4TEju/11rt4KJPTMgbfmv4A== + dependencies: + is-glob "^4.0.3" + +glob-to-regexp@^0.4.1: + version "0.4.1" + resolved "https://registry.yarnpkg.com/glob-to-regexp/-/glob-to-regexp-0.4.1.tgz#c75297087c851b9a578bd217dd59a92f59fe546e" + integrity sha512-lkX1HJXwyMcprw/5YUZc2s7DrpAiHB21/V+E1rHUrVNokkvB6bqMzT0VfV6/86ZNabt1k14YOIaT7nDvOX3Iiw== + +glob@7.1.6: + version "7.1.6" + resolved "https://registry.yarnpkg.com/glob/-/glob-7.1.6.tgz#141f33b81a7c2492e125594307480c46679278a6" + integrity sha512-LwaxwyZ72Lk7vZINtNNrywX0ZuLyStrdDtabefZKAY5ZGJhVtgdznluResxNmPitE0SAO+O26sWTHeKSI2wMBA== + dependencies: + fs.realpath "^1.0.0" + inflight "^1.0.4" + inherits "2" + minimatch "^3.0.4" + once "^1.3.0" + path-is-absolute "^1.0.0" + +glob@7.1.7: + version "7.1.7" + resolved "https://registry.yarnpkg.com/glob/-/glob-7.1.7.tgz#3b193e9233f01d42d0b3f78294bbeeb418f94a90" + integrity sha512-OvD9ENzPLbegENnYP5UUfJIirTg4+XwMWGaQfQTY0JenxNvvIKP3U3/tAQSPIu/lHxXYSZmpXlUHeqAIdKzBLQ== + dependencies: + fs.realpath "^1.0.0" + inflight "^1.0.4" + inherits "2" + minimatch "^3.0.4" + once "^1.3.0" + path-is-absolute "^1.0.0" + +glob@^10.0.0: + version "10.3.4" + resolved "https://registry.yarnpkg.com/glob/-/glob-10.3.4.tgz#c85c9c7ab98669102b6defda76d35c5b1ef9766f" + integrity sha512-6LFElP3A+i/Q8XQKEvZjkEWEOTgAIALR9AO2rwT8bgPhDd1anmqDJDZ6lLddI4ehxxxR1S5RIqKe1uapMQfYaQ== + dependencies: + foreground-child "^3.1.0" + jackspeak "^2.0.3" + minimatch "^9.0.1" + minipass "^5.0.0 || ^6.0.2 || ^7.0.0" + path-scurry "^1.10.1" + +glob@^7.0.3, glob@^7.1.3, glob@^7.1.4, glob@^7.2.0: + version "7.2.3" + resolved "https://registry.yarnpkg.com/glob/-/glob-7.2.3.tgz#b8df0fb802bbfa8e89bd1d938b4e16578ed44f2b" + integrity sha512-nFR0zLpU2YCaRxwoCJvL6UvCH2JFyFVIvwTLsIf21AuHlMskA1hhTdk+LlYJtOlYt9v6dvszD2BGRqBL+iQK9Q== + dependencies: + fs.realpath "^1.0.0" + inflight "^1.0.4" + inherits "2" + minimatch "^3.1.1" + once "^1.3.0" + path-is-absolute "^1.0.0" + +globals@^11.1.0: + version "11.12.0" + resolved "https://registry.yarnpkg.com/globals/-/globals-11.12.0.tgz#ab8795338868a0babd8525758018c2a7eb95c42e" + integrity sha512-WOBp/EEGUiIsJSp7wcv/y6MO+lV9UoncWqxuFfm8eBwzWNgyfBd6Gz+IeKQ9jCmyhoH99g15M3T+QaVHFjizVA== + +globals@^13.19.0: + version "13.21.0" + resolved "https://registry.yarnpkg.com/globals/-/globals-13.21.0.tgz#163aae12f34ef502f5153cfbdd3600f36c63c571" + integrity sha512-ybyme3s4yy/t/3s35bewwXKOf7cvzfreG2lH0lZl0JB7I4GxRP2ghxOK/Nb9EkRXdbBXZLfq/p/0W2JUONB/Gg== + dependencies: + type-fest "^0.20.2" + +globalthis@^1.0.3: + version "1.0.3" + resolved "https://registry.yarnpkg.com/globalthis/-/globalthis-1.0.3.tgz#5852882a52b80dc301b0660273e1ed082f0b6ccf" + integrity sha512-sFdI5LyBiNTHjRd7cGPWapiHWMOXKyuBNX/cWJ3NfzrZQVa8GI/8cofCl74AOVqq9W5kNmguTIzJ/1s2gyI9wA== + dependencies: + define-properties "^1.1.3" + +globby@^11.0.1, globby@^11.0.2, globby@^11.1.0: + version "11.1.0" + resolved "https://registry.yarnpkg.com/globby/-/globby-11.1.0.tgz#bd4be98bb042f83d796f7e3811991fbe82a0d34b" + integrity sha512-jhIXaOzy1sb8IyocaruWSn1TjmnBVs8Ayhcy83rmxNJ8q2uWKCAj3CnJY+KpGSXCueAPc0i05kVvVKtP1t9S3g== + dependencies: + array-union "^2.1.0" + dir-glob "^3.0.1" + fast-glob "^3.2.9" + ignore "^5.2.0" + merge2 "^1.4.1" + slash "^3.0.0" + +globby@^13.1.1: + version "13.2.2" + resolved "https://registry.yarnpkg.com/globby/-/globby-13.2.2.tgz#63b90b1bf68619c2135475cbd4e71e66aa090592" + integrity sha512-Y1zNGV+pzQdh7H39l9zgB4PJqjRNqydvdYCDG4HFXM4XuvSaQQlEc91IU1yALL8gUTDomgBAfz3XJdmUS+oo0w== + dependencies: + dir-glob "^3.0.1" + fast-glob "^3.3.0" + ignore "^5.2.4" + merge2 "^1.4.1" + slash "^4.0.0" + +globby@^6.1.0: + version "6.1.0" + resolved "https://registry.yarnpkg.com/globby/-/globby-6.1.0.tgz#f5a6d70e8395e21c858fb0489d64df02424d506c" + integrity sha512-KVbFv2TQtbzCoxAnfD6JcHZTYCzyliEaaeM/gH8qQdkKr5s0OP9scEgvdcngyk7AVdY6YVW/TJHd+lQ/Df3Daw== + dependencies: + array-union "^1.0.1" + glob "^7.0.3" + object-assign "^4.0.1" + pify "^2.0.0" + pinkie-promise "^2.0.0" + +gopd@^1.0.1: + version "1.0.1" + resolved "https://registry.yarnpkg.com/gopd/-/gopd-1.0.1.tgz#29ff76de69dac7489b7c0918a5788e56477c332c" + integrity sha512-d65bNlIadxvpb/A2abVdlqKqV563juRnZ1Wtk6s1sIR8uNsXR70xqIzVqxVf1eTqDunwT2MkczEeaezCKTZhwA== + dependencies: + get-intrinsic "^1.1.3" + +graceful-fs@^4.1.11, graceful-fs@^4.1.2, graceful-fs@^4.1.6, graceful-fs@^4.2.0, graceful-fs@^4.2.4, graceful-fs@^4.2.9: + version "4.2.11" + resolved "https://registry.yarnpkg.com/graceful-fs/-/graceful-fs-4.2.11.tgz#4183e4e8bf08bb6e05bbb2f7d2e0c8f712ca40e3" + integrity sha512-RbJ5/jmFcNNCcDV5o9eTnBLJ/HszWV0P73bc+Ff4nS/rJj+YaS6IGyiOL0VoBYX+l1Wrl3k63h/KrH+nhJ0XvQ== + +graphemer@^1.4.0: + version "1.4.0" + resolved "https://registry.yarnpkg.com/graphemer/-/graphemer-1.4.0.tgz#fb2f1d55e0e3a1849aeffc90c4fa0dd53a0e66c6" + integrity sha512-EtKwoO6kxCL9WO5xipiHTZlSzBm7WLT627TqC/uVRd0HKmq8NXyebnNYxDoBi7wt8eTWrUrKXCOVaFq9x1kgag== + +gunzip-maybe@^1.4.2: + version "1.4.2" + resolved "https://registry.yarnpkg.com/gunzip-maybe/-/gunzip-maybe-1.4.2.tgz#b913564ae3be0eda6f3de36464837a9cd94b98ac" + integrity sha512-4haO1M4mLO91PW57BMsDFf75UmwoRX0GkdD+Faw+Lr+r/OZrOCS0pIBwOL1xCKQqnQzbNFGgK2V2CpBUPeFNTw== + dependencies: + browserify-zlib "^0.1.4" + is-deflate "^1.0.0" + is-gzip "^1.0.0" + peek-stream "^1.1.0" + pumpify "^1.3.3" + through2 "^2.0.3" + +handlebars@^4.7.7: + version "4.7.8" + resolved "https://registry.yarnpkg.com/handlebars/-/handlebars-4.7.8.tgz#41c42c18b1be2365439188c77c6afae71c0cd9e9" + integrity sha512-vafaFqs8MZkRrSX7sFVUdo3ap/eNiLnb4IakshzvP56X5Nr1iGKAIqdX6tMlm6HcNRIkr6AxO5jFEoJzzpT8aQ== + dependencies: + minimist "^1.2.5" + neo-async "^2.6.2" + source-map "^0.6.1" + wordwrap "^1.0.0" + optionalDependencies: + uglify-js "^3.1.4" + +has-bigints@^1.0.1, has-bigints@^1.0.2: + version "1.0.2" + resolved "https://registry.yarnpkg.com/has-bigints/-/has-bigints-1.0.2.tgz#0871bd3e3d51626f6ca0966668ba35d5602d6eaa" + integrity sha512-tSvCKtBr9lkF0Ex0aQiP9N+OpV4zi2r/Nee5VkRDbaqv35RLYMzbwQfFSZZH0kR+Rd6302UJZ2p/bJCEoR3VoQ== + +has-flag@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/has-flag/-/has-flag-3.0.0.tgz#b5d454dc2199ae225699f3467e5a07f3b955bafd" + integrity sha512-sKJf1+ceQBr4SMkvQnBDNDtf4TXpVhVGateu0t918bl30FnbE2m4vNLX+VWe/dpjlb+HugGYzW7uQXH98HPEYw== + +has-flag@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/has-flag/-/has-flag-4.0.0.tgz#944771fd9c81c81265c4d6941860da06bb59479b" + integrity sha512-EykJT/Q1KjTWctppgIAgfSO0tKVuZUjhgMr17kqTumMl6Afv3EISleU7qZUzoXDFTAHTDC4NOoG/ZxU3EvlMPQ== + +has-property-descriptors@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/has-property-descriptors/-/has-property-descriptors-1.0.0.tgz#610708600606d36961ed04c196193b6a607fa861" + integrity sha512-62DVLZGoiEBDHQyqG4w9xCuZ7eJEwNmJRWw2VY84Oedb7WFcA27fiEVe8oUQx9hAUJ4ekurquucTGwsyO1XGdQ== + dependencies: + get-intrinsic "^1.1.1" + +has-proto@^1.0.1: + version "1.0.1" + resolved "https://registry.yarnpkg.com/has-proto/-/has-proto-1.0.1.tgz#1885c1305538958aff469fef37937c22795408e0" + integrity sha512-7qE+iP+O+bgF9clE5+UoBFzE65mlBiVj3tKCrlNQ0Ogwm0BjpT/gK4SlLYDMybDh5I3TCTKnPPa0oMG7JDYrhg== + +has-symbols@^1.0.2, has-symbols@^1.0.3: + version "1.0.3" + resolved "https://registry.yarnpkg.com/has-symbols/-/has-symbols-1.0.3.tgz#bb7b2c4349251dce87b125f7bdf874aa7c8b39f8" + integrity sha512-l3LCuF6MgDNwTDKkdYGEihYjt5pRPbEg46rtlmnSPlUbgmB8LOIrKJbYYFBSbnPaJexMKtiPO8hmeRjRz2Td+A== + +has-tostringtag@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/has-tostringtag/-/has-tostringtag-1.0.0.tgz#7e133818a7d394734f941e73c3d3f9291e658b25" + integrity sha512-kFjcSNhnlGV1kyoGk7OXKSawH5JOb/LzUc5w9B02hOTO0dfFRjbHQKvg1d6cf3HbeUmtU9VbbV3qzZ2Teh97WQ== + dependencies: + has-symbols "^1.0.2" + +has@^1.0.3: + version "1.0.3" + resolved "https://registry.yarnpkg.com/has/-/has-1.0.3.tgz#722d7cbfc1f6aa8241f16dd814e011e1f41e8796" + integrity sha512-f2dvO0VU6Oej7RkWJGrehjbzMAjFp5/VKPp5tTpWIV4JHHZK1/BxbFRtf/siA2SWTe09caDmVtYYzWEIbBS4zw== + dependencies: + function-bind "^1.1.1" + +hash-base@^3.0.0: + version "3.1.0" + resolved "https://registry.yarnpkg.com/hash-base/-/hash-base-3.1.0.tgz#55c381d9e06e1d2997a883b4a3fddfe7f0d3af33" + integrity sha512-1nmYp/rhMDiE7AYkDw+lLwlAzz0AntGIe51F3RfFfEqyQ3feY2eI/NcwC6umIQVOASPMsWJLJScWKSSvzL9IVA== + dependencies: + inherits "^2.0.4" + readable-stream "^3.6.0" + safe-buffer "^5.2.0" + +hash.js@^1.0.0, hash.js@^1.0.3: + version "1.1.7" + resolved "https://registry.yarnpkg.com/hash.js/-/hash.js-1.1.7.tgz#0babca538e8d4ee4a0f8988d68866537a003cf42" + integrity sha512-taOaskGt4z4SOANNseOviYDvjEJinIkRgmp7LbKP2YTTmVxWBl87s/uzK9r+44BclBSp2X7K1hqeNfz9JbBeXA== + dependencies: + inherits "^2.0.3" + minimalistic-assert "^1.0.1" + +he@^1.2.0: + version "1.2.0" + resolved "https://registry.yarnpkg.com/he/-/he-1.2.0.tgz#84ae65fa7eafb165fddb61566ae14baf05664f0f" + integrity sha512-F/1DnUGPopORZi0ni+CvrCgHQ5FyEAHRLSApuYWMmrbSwoN2Mn/7k+Gl38gJnR7yyDZk6WLXwiGod1JOWNDKGw== + +hmac-drbg@^1.0.1: + version "1.0.1" + resolved "https://registry.yarnpkg.com/hmac-drbg/-/hmac-drbg-1.0.1.tgz#d2745701025a6c775a6c545793ed502fc0c649a1" + integrity sha512-Tti3gMqLdZfhOQY1Mzf/AanLiqh1WTiJgEj26ZuYQ9fbkLomzGchCws4FyrSd4VkpBfiNhaE1On+lOz894jvXg== + dependencies: + hash.js "^1.0.3" + minimalistic-assert "^1.0.0" + minimalistic-crypto-utils "^1.0.1" + +hosted-git-info@^2.1.4: + version "2.8.9" + resolved "https://registry.yarnpkg.com/hosted-git-info/-/hosted-git-info-2.8.9.tgz#dffc0bf9a21c02209090f2aa69429e1414daf3f9" + integrity sha512-mxIDAb9Lsm6DoOJ7xH+5+X4y1LU/4Hi50L9C5sIswK3JzULS4bwk1FvjdBgvYR4bzT4tuUQiC15FE2f5HbLvYw== + +html-entities@^2.1.0: + version "2.4.0" + resolved "https://registry.yarnpkg.com/html-entities/-/html-entities-2.4.0.tgz#edd0cee70402584c8c76cc2c0556db09d1f45061" + integrity sha512-igBTJcNNNhvZFRtm8uA6xMY6xYleeDwn3PeBCkDz7tHttv4F2hsDI2aPgNERWzvRcNYHNT3ymRaQzllmXj4YsQ== + +html-escaper@^2.0.0: + version "2.0.2" + resolved "https://registry.yarnpkg.com/html-escaper/-/html-escaper-2.0.2.tgz#dfd60027da36a36dfcbe236262c00a5822681453" + integrity sha512-H2iMtd0I4Mt5eYiapRdIDjp+XzelXQ0tFE4JS7YFwFevXXMmOp9myNrUvCg0D6ws8iqkRPBfKHgbwig1SmlLfg== + +html-minifier-terser@^6.0.2: + version "6.1.0" + resolved "https://registry.yarnpkg.com/html-minifier-terser/-/html-minifier-terser-6.1.0.tgz#bfc818934cc07918f6b3669f5774ecdfd48f32ab" + integrity sha512-YXxSlJBZTP7RS3tWnQw74ooKa6L9b9i9QYXY21eUEvhZ3u9XLfv6OnFsQq6RxkhHygsaUMvYsZRV5rU/OVNZxw== + dependencies: + camel-case "^4.1.2" + clean-css "^5.2.2" + commander "^8.3.0" + he "^1.2.0" + param-case "^3.0.4" + relateurl "^0.2.7" + terser "^5.10.0" + +html-tags@^3.1.0: + version "3.3.1" + resolved "https://registry.yarnpkg.com/html-tags/-/html-tags-3.3.1.tgz#a04026a18c882e4bba8a01a3d39cfe465d40b5ce" + integrity sha512-ztqyC3kLto0e9WbNp0aeP+M3kTt+nbaIveGmUxAtZa+8iFgKLUOD4YKM5j+f3QD89bra7UeumolZHKuOXnTmeQ== + +html-webpack-plugin@^5.5.0: + version "5.5.3" + resolved "https://registry.yarnpkg.com/html-webpack-plugin/-/html-webpack-plugin-5.5.3.tgz#72270f4a78e222b5825b296e5e3e1328ad525a3e" + integrity sha512-6YrDKTuqaP/TquFH7h4srYWsZx+x6k6+FbsTm0ziCwGHDP78Unr1r9F/H4+sGmMbX08GQcJ+K64x55b+7VM/jg== + dependencies: + "@types/html-minifier-terser" "^6.0.0" + html-minifier-terser "^6.0.2" + lodash "^4.17.21" + pretty-error "^4.0.0" + tapable "^2.0.0" + +htmlparser2@^6.1.0: + version "6.1.0" + resolved "https://registry.yarnpkg.com/htmlparser2/-/htmlparser2-6.1.0.tgz#c4d762b6c3371a05dbe65e94ae43a9f845fb8fb7" + integrity sha512-gyyPk6rgonLFEDGoeRgQNaEUvdJ4ktTmmUh/h2t7s+M8oPpIPxgNACWa+6ESR57kXstwqPiCut0V8NRpcwgU7A== + dependencies: + domelementtype "^2.0.1" + domhandler "^4.0.0" + domutils "^2.5.2" + entities "^2.0.0" + +http-errors@2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/http-errors/-/http-errors-2.0.0.tgz#b7774a1486ef73cf7667ac9ae0858c012c57b9d3" + integrity sha512-FtwrG/euBzaEjYeRqOgly7G0qviiXoJWnvEH2Z1plBdXgbyjv34pHTSb9zoeHMyDy33+DWy5Wt9Wo+TURtOYSQ== + dependencies: + depd "2.0.0" + inherits "2.0.4" + setprototypeof "1.2.0" + statuses "2.0.1" + toidentifier "1.0.1" + +https-browserify@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/https-browserify/-/https-browserify-1.0.0.tgz#ec06c10e0a34c0f2faf199f7fd7fc78fffd03c73" + integrity sha512-J+FkSdyD+0mA0N+81tMotaRMfSL9SGi+xpD3T6YApKsc3bGSXJlfXri3VyFOeYkfLRQisDk1W+jIFFKBeUBbBg== + +https-proxy-agent@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/https-proxy-agent/-/https-proxy-agent-4.0.0.tgz#702b71fb5520a132a66de1f67541d9e62154d82b" + integrity sha512-zoDhWrkR3of1l9QAL8/scJZyLu8j/gBkcwcaQOZh7Gyh/+uJQzGVETdgT30akuwkpL8HTRfssqI3BZuV18teDg== + dependencies: + agent-base "5" + debug "4" + +https-proxy-agent@^5.0.1: + version "5.0.1" + resolved "https://registry.yarnpkg.com/https-proxy-agent/-/https-proxy-agent-5.0.1.tgz#c59ef224a04fe8b754f3db0063a25ea30d0005d6" + integrity sha512-dFcAjpTQFgoLMzC2VwU+C/CbS7uRL0lWmxDITmqm7C+7F0Odmj6s9l6alZc6AELXhrnggM2CeWSXHGOdX2YtwA== + dependencies: + agent-base "6" + debug "4" + +human-signals@^2.1.0: + version "2.1.0" + resolved "https://registry.yarnpkg.com/human-signals/-/human-signals-2.1.0.tgz#dc91fcba42e4d06e4abaed33b3e7a3c02f514ea0" + integrity sha512-B4FFZ6q/T2jhhksgkbEW3HBvWIfDW85snkQgawt07S7J5QXTk6BkNV+0yAeZrM5QpMAdYlocGoljn0sJ/WQkFw== + +iconv-lite@0.4.24: + version "0.4.24" + resolved "https://registry.yarnpkg.com/iconv-lite/-/iconv-lite-0.4.24.tgz#2022b4b25fbddc21d2f524974a474aafe733908b" + integrity sha512-v3MXnZAcvnywkTUEZomIActle7RXXeedOR31wwl7VlyoXO4Qi9arvSenNQWne1TcRwhCL1HwLI21bEqdpj8/rA== + dependencies: + safer-buffer ">= 2.1.2 < 3" + +iconv-lite@^0.6.3: + version "0.6.3" + resolved "https://registry.yarnpkg.com/iconv-lite/-/iconv-lite-0.6.3.tgz#a52f80bf38da1952eb5c681790719871a1a72501" + integrity sha512-4fCk79wshMdzMp2rH06qWrJE4iolqLhCUH+OiuIgU++RB0+94NlDL81atO7GX55uUKueo0txHNtvEyI6D7WdMw== + dependencies: + safer-buffer ">= 2.1.2 < 3.0.0" + +icss-utils@^5.0.0, icss-utils@^5.1.0: + version "5.1.0" + resolved "https://registry.yarnpkg.com/icss-utils/-/icss-utils-5.1.0.tgz#c6be6858abd013d768e98366ae47e25d5887b1ae" + integrity sha512-soFhflCVWLfRNOPU3iv5Z9VUdT44xFRbzjLsEzSr5AQmgqPMTHdU3PMT1Cf1ssx8fLNJDA1juftYl+PUcv3MqA== + +ieee754@^1.1.13, ieee754@^1.2.1: + version "1.2.1" + resolved "https://registry.yarnpkg.com/ieee754/-/ieee754-1.2.1.tgz#8eb7a10a63fff25d15a57b001586d177d1b0d352" + integrity sha512-dcyqhDvX1C46lXZcVqCpK+FtMRQVdIMN6/Df5js2zouUsqG7I6sFxitIC+7KYK29KdXOLHdu9zL4sFnoVQnqaA== + +ignore@^5.2.0, ignore@^5.2.4: + version "5.2.4" + resolved "https://registry.yarnpkg.com/ignore/-/ignore-5.2.4.tgz#a291c0c6178ff1b960befe47fcdec301674a6324" + integrity sha512-MAb38BcSbH0eHNBxn7ql2NH/kX33OkB3lZ1BNdh7ENeRChHTYsTvWrMubiIAMNS2llXEEgZ1MUOBtXChP3kaFQ== + +image-size@^1.0.0: + version "1.0.2" + resolved "https://registry.yarnpkg.com/image-size/-/image-size-1.0.2.tgz#d778b6d0ab75b2737c1556dd631652eb963bc486" + integrity sha512-xfOoWjceHntRb3qFCrh5ZFORYH8XCdYpASltMhZ/Q0KZiOwjdE/Yl2QCiWdwD+lygV5bMCvauzgu5PxBX/Yerg== + dependencies: + queue "6.0.2" + +image-size@~0.5.0: + version "0.5.5" + resolved "https://registry.yarnpkg.com/image-size/-/image-size-0.5.5.tgz#09dfd4ab9d20e29eb1c3e80b8990378df9e3cb9c" + integrity sha512-6TDAlDPZxUFCv+fuOkIoXT/V/f3Qbq8e37p+YOiYrUv3v9cc3/6x78VdfPgFVaB9dZYeLUfKgHRebpkm/oP2VQ== + +import-fresh@^3.2.1, import-fresh@^3.3.0: + version "3.3.0" + resolved "https://registry.yarnpkg.com/import-fresh/-/import-fresh-3.3.0.tgz#37162c25fcb9ebaa2e6e53d5b4d88ce17d9e0c2b" + integrity sha512-veYYhQa+D1QBKznvhUHxb8faxlrwUnxseDAbAp457E0wLNio2bOSKnjYDhMj+YiAq61xrMGhQk9iXVk5FzgQMw== + dependencies: + parent-module "^1.0.0" + resolve-from "^4.0.0" + +imurmurhash@^0.1.4: + version "0.1.4" + resolved "https://registry.yarnpkg.com/imurmurhash/-/imurmurhash-0.1.4.tgz#9218b9b2b928a238b13dc4fb6b6d576f231453ea" + integrity sha512-JmXMZ6wuvDmLiHEml9ykzqO6lwFbof0GG4IkcGaENdCRDDmMVnny7s5HsIgHCbaq0w2MyPhDqkhTUgS2LU2PHA== + +indent-string@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/indent-string/-/indent-string-4.0.0.tgz#624f8f4497d619b2d9768531d58f4122854d7251" + integrity sha512-EdDDZu4A2OyIK7Lr/2zG+w5jmbuk1DVBnEwREQvBzspBJkCEbRa8GxU1lghYcaGJCnRWibjDXlq779X1/y5xwg== + +inflight@^1.0.4: + version "1.0.6" + resolved "https://registry.yarnpkg.com/inflight/-/inflight-1.0.6.tgz#49bd6331d7d02d0c09bc910a1075ba8165b56df9" + integrity sha512-k92I/b08q4wvFscXCLvqfsHCrjrF7yiXsQuIVvVE7N82W3+aqpzuUdBbfhWcy/FZR3/4IgflMgKLOsvPDrGCJA== + dependencies: + once "^1.3.0" + wrappy "1" + +inherits@2, inherits@2.0.4, inherits@^2.0.1, inherits@^2.0.3, inherits@^2.0.4, inherits@~2.0.3, inherits@~2.0.4: + version "2.0.4" + resolved "https://registry.yarnpkg.com/inherits/-/inherits-2.0.4.tgz#0fa2c64f932917c3433a0ded55363aae37416b7c" + integrity sha512-k/vGaX4/Yla3WzyMCvTQOXYeIHvqOKtnqBduzTHpzpQZzAskKMhZ2K+EnBiSM9zGSoIFeMpXKxa4dYeZIQqewQ== + +internal-slot@^1.0.4, internal-slot@^1.0.5: + version "1.0.5" + resolved "https://registry.yarnpkg.com/internal-slot/-/internal-slot-1.0.5.tgz#f2a2ee21f668f8627a4667f309dc0f4fb6674986" + integrity sha512-Y+R5hJrzs52QCG2laLn4udYVnxsfny9CpOhNhUvk/SSSVyF6T27FzRbF0sroPidSu3X8oEAkOn2K804mjpt6UQ== + dependencies: + get-intrinsic "^1.2.0" + has "^1.0.3" + side-channel "^1.0.4" + +"internmap@1 - 2": + version "2.0.3" + resolved "https://registry.yarnpkg.com/internmap/-/internmap-2.0.3.tgz#6685f23755e43c524e251d29cbc97248e3061009" + integrity sha512-5Hh7Y1wQbvY5ooGgPbDaL5iYLAPzMTUrjMulskHLH6wnv/A+1q5rgEaiuqEjB+oxGXIVZs1FF+R/KPN3ZSQYYg== + +invariant@^2.2.4: + version "2.2.4" + resolved "https://registry.yarnpkg.com/invariant/-/invariant-2.2.4.tgz#610f3c92c9359ce1db616e538008d23ff35158e6" + integrity sha512-phJfQVBuaJM5raOpJjSfkiD6BpbCE4Ns//LaXl6wGYtUBY83nWS6Rf9tXm2e8VaK60JEjYldbPif/A2B1C2gNA== + dependencies: + loose-envify "^1.0.0" + +ip@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/ip/-/ip-2.0.0.tgz#4cf4ab182fee2314c75ede1276f8c80b479936da" + integrity sha512-WKa+XuLG1A1R0UWhl2+1XQSi+fZWMsYKffMZTTYsiZaUD8k2yDAj5atimTUD2TZkyCkNEeYE5NhFZmupOGtjYQ== + +ipaddr.js@1.9.1: + version "1.9.1" + resolved "https://registry.yarnpkg.com/ipaddr.js/-/ipaddr.js-1.9.1.tgz#bff38543eeb8984825079ff3a2a8e6cbd46781b3" + integrity sha512-0KI/607xoxSToH7GjN1FfSbLoU0+btTicjsQSWQlh/hZykN8KpmMf7uYwPW3R+akZ6R/w18ZlXSHBYXiYUPO3g== + +is-absolute-url@^3.0.0: + version "3.0.3" + resolved "https://registry.yarnpkg.com/is-absolute-url/-/is-absolute-url-3.0.3.tgz#96c6a22b6a23929b11ea0afb1836c36ad4a5d698" + integrity sha512-opmNIX7uFnS96NtPmhWQgQx6/NYFgsUXYMllcfzwWKUMwfo8kku1TvE6hkNcH+Q1ts5cMVrsY7j0bxXQDciu9Q== + +is-arguments@^1.0.4, is-arguments@^1.1.1: + version "1.1.1" + resolved "https://registry.yarnpkg.com/is-arguments/-/is-arguments-1.1.1.tgz#15b3f88fda01f2a97fec84ca761a560f123efa9b" + integrity sha512-8Q7EARjzEnKpt/PCD7e1cgUS0a6X8u5tdSiMqXhojOdoV9TsMsiO+9VLC5vAmO8N7/GmXn7yjR8qnA6bVAEzfA== + dependencies: + call-bind "^1.0.2" + has-tostringtag "^1.0.0" + +is-array-buffer@^3.0.1, is-array-buffer@^3.0.2: + version "3.0.2" + resolved "https://registry.yarnpkg.com/is-array-buffer/-/is-array-buffer-3.0.2.tgz#f2653ced8412081638ecb0ebbd0c41c6e0aecbbe" + integrity sha512-y+FyyR/w8vfIRq4eQcM1EYgSTnmHXPqaF+IgzgraytCFq5Xh8lllDVmAZolPJiZttZLeFSINPYMaEJ7/vWUa1w== + dependencies: + call-bind "^1.0.2" + get-intrinsic "^1.2.0" + is-typed-array "^1.1.10" + +is-arrayish@^0.2.1: + version "0.2.1" + resolved "https://registry.yarnpkg.com/is-arrayish/-/is-arrayish-0.2.1.tgz#77c99840527aa8ecb1a8ba697b80645a7a926a9d" + integrity sha512-zz06S8t0ozoDXMG+ube26zeCTNXcKIPJZJi8hBrF4idCLms4CG9QtK7qBl1boi5ODzFpjswb5JPmHCbMpjaYzg== + +is-async-function@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/is-async-function/-/is-async-function-2.0.0.tgz#8e4418efd3e5d3a6ebb0164c05ef5afb69aa9646" + integrity sha512-Y1JXKrfykRJGdlDwdKlLpLyMIiWqWvuSd17TvZk68PLAOGOoF4Xyav1z0Xhoi+gCYjZVeC5SI+hYFOfvXmGRCA== + dependencies: + has-tostringtag "^1.0.0" + +is-bigint@^1.0.1: + version "1.0.4" + resolved "https://registry.yarnpkg.com/is-bigint/-/is-bigint-1.0.4.tgz#08147a1875bc2b32005d41ccd8291dffc6691df3" + integrity sha512-zB9CruMamjym81i2JZ3UMn54PKGsQzsJeo6xvN3HJJ4CAsQNB6iRutp2To77OfCNuoxspsIhzaPoO1zyCEhFOg== + dependencies: + has-bigints "^1.0.1" + +is-binary-path@~2.1.0: + version "2.1.0" + resolved "https://registry.yarnpkg.com/is-binary-path/-/is-binary-path-2.1.0.tgz#ea1f7f3b80f064236e83470f86c09c254fb45b09" + integrity sha512-ZMERYes6pDydyuGidse7OsHxtbI7WVeUEozgR/g7rd0xUimYNlvZRE/K2MgZTjWy725IfelLeVcEM97mmtRGXw== + dependencies: + binary-extensions "^2.0.0" + +is-boolean-object@^1.1.0: + version "1.1.2" + resolved "https://registry.yarnpkg.com/is-boolean-object/-/is-boolean-object-1.1.2.tgz#5c6dc200246dd9321ae4b885a114bb1f75f63719" + integrity sha512-gDYaKHJmnj4aWxyj6YHyXVpdQawtVLHU5cb+eztPGczf6cjuTdwve5ZIEfgXqH4e57An1D1AKf8CZ3kYrQRqYA== + dependencies: + call-bind "^1.0.2" + has-tostringtag "^1.0.0" + +is-callable@^1.1.3, is-callable@^1.1.4, is-callable@^1.2.7: + version "1.2.7" + resolved "https://registry.yarnpkg.com/is-callable/-/is-callable-1.2.7.tgz#3bc2a85ea742d9e36205dcacdd72ca1fdc51b055" + integrity sha512-1BC0BVFhS/p0qtw6enp8e+8OD0UrK0oFLztSjNzhcKA3WDuJxxAPXzPuPtKkjEY9UUoEWlX/8fgKeu2S8i9JTA== + +is-core-module@^2.11.0, is-core-module@^2.13.0, is-core-module@^2.9.0: + version "2.13.0" + resolved "https://registry.yarnpkg.com/is-core-module/-/is-core-module-2.13.0.tgz#bb52aa6e2cbd49a30c2ba68c42bf3435ba6072db" + integrity sha512-Z7dk6Qo8pOCp3l4tsX2C5ZVas4V+UxwQodwZhLopL91TX8UyyHEXafPcyoeeWuLrwzHcr3igO78wNLwHJHsMCQ== + dependencies: + has "^1.0.3" + +is-date-object@^1.0.1, is-date-object@^1.0.5: + version "1.0.5" + resolved "https://registry.yarnpkg.com/is-date-object/-/is-date-object-1.0.5.tgz#0841d5536e724c25597bf6ea62e1bd38298df31f" + integrity sha512-9YQaSxsAiSwcvS33MBk3wTCVnWK+HhF8VZR2jRxehM16QcVOdHqPn4VPHmRK4lSr38n9JriurInLcP90xsYNfQ== + dependencies: + has-tostringtag "^1.0.0" + +is-deflate@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/is-deflate/-/is-deflate-1.0.0.tgz#c862901c3c161fb09dac7cdc7e784f80e98f2f14" + integrity sha512-YDoFpuZWu1VRXlsnlYMzKyVRITXj7Ej/V9gXQ2/pAe7X1J7M/RNOqaIYi6qUn+B7nGyB9pDXrv02dsB58d2ZAQ== + +is-docker@^2.0.0, is-docker@^2.1.1: + version "2.2.1" + resolved "https://registry.yarnpkg.com/is-docker/-/is-docker-2.2.1.tgz#33eeabe23cfe86f14bde4408a02c0cfb853acdaa" + integrity sha512-F+i2BKsFrH66iaUFc0woD8sLy8getkwTwtOBjvs56Cx4CgJDeKQeqfz8wAYiSb8JOprWhHH5p77PbmYCvvUuXQ== + +is-extglob@^2.1.1: + version "2.1.1" + resolved "https://registry.yarnpkg.com/is-extglob/-/is-extglob-2.1.1.tgz#a88c02535791f02ed37c76a1b9ea9773c833f8c2" + integrity sha512-SbKbANkN603Vi4jEZv49LeVJMn4yGwsbzZworEoyEiutsN3nJYdbO36zfhGJ6QEDpOZIFkDtnq5JRxmvl3jsoQ== + +is-finalizationregistry@^1.0.2: + version "1.0.2" + resolved "https://registry.yarnpkg.com/is-finalizationregistry/-/is-finalizationregistry-1.0.2.tgz#c8749b65f17c133313e661b1289b95ad3dbd62e6" + integrity sha512-0by5vtUJs8iFQb5TYUHHPudOR+qXYIMKtiUzvLIZITZUjknFmziyBJuLhVRc+Ds0dREFlskDNJKYIdIzu/9pfw== + dependencies: + call-bind "^1.0.2" + +is-fullwidth-code-point@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/is-fullwidth-code-point/-/is-fullwidth-code-point-3.0.0.tgz#f116f8064fe90b3f7844a38997c0b75051269f1d" + integrity sha512-zymm5+u+sCsSWyD9qNaejV3DFvhCKclKdizYaJUuHA83RLjb7nSuGnddCHGv0hk+KY7BMAlsWeK4Ueg6EV6XQg== + +is-generator-function@^1.0.10, is-generator-function@^1.0.7: + version "1.0.10" + resolved "https://registry.yarnpkg.com/is-generator-function/-/is-generator-function-1.0.10.tgz#f1558baf1ac17e0deea7c0415c438351ff2b3c72" + integrity sha512-jsEjy9l3yiXEQ+PsXdmBwEPcOxaXWLspKdplFUVI9vq1iZgIekeC0L167qeu86czQaxed3q/Uzuw0swL0irL8A== + dependencies: + has-tostringtag "^1.0.0" + +is-glob@^4.0.0, is-glob@^4.0.1, is-glob@^4.0.3, is-glob@~4.0.1: + version "4.0.3" + resolved "https://registry.yarnpkg.com/is-glob/-/is-glob-4.0.3.tgz#64f61e42cbbb2eec2071a9dac0b28ba1e65d5084" + integrity sha512-xelSayHH36ZgE7ZWhli7pW34hNbNl8Ojv5KVmkJD4hBdD3th8Tfk9vYasLM+mXWOZhFkgZfxhLSnrwRr4elSSg== + dependencies: + is-extglob "^2.1.1" + +is-gzip@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/is-gzip/-/is-gzip-1.0.0.tgz#6ca8b07b99c77998025900e555ced8ed80879a83" + integrity sha512-rcfALRIb1YewtnksfRIHGcIY93QnK8BIQ/2c9yDYcG/Y6+vRoJuTWBmmSEbyLLYtXm7q35pHOHbZFQBaLrhlWQ== + +is-interactive@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/is-interactive/-/is-interactive-1.0.0.tgz#cea6e6ae5c870a7b0a0004070b7b587e0252912e" + integrity sha512-2HvIEKRoqS62guEC+qBjpvRubdX910WCMuJTZ+I9yvqKU2/12eSL549HMwtabb4oupdj2sMP50k+XJfB/8JE6w== + +is-map@^2.0.1, is-map@^2.0.2: + version "2.0.2" + resolved "https://registry.yarnpkg.com/is-map/-/is-map-2.0.2.tgz#00922db8c9bf73e81b7a335827bc2a43f2b91127" + integrity sha512-cOZFQQozTha1f4MxLFzlgKYPTyj26picdZTx82hbc/Xf4K/tZOOXSCkMvU4pKioRXGDLJRn0GM7Upe7kR721yg== + +is-nan@^1.2.1: + version "1.3.2" + resolved "https://registry.yarnpkg.com/is-nan/-/is-nan-1.3.2.tgz#043a54adea31748b55b6cd4e09aadafa69bd9e1d" + integrity sha512-E+zBKpQ2t6MEo1VsonYmluk9NxGrbzpeeLC2xIViuO2EjU2xsXsBPwTr3Ykv9l08UYEVEdWeRZNouaZqF6RN0w== + dependencies: + call-bind "^1.0.0" + define-properties "^1.1.3" + +is-negative-zero@^2.0.2: + version "2.0.2" + resolved "https://registry.yarnpkg.com/is-negative-zero/-/is-negative-zero-2.0.2.tgz#7bf6f03a28003b8b3965de3ac26f664d765f3150" + integrity sha512-dqJvarLawXsFbNDeJW7zAz8ItJ9cd28YufuuFzh0G8pNHjJMnY08Dv7sYX2uF5UpQOwieAeOExEYAWWfu7ZZUA== + +is-number-object@^1.0.4: + version "1.0.7" + resolved "https://registry.yarnpkg.com/is-number-object/-/is-number-object-1.0.7.tgz#59d50ada4c45251784e9904f5246c742f07a42fc" + integrity sha512-k1U0IRzLMo7ZlYIfzRu23Oh6MiIFasgpb9X76eqfFZAqwH44UI4KTBvBYIZ1dSL9ZzChTB9ShHfLkR4pdW5krQ== + dependencies: + has-tostringtag "^1.0.0" + +is-number@^7.0.0: + version "7.0.0" + resolved "https://registry.yarnpkg.com/is-number/-/is-number-7.0.0.tgz#7535345b896734d5f80c4d06c50955527a14f12b" + integrity sha512-41Cifkg6e8TylSpdtTpeLVMqvSBEVzTttHvERD741+pnZ8ANv0004MRL43QKPDlK9cGvNp6NZWZUBlbGXYxxng== + +is-path-cwd@^2.2.0: + version "2.2.0" + resolved "https://registry.yarnpkg.com/is-path-cwd/-/is-path-cwd-2.2.0.tgz#67d43b82664a7b5191fd9119127eb300048a9fdb" + integrity sha512-w942bTcih8fdJPJmQHFzkS76NEP8Kzzvmw92cXsazb8intwLqPibPPdXf4ANdKV3rYMuuQYGIWtvz9JilB3NFQ== + +is-path-inside@^3.0.2, is-path-inside@^3.0.3: + version "3.0.3" + resolved "https://registry.yarnpkg.com/is-path-inside/-/is-path-inside-3.0.3.tgz#d231362e53a07ff2b0e0ea7fed049161ffd16283" + integrity sha512-Fd4gABb+ycGAmKou8eMftCupSir5lRxqf4aD/vd0cD2qc4HL07OjCeuHMr8Ro4CoMaeCKDB0/ECBOVWjTwUvPQ== + +is-plain-object@5.0.0: + version "5.0.0" + resolved "https://registry.yarnpkg.com/is-plain-object/-/is-plain-object-5.0.0.tgz#4427f50ab3429e9025ea7d52e9043a9ef4159344" + integrity sha512-VRSzKkbMm5jMDoKLbltAkFQ5Qr7VDiTFGXxYFXXowVj387GeGNOCsOH6Msy00SGZ3Fp84b1Naa1psqgcCIEP5Q== + +is-plain-object@^2.0.4: + version "2.0.4" + resolved "https://registry.yarnpkg.com/is-plain-object/-/is-plain-object-2.0.4.tgz#2c163b3fafb1b606d9d17928f05c2a1c38e07677" + integrity sha512-h5PpgXkWitc38BBMYawTYMWJHFZJVnBquFE57xFpjB8pJFiF6gZ+bU+WyI/yqXiFR5mdLsgYNaPe8uao6Uv9Og== + dependencies: + isobject "^3.0.1" + +is-regex@^1.1.4: + version "1.1.4" + resolved "https://registry.yarnpkg.com/is-regex/-/is-regex-1.1.4.tgz#eef5663cd59fa4c0ae339505323df6854bb15958" + integrity sha512-kvRdxDsxZjhzUX07ZnLydzS1TU/TJlTUHHY4YLL87e37oUA49DfkLqgy+VjFocowy29cKvcSiu+kIv728jTTVg== + dependencies: + call-bind "^1.0.2" + has-tostringtag "^1.0.0" + +is-set@^2.0.1, is-set@^2.0.2: + version "2.0.2" + resolved "https://registry.yarnpkg.com/is-set/-/is-set-2.0.2.tgz#90755fa4c2562dc1c5d4024760d6119b94ca18ec" + integrity sha512-+2cnTEZeY5z/iXGbLhPrOAaK/Mau5k5eXq9j14CpRTftq0pAJu2MwVRSZhyZWBzx3o6X795Lz6Bpb6R0GKf37g== + +is-shared-array-buffer@^1.0.2: + version "1.0.2" + resolved "https://registry.yarnpkg.com/is-shared-array-buffer/-/is-shared-array-buffer-1.0.2.tgz#8f259c573b60b6a32d4058a1a07430c0a7344c79" + integrity sha512-sqN2UDu1/0y6uvXyStCOzyhAjCSlHceFoMKJW8W9EU9cvic/QdsZ0kEU93HEy3IUEFZIiH/3w+AH/UQbPHNdhA== + dependencies: + call-bind "^1.0.2" + +is-stream@^2.0.0: + version "2.0.1" + resolved "https://registry.yarnpkg.com/is-stream/-/is-stream-2.0.1.tgz#fac1e3d53b97ad5a9d0ae9cef2389f5810a5c077" + integrity sha512-hFoiJiTl63nn+kstHGBtewWSKnQLpyb155KHheA1l39uvtO9nWIop1p3udqPcUd/xbF1VLMO4n7OI6p7RbngDg== + +is-string@^1.0.5, is-string@^1.0.7: + version "1.0.7" + resolved "https://registry.yarnpkg.com/is-string/-/is-string-1.0.7.tgz#0dd12bf2006f255bb58f695110eff7491eebc0fd" + integrity sha512-tE2UXzivje6ofPW7l23cjDOMa09gb7xlAqG6jG5ej6uPV32TlWP3NKPigtaGeHNu9fohccRYvIiZMfOOnOYUtg== + dependencies: + has-tostringtag "^1.0.0" + +is-symbol@^1.0.2, is-symbol@^1.0.3: + version "1.0.4" + resolved "https://registry.yarnpkg.com/is-symbol/-/is-symbol-1.0.4.tgz#a6dac93b635b063ca6872236de88910a57af139c" + integrity sha512-C/CPBqKWnvdcxqIARxyOh4v1UUEOCHpgDa0WYgpKDFMszcrPcffg5uhwSgPCLD2WWxmq6isisz87tzT01tuGhg== + dependencies: + has-symbols "^1.0.2" + +is-typed-array@^1.1.10, is-typed-array@^1.1.3, is-typed-array@^1.1.9: + version "1.1.12" + resolved "https://registry.yarnpkg.com/is-typed-array/-/is-typed-array-1.1.12.tgz#d0bab5686ef4a76f7a73097b95470ab199c57d4a" + integrity sha512-Z14TF2JNG8Lss5/HMqt0//T9JeHXttXy5pH/DBU4vi98ozO2btxzq9MwYDZYnKwU8nRsz/+GVFVRDq3DkVuSPg== + dependencies: + which-typed-array "^1.1.11" + +is-unicode-supported@^0.1.0: + version "0.1.0" + resolved "https://registry.yarnpkg.com/is-unicode-supported/-/is-unicode-supported-0.1.0.tgz#3f26c76a809593b52bfa2ecb5710ed2779b522a7" + integrity sha512-knxG2q4UC3u8stRGyAVJCOdxFmv5DZiRcdlIaAQXAbSfJya+OhopNotLQrstBhququ4ZpuKbDc/8S6mgXgPFPw== + +is-weakmap@^2.0.1: + version "2.0.1" + resolved "https://registry.yarnpkg.com/is-weakmap/-/is-weakmap-2.0.1.tgz#5008b59bdc43b698201d18f62b37b2ca243e8cf2" + integrity sha512-NSBR4kH5oVj1Uwvv970ruUkCV7O1mzgVFO4/rev2cLRda9Tm9HrL70ZPut4rOHgY0FNrUu9BCbXA2sdQ+x0chA== + +is-weakref@^1.0.2: + version "1.0.2" + resolved "https://registry.yarnpkg.com/is-weakref/-/is-weakref-1.0.2.tgz#9529f383a9338205e89765e0392efc2f100f06f2" + integrity sha512-qctsuLZmIQ0+vSSMfoVvyFe2+GSEvnmZ2ezTup1SBse9+twCCeial6EEi3Nc2KFcf6+qz2FBPnjXsk8xhKSaPQ== + dependencies: + call-bind "^1.0.2" + +is-weakset@^2.0.1: + version "2.0.2" + resolved "https://registry.yarnpkg.com/is-weakset/-/is-weakset-2.0.2.tgz#4569d67a747a1ce5a994dfd4ef6dcea76e7c0a1d" + integrity sha512-t2yVvttHkQktwnNNmBQ98AhENLdPUTDTE21uPqAQ0ARwQfGeQKRVS0NNurH7bTf7RrvcVn1OOge45CnBeHCSmg== + dependencies: + call-bind "^1.0.2" + get-intrinsic "^1.1.1" + +is-what@^3.14.1: + version "3.14.1" + resolved "https://registry.yarnpkg.com/is-what/-/is-what-3.14.1.tgz#e1222f46ddda85dead0fd1c9df131760e77755c1" + integrity sha512-sNxgpk9793nzSs7bA6JQJGeIuRBQhAaNGG77kzYQgMkrID+lS6SlK07K5LaptscDlSaIgH+GPFzf+d75FVxozA== + +is-wsl@^2.2.0: + version "2.2.0" + resolved "https://registry.yarnpkg.com/is-wsl/-/is-wsl-2.2.0.tgz#74a4c76e77ca9fd3f932f290c17ea326cd157271" + integrity sha512-fKzAra0rGJUUBwGBgNkHZuToZcn+TtXHpeCgmkMJMMYx1sQDYaCSyjJBSCa2nH1DGm7s3n1oBnohoVTBaN7Lww== + dependencies: + is-docker "^2.0.0" + +isarray@^2.0.5: + version "2.0.5" + resolved "https://registry.yarnpkg.com/isarray/-/isarray-2.0.5.tgz#8af1e4c1221244cc62459faf38940d4e644a5723" + integrity sha512-xHjhDr3cNBK0BzdUJSPXZntQUx/mwMS5Rw4A7lPJ90XGAO6ISP/ePDNuo0vhqOZU+UD5JoodwCAAoZQd3FeAKw== + +isarray@~1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/isarray/-/isarray-1.0.0.tgz#bb935d48582cba168c06834957a54a3e07124f11" + integrity sha512-VLghIWNM6ELQzo7zwmcg0NmTVyWKYjvIeM83yjp0wRDTmUnrM678fQbcKBo6n2CJEF0szoG//ytg+TKla89ALQ== + +isexe@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/isexe/-/isexe-2.0.0.tgz#e8fbf374dc556ff8947a10dcb0572d633f2cfa10" + integrity sha512-RHxMLp9lnKHGHRng9QFhRCMbYAcVpn69smSGcq3f36xjgVVWThj4qqLbTLlq7Ssj8B+fIQ1EuCEGI2lKsyQeIw== + +isobject@^3.0.1: + version "3.0.1" + resolved "https://registry.yarnpkg.com/isobject/-/isobject-3.0.1.tgz#4e431e92b11a9731636aa1f9c8d1ccbcfdab78df" + integrity sha512-WhB9zCku7EGTj/HQQRz5aUQEUeoQZH2bWcltRErOpymJ4boYE6wL9Tbr23krRPSZ+C5zqNSrSw+Cc7sZZ4b7vg== + +istanbul-lib-coverage@^3.0.0, istanbul-lib-coverage@^3.2.0: + version "3.2.0" + resolved "https://registry.yarnpkg.com/istanbul-lib-coverage/-/istanbul-lib-coverage-3.2.0.tgz#189e7909d0a39fa5a3dfad5b03f71947770191d3" + integrity sha512-eOeJ5BHCmHYvQK7xt9GkdHuzuCGS1Y6g9Gvnx3Ym33fz/HpLRYxiS0wHNr+m/MBC8B647Xt608vCDEvhl9c6Mw== + +istanbul-lib-instrument@^5.0.4: + version "5.2.1" + resolved "https://registry.yarnpkg.com/istanbul-lib-instrument/-/istanbul-lib-instrument-5.2.1.tgz#d10c8885c2125574e1c231cacadf955675e1ce3d" + integrity sha512-pzqtp31nLv/XFOzXGuvhCb8qhjmTVo5vjVk19XE4CRlSWz0KoeJ3bw9XsA7nOp9YBf4qHjwBxkDzKcME/J29Yg== + dependencies: + "@babel/core" "^7.12.3" + "@babel/parser" "^7.14.7" + "@istanbuljs/schema" "^0.1.2" + istanbul-lib-coverage "^3.2.0" + semver "^6.3.0" + +istanbul-lib-report@^3.0.0: + version "3.0.1" + resolved "https://registry.yarnpkg.com/istanbul-lib-report/-/istanbul-lib-report-3.0.1.tgz#908305bac9a5bd175ac6a74489eafd0fc2445a7d" + integrity sha512-GCfE1mtsHGOELCU8e/Z7YWzpmybrx/+dSTfLrvY8qRmaY6zXTKWn6WQIjaAFw069icm6GVMNkgu0NzI4iPZUNw== + dependencies: + istanbul-lib-coverage "^3.0.0" + make-dir "^4.0.0" + supports-color "^7.1.0" + +istanbul-reports@^3.1.4: + version "3.1.6" + resolved "https://registry.yarnpkg.com/istanbul-reports/-/istanbul-reports-3.1.6.tgz#2544bcab4768154281a2f0870471902704ccaa1a" + integrity sha512-TLgnMkKg3iTDsQ9PbPTdpfAK2DzjF9mqUG7RMgcQl8oFjad8ob4laGxv5XV5U9MAfx8D6tSJiUyuAwzLicaxlg== + dependencies: + html-escaper "^2.0.0" + istanbul-lib-report "^3.0.0" + +iterator.prototype@^1.1.0: + version "1.1.1" + resolved "https://registry.yarnpkg.com/iterator.prototype/-/iterator.prototype-1.1.1.tgz#ab5b790e23ec00658f5974e032a2b05188bd3a5c" + integrity sha512-9E+nePc8C9cnQldmNl6bgpTY6zI4OPRZd97fhJ/iVZ1GifIUDVV5F6x1nEDqpe8KaMEZGT4xgrwKQDxXnjOIZQ== + dependencies: + define-properties "^1.2.0" + get-intrinsic "^1.2.1" + has-symbols "^1.0.3" + reflect.getprototypeof "^1.0.3" + +jackspeak@^2.0.3: + version "2.3.3" + resolved "https://registry.yarnpkg.com/jackspeak/-/jackspeak-2.3.3.tgz#95e4cbcc03b3eb357bf6bcce14a903fb3d1151e1" + integrity sha512-R2bUw+kVZFS/h1AZqBKrSgDmdmjApzgY0AlCPumopFiAlbUxE2gf+SCuBzQ0cP5hHmUmFYF5yw55T97Th5Kstg== + dependencies: + "@isaacs/cliui" "^8.0.2" + optionalDependencies: + "@pkgjs/parseargs" "^0.11.0" + +jake@^10.8.5: + version "10.8.7" + resolved "https://registry.yarnpkg.com/jake/-/jake-10.8.7.tgz#63a32821177940c33f356e0ba44ff9d34e1c7d8f" + integrity sha512-ZDi3aP+fG/LchyBzUM804VjddnwfSfsdeYkwt8NcbKRvo4rFkjhs456iLFn3k2ZUWvNe4i48WACDbza8fhq2+w== + dependencies: + async "^3.2.3" + chalk "^4.0.2" + filelist "^1.0.4" + minimatch "^3.1.2" + +jest-haste-map@^29.6.4: + version "29.6.4" + resolved "https://registry.yarnpkg.com/jest-haste-map/-/jest-haste-map-29.6.4.tgz#97143ce833829157ea7025204b08f9ace609b96a" + integrity sha512-12Ad+VNTDHxKf7k+M65sviyynRoZYuL1/GTuhEVb8RYsNSNln71nANRb/faSyWvx0j+gHcivChXHIoMJrGYjog== + dependencies: + "@jest/types" "^29.6.3" + "@types/graceful-fs" "^4.1.3" + "@types/node" "*" + anymatch "^3.0.3" + fb-watchman "^2.0.0" + graceful-fs "^4.2.9" + jest-regex-util "^29.6.3" + jest-util "^29.6.3" + jest-worker "^29.6.4" + micromatch "^4.0.4" + walker "^1.0.8" + optionalDependencies: + fsevents "^2.3.2" + +jest-mock@^27.0.6: + version "27.5.1" + resolved "https://registry.yarnpkg.com/jest-mock/-/jest-mock-27.5.1.tgz#19948336d49ef4d9c52021d34ac7b5f36ff967d6" + integrity sha512-K4jKbY1d4ENhbrG2zuPWaQBvDly+iZ2yAW+T1fATN78hc0sInwn7wZB8XtlNnvHug5RMwV897Xm4LqmPM4e2Og== + dependencies: + "@jest/types" "^27.5.1" + "@types/node" "*" + +jest-regex-util@^29.6.3: + version "29.6.3" + resolved "https://registry.yarnpkg.com/jest-regex-util/-/jest-regex-util-29.6.3.tgz#4a556d9c776af68e1c5f48194f4d0327d24e8a52" + integrity sha512-KJJBsRCyyLNWCNBOvZyRDnAIfUiRJ8v+hOBQYGn8gDyF3UegwiP4gwRR3/SDa42g1YbVycTidUF3rKjyLFDWbg== + +jest-util@^29.6.3: + version "29.6.3" + resolved "https://registry.yarnpkg.com/jest-util/-/jest-util-29.6.3.tgz#e15c3eac8716440d1ed076f09bc63ace1aebca63" + integrity sha512-QUjna/xSy4B32fzcKTSz1w7YYzgiHrjjJjevdRf61HYk998R5vVMMNmrHESYZVDS5DSWs+1srPLPKxXPkeSDOA== + dependencies: + "@jest/types" "^29.6.3" + "@types/node" "*" + chalk "^4.0.0" + ci-info "^3.2.0" + graceful-fs "^4.2.9" + picomatch "^2.2.3" + +jest-worker@^27.4.5: + version "27.5.1" + resolved "https://registry.yarnpkg.com/jest-worker/-/jest-worker-27.5.1.tgz#8d146f0900e8973b106b6f73cc1e9a8cb86f8db0" + integrity sha512-7vuh85V5cdDofPyxn58nrPjBktZo0u9x1g8WtjQol+jZDaE+fhN+cIvTj11GndBnMnyfrUOG1sZQxCdjKh+DKg== + dependencies: + "@types/node" "*" + merge-stream "^2.0.0" + supports-color "^8.0.0" + +jest-worker@^29.6.4: + version "29.6.4" + resolved "https://registry.yarnpkg.com/jest-worker/-/jest-worker-29.6.4.tgz#f34279f4afc33c872b470d4af21b281ac616abd3" + integrity sha512-6dpvFV4WjcWbDVGgHTWo/aupl8/LbBx2NSKfiwqf79xC/yeJjKHT1+StcKy/2KTmW16hE68ccKVOtXf+WZGz7Q== + dependencies: + "@types/node" "*" + jest-util "^29.6.3" + merge-stream "^2.0.0" + supports-color "^8.0.0" + +jiti@^1.18.2: + version "1.19.3" + resolved "https://registry.yarnpkg.com/jiti/-/jiti-1.19.3.tgz#ef554f76465b3c2b222dc077834a71f0d4a37569" + integrity sha512-5eEbBDQT/jF1xg6l36P+mWGGoH9Spuy0PCdSr2dtWRDGC6ph/w9ZCL4lmESW8f8F7MwT3XKescfP0wnZWAKL9w== + +"js-tokens@^3.0.0 || ^4.0.0", js-tokens@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/js-tokens/-/js-tokens-4.0.0.tgz#19203fb59991df98e3a287050d4647cdeaf32499" + integrity sha512-RdJUflcE3cUzKiMqQgsCu06FPu9UdIJO0beYbPhHN4k6apgJtifcoCtT9bcxOpYBtpD2kCM6Sbzg4CausW/PKQ== + +js-yaml@^3.13.1: + version "3.14.1" + resolved "https://registry.yarnpkg.com/js-yaml/-/js-yaml-3.14.1.tgz#dae812fdb3825fa306609a8717383c50c36a0537" + integrity sha512-okMH7OXXJ7YrN9Ok3/SXrnu4iX9yOk+25nqX4imS2npuvTYDmo/QEZoqwZkYaIDk3jVvBOTOIEgEhaLOynBS9g== + dependencies: + argparse "^1.0.7" + esprima "^4.0.0" + +js-yaml@^4.1.0: + version "4.1.0" + resolved "https://registry.yarnpkg.com/js-yaml/-/js-yaml-4.1.0.tgz#c1fb65f8f5017901cdd2c951864ba18458a10602" + integrity sha512-wpxZs9NoxZaJESJGIZTyDEaYpl0FKSA+FB9aJiyemKhMwkxQg63h4T1KJgUGHpTqPDNRcmmYLugrRjJlBtWvRA== + dependencies: + argparse "^2.0.1" + +jscodeshift@^0.14.0: + version "0.14.0" + resolved "https://registry.yarnpkg.com/jscodeshift/-/jscodeshift-0.14.0.tgz#7542e6715d6d2e8bde0b4e883f0ccea358b46881" + integrity sha512-7eCC1knD7bLUPuSCwXsMZUH51O8jIcoVyKtI6P0XM0IVzlGjckPy3FIwQlorzbN0Sg79oK+RlohN32Mqf/lrYA== + dependencies: + "@babel/core" "^7.13.16" + "@babel/parser" "^7.13.16" + "@babel/plugin-proposal-class-properties" "^7.13.0" + "@babel/plugin-proposal-nullish-coalescing-operator" "^7.13.8" + "@babel/plugin-proposal-optional-chaining" "^7.13.12" + "@babel/plugin-transform-modules-commonjs" "^7.13.8" + "@babel/preset-flow" "^7.13.13" + "@babel/preset-typescript" "^7.13.0" + "@babel/register" "^7.13.16" + babel-core "^7.0.0-bridge.0" + chalk "^4.1.2" + flow-parser "0.*" + graceful-fs "^4.2.4" + micromatch "^4.0.4" + neo-async "^2.5.0" + node-dir "^0.1.17" + recast "^0.21.0" + temp "^0.8.4" + write-file-atomic "^2.3.0" + +jsesc@^2.5.1: + version "2.5.2" + resolved "https://registry.yarnpkg.com/jsesc/-/jsesc-2.5.2.tgz#80564d2e483dacf6e8ef209650a67df3f0c283a4" + integrity sha512-OYu7XEzjkCQ3C5Ps3QIZsQfNpqoJyZZA99wd9aWd05NCtC5pWOkShK2mkL6HXQR6/Cy2lbNdPlZBpuQHXE63gA== + +jsesc@~0.5.0: + version "0.5.0" + resolved "https://registry.yarnpkg.com/jsesc/-/jsesc-0.5.0.tgz#e7dee66e35d6fc16f710fe91d5cf69f70f08911d" + integrity sha512-uZz5UnB7u4T9LvwmFqXii7pZSouaRPorGs5who1Ip7VO0wxanFvBL7GkM6dTHlgX+jhBApRetaWpnDabOeTcnA== + +json-buffer@3.0.1: + version "3.0.1" + resolved "https://registry.yarnpkg.com/json-buffer/-/json-buffer-3.0.1.tgz#9338802a30d3b6605fbe0613e094008ca8c05a13" + integrity sha512-4bV5BfR2mqfQTJm+V5tPPdf+ZpuhiIvTuAB5g8kcrXOZpTT/QwwVRWBywX1ozr6lEuPdbHxwaJlm9G6mI2sfSQ== + +json-parse-even-better-errors@^2.3.0, json-parse-even-better-errors@^2.3.1: + version "2.3.1" + resolved "https://registry.yarnpkg.com/json-parse-even-better-errors/-/json-parse-even-better-errors-2.3.1.tgz#7c47805a94319928e05777405dc12e1f7a4ee02d" + integrity sha512-xyFwyhro/JEof6Ghe2iz2NcXoj2sloNsWr/XsERDK/oiPCfaNhl5ONfp+jQdAZRQQ0IJWNzH9zIZF7li91kh2w== + +json-schema-traverse@^0.4.1: + version "0.4.1" + resolved "https://registry.yarnpkg.com/json-schema-traverse/-/json-schema-traverse-0.4.1.tgz#69f6a87d9513ab8bb8fe63bdb0979c448e684660" + integrity sha512-xbbCH5dCYU5T8LcEhhuh7HJ88HXuW3qsI3Y0zOZFKfZEHcpWiHU/Jxzk629Brsab/mMiHQti9wMP+845RPe3Vg== + +json-schema-traverse@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/json-schema-traverse/-/json-schema-traverse-1.0.0.tgz#ae7bcb3656ab77a73ba5c49bf654f38e6b6860e2" + integrity sha512-NM8/P9n3XjXhIZn1lLhkFaACTOURQXjWhV4BA/RnOv8xvgqtqpAX9IO4mRQxSx1Rlo4tqzeqb0sOlruaOy3dug== + +json-stable-stringify-without-jsonify@^1.0.1: + version "1.0.1" + resolved "https://registry.yarnpkg.com/json-stable-stringify-without-jsonify/-/json-stable-stringify-without-jsonify-1.0.1.tgz#9db7b59496ad3f3cfef30a75142d2d930ad72651" + integrity sha512-Bdboy+l7tA3OGW6FjyFHWkP5LuByj1Tk33Ljyq0axyzdk9//JSi2u3fP1QSmd1KNwq6VOKYGlAu87CisVir6Pw== + +json5@^1.0.2: + version "1.0.2" + resolved "https://registry.yarnpkg.com/json5/-/json5-1.0.2.tgz#63d98d60f21b313b77c4d6da18bfa69d80e1d593" + integrity sha512-g1MWMLBiz8FKi1e4w0UyVL3w+iJceWAFBAaBnnGKOpNa5f8TLktkbre1+s6oICydWAm+HRUGTmI+//xv2hvXYA== + dependencies: + minimist "^1.2.0" + +json5@^2.1.2, json5@^2.2.2, json5@^2.2.3: + version "2.2.3" + resolved "https://registry.yarnpkg.com/json5/-/json5-2.2.3.tgz#78cd6f1a19bdc12b73db5ad0c61efd66c1e29283" + integrity sha512-XmOWe7eyHYH14cLdVPoyg+GOH3rYX++KpzrylJwSW98t3Nk+U8XOl8FWKOgwtzdb8lXGf6zYwDUzeHMWfxasyg== + +jsonfile@^6.0.1: + version "6.1.0" + resolved "https://registry.yarnpkg.com/jsonfile/-/jsonfile-6.1.0.tgz#bc55b2634793c679ec6403094eb13698a6ec0aae" + integrity sha512-5dgndWOriYSm5cnYaJNhalLNDKOqFwyDB/rr1E9ZsGciGvKPs8R2xYGCacuf3z6K1YKDz182fd+fY3cn3pMqXQ== + dependencies: + universalify "^2.0.0" + optionalDependencies: + graceful-fs "^4.1.6" + +"jsx-ast-utils@^2.4.1 || ^3.0.0", jsx-ast-utils@^3.3.3: + version "3.3.5" + resolved "https://registry.yarnpkg.com/jsx-ast-utils/-/jsx-ast-utils-3.3.5.tgz#4766bd05a8e2a11af222becd19e15575e52a853a" + integrity sha512-ZZow9HBI5O6EPgSJLUb8n2NKgmVWTwCvHGwFuJlMjvLFqlGG6pjirPhtdsseaLZjSibD8eegzmYpUZwoIlj2cQ== + dependencies: + array-includes "^3.1.6" + array.prototype.flat "^1.3.1" + object.assign "^4.1.4" + object.values "^1.1.6" + +keyv@^4.5.3: + version "4.5.3" + resolved "https://registry.yarnpkg.com/keyv/-/keyv-4.5.3.tgz#00873d2b046df737963157bd04f294ca818c9c25" + integrity sha512-QCiSav9WaX1PgETJ+SpNnx2PRRapJ/oRSXM4VO5OGYGSjrxbKPVFVhB3l2OCbLCk329N8qyAtsJjSjvVBWzEug== + dependencies: + json-buffer "3.0.1" + +kind-of@^6.0.2: + version "6.0.3" + resolved "https://registry.yarnpkg.com/kind-of/-/kind-of-6.0.3.tgz#07c05034a6c349fa06e24fa35aa76db4580ce4dd" + integrity sha512-dcS1ul+9tmeD95T+x28/ehLgd9mENa3LsvDTtzm3vyBEO7RPptvAD+t44WVXaUjTBRcrpFeFlC8WCruUR456hw== + +kleur@^3.0.3: + version "3.0.3" + resolved "https://registry.yarnpkg.com/kleur/-/kleur-3.0.3.tgz#a79c9ecc86ee1ce3fa6206d1216c501f147fc07e" + integrity sha512-eTIzlVOSUR+JxdDFepEYcBMtZ9Qqdef+rnzWdRZuMbOywu5tO2w2N7rqjoANZ5k9vywhL6Br1VRjUIgTQx4E8w== + +klona@^2.0.4: + version "2.0.6" + resolved "https://registry.yarnpkg.com/klona/-/klona-2.0.6.tgz#85bffbf819c03b2f53270412420a4555ef882e22" + integrity sha512-dhG34DXATL5hSxJbIexCft8FChFXtmskoZYnoPWjXQuebWYCNkVeV3KkGegCK9CP1oswI/vQibS2GY7Em/sJJA== + +language-subtag-registry@~0.3.2: + version "0.3.22" + resolved "https://registry.yarnpkg.com/language-subtag-registry/-/language-subtag-registry-0.3.22.tgz#2e1500861b2e457eba7e7ae86877cbd08fa1fd1d" + integrity sha512-tN0MCzyWnoz/4nHS6uxdlFWoUZT7ABptwKPQ52Ea7URk6vll88bWBVhodtnlfEuCcKWNGoc+uGbw1cwa9IKh/w== + +language-tags@=1.0.5: + version "1.0.5" + resolved "https://registry.yarnpkg.com/language-tags/-/language-tags-1.0.5.tgz#d321dbc4da30ba8bf3024e040fa5c14661f9193a" + integrity sha512-qJhlO9cGXi6hBGKoxEG/sKZDAHD5Hnu9Hs4WbOY3pCWXDhw0N8x1NenNzm2EnNLkLkk7J2SdxAkDSbb6ftT+UQ== + dependencies: + language-subtag-registry "~0.3.2" + +lazy-universal-dotenv@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/lazy-universal-dotenv/-/lazy-universal-dotenv-4.0.0.tgz#0b220c264e89a042a37181a4928cdd298af73422" + integrity sha512-aXpZJRnTkpK6gQ/z4nk+ZBLd/Qdp118cvPruLSIQzQNRhKwEcdXCOzXuF55VDqIiuAaY3UGZ10DJtvZzDcvsxg== + dependencies: + app-root-dir "^1.0.2" + dotenv "^16.0.0" + dotenv-expand "^10.0.0" + +less-loader@^11.1.0: + version "11.1.3" + resolved "https://registry.yarnpkg.com/less-loader/-/less-loader-11.1.3.tgz#1bb62d6ca9bf00a177c02793b54baac40f9be694" + integrity sha512-A5b7O8dH9xpxvkosNrP0dFp2i/dISOJa9WwGF3WJflfqIERE2ybxh1BFDj5CovC2+jCE4M354mk90hN6ziXlVw== + +less@^4.1.2: + version "4.2.0" + resolved "https://registry.yarnpkg.com/less/-/less-4.2.0.tgz#cbefbfaa14a4cd388e2099b2b51f956e1465c450" + integrity sha512-P3b3HJDBtSzsXUl0im2L7gTO5Ubg8mEN6G8qoTS77iXxXX4Hvu4Qj540PZDvQ8V6DmX6iXo98k7Md0Cm1PrLaA== + dependencies: + copy-anything "^2.0.1" + parse-node-version "^1.0.1" + tslib "^2.3.0" + optionalDependencies: + errno "^0.1.1" + graceful-fs "^4.1.2" + image-size "~0.5.0" + make-dir "^2.1.0" + mime "^1.4.1" + needle "^3.1.0" + source-map "~0.6.0" + +leven@^3.1.0: + version "3.1.0" + resolved "https://registry.yarnpkg.com/leven/-/leven-3.1.0.tgz#77891de834064cccba82ae7842bb6b14a13ed7f2" + integrity sha512-qsda+H8jTaUaN/x5vzW2rzc+8Rw4TAQ/4KjB46IwK5VH+IlVeeeje/EoZRpiXvIqjFgK84QffqPztGI3VBLG1A== + +levn@^0.4.1: + version "0.4.1" + resolved "https://registry.yarnpkg.com/levn/-/levn-0.4.1.tgz#ae4562c007473b932a6200d403268dd2fffc6ade" + integrity sha512-+bT2uH4E5LGE7h/n3evcS/sQlJXCpIp6ym8OWJ5eV6+67Dsql/LaaT7qJBAt2rzfoa/5QBGBhxDix1dMt2kQKQ== + dependencies: + prelude-ls "^1.2.1" + type-check "~0.4.0" + +lilconfig@^2.0.5, lilconfig@^2.1.0: + version "2.1.0" + resolved "https://registry.yarnpkg.com/lilconfig/-/lilconfig-2.1.0.tgz#78e23ac89ebb7e1bfbf25b18043de756548e7f52" + integrity sha512-utWOt/GHzuUxnLKxB6dk81RoOeoNeHgbrXiuGk4yyF5qlRz+iIVWu56E2fqGHFrXz0QNUhLB/8nKqvRH66JKGQ== + +lines-and-columns@^1.1.6: + version "1.2.4" + resolved "https://registry.yarnpkg.com/lines-and-columns/-/lines-and-columns-1.2.4.tgz#eca284f75d2965079309dc0ad9255abb2ebc1632" + integrity sha512-7ylylesZQ/PV29jhEDl3Ufjo6ZX7gCqJr5F7PKrqc93v7fzSymt1BpwEU8nAUXs8qzzvqhbjhK5QZg6Mt/HkBg== + +loader-runner@^4.2.0: + version "4.3.0" + resolved "https://registry.yarnpkg.com/loader-runner/-/loader-runner-4.3.0.tgz#c1b4a163b99f614830353b16755e7149ac2314e1" + integrity sha512-3R/1M+yS3j5ou80Me59j7F9IMs4PXs3VqRrm0TU3AbKPxlmpoY1TNscJV/oGJXo8qCatFGTfDbY6W6ipGOYXfg== + +loader-utils@^2.0.0, loader-utils@^2.0.4: + version "2.0.4" + resolved "https://registry.yarnpkg.com/loader-utils/-/loader-utils-2.0.4.tgz#8b5cb38b5c34a9a018ee1fc0e6a066d1dfcc528c" + integrity sha512-xXqpXoINfFhgua9xiqD8fPFHgkoq1mmmpE92WlDbm9rNRd/EbRb+Gqf908T2DMfuHjjJlksiK2RbHVOdD/MqSw== + dependencies: + big.js "^5.2.2" + emojis-list "^3.0.0" + json5 "^2.1.2" + +loader-utils@^3.2.0: + version "3.2.1" + resolved "https://registry.yarnpkg.com/loader-utils/-/loader-utils-3.2.1.tgz#4fb104b599daafd82ef3e1a41fb9265f87e1f576" + integrity sha512-ZvFw1KWS3GVyYBYb7qkmRM/WwL2TQQBxgCK62rlvm4WpVQ23Nb4tYjApUlfjrEGvOs7KHEsmyUn75OHZrJMWPw== + +locate-path@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/locate-path/-/locate-path-3.0.0.tgz#dbec3b3ab759758071b58fe59fc41871af21400e" + integrity sha512-7AO748wWnIhNqAuaty2ZWHkQHRSNfPVIsPIfwEOWO22AmaoVrWavlOcMR5nzTLNYvp36X220/maaRsrec1G65A== + dependencies: + p-locate "^3.0.0" + path-exists "^3.0.0" + +locate-path@^5.0.0: + version "5.0.0" + resolved "https://registry.yarnpkg.com/locate-path/-/locate-path-5.0.0.tgz#1afba396afd676a6d42504d0a67a3a7eb9f62aa0" + integrity sha512-t7hw9pI+WvuwNJXwk5zVHpyhIqzg2qTlklJOf0mVxGSbe3Fp2VieZcduNYjaLDoy6p9uGpQEGWG87WpMKlNq8g== + dependencies: + p-locate "^4.1.0" + +locate-path@^6.0.0: + version "6.0.0" + resolved "https://registry.yarnpkg.com/locate-path/-/locate-path-6.0.0.tgz#55321eb309febbc59c4801d931a72452a681d286" + integrity sha512-iPZK6eYjbxRu3uB4/WZ3EsEIMJFMqAoopl3R+zuq0UjcAm/MO6KCweDgPfP3elTztoKP3KtnVHxTn2NHBSDVUw== + dependencies: + p-locate "^5.0.0" + +locate-path@^7.1.0: + version "7.2.0" + resolved "https://registry.yarnpkg.com/locate-path/-/locate-path-7.2.0.tgz#69cb1779bd90b35ab1e771e1f2f89a202c2a8a8a" + integrity sha512-gvVijfZvn7R+2qyPX8mAuKcFGDf6Nc61GdvGafQsHL0sBIxfKzA+usWn4GFC/bk+QdwPUD4kWFJLhElipq+0VA== + dependencies: + p-locate "^6.0.0" + +lodash.camelcase@^4.3.0: + version "4.3.0" + resolved "https://registry.yarnpkg.com/lodash.camelcase/-/lodash.camelcase-4.3.0.tgz#b28aa6288a2b9fc651035c7711f65ab6190331a6" + integrity sha512-TwuEnCnxbc3rAvhf/LbG7tJUDzhqXyFnv3dtzLOPgCG/hODL7WFnsbwktkD7yUV0RrreP/l1PALq/YSg6VvjlA== + +lodash.debounce@^4.0.8: + version "4.0.8" + resolved "https://registry.yarnpkg.com/lodash.debounce/-/lodash.debounce-4.0.8.tgz#82d79bff30a67c4005ffd5e2515300ad9ca4d7af" + integrity sha512-FT1yDzDYEoYWhnSGnpE/4Kj1fLZkDFyqRb7fNt6FdYOSxlUWAtp42Eh6Wb0rGIv/m9Bgo7x4GhQbm5Ys4SG5ow== + +lodash.merge@^4.6.2: + version "4.6.2" + resolved "https://registry.yarnpkg.com/lodash.merge/-/lodash.merge-4.6.2.tgz#558aa53b43b661e1925a0afdfa36a9a1085fe57a" + integrity sha512-0KpjqXRVvrYyCsX1swR/XTK0va6VQkQM6MNo7PqW77ByjAhoARA8EfrP1N4+KlKj8YS0ZUCtRT/YUuhyYDujIQ== + +lodash@^4.17.15, lodash@^4.17.19, lodash@^4.17.20, lodash@^4.17.21: + version "4.17.21" + resolved "https://registry.yarnpkg.com/lodash/-/lodash-4.17.21.tgz#679591c564c3bffaae8454cf0b3df370c3d6911c" + integrity sha512-v2kDEe57lecTulaDIuNTPy3Ry4gLGJ6Z1O3vE1krgXZNrsQ+LFTGHVxVjcXPs17LhbZVGedAJv8XZ1tvj5FvSg== + +log-symbols@^4.1.0: + version "4.1.0" + resolved "https://registry.yarnpkg.com/log-symbols/-/log-symbols-4.1.0.tgz#3fbdbb95b4683ac9fc785111e792e558d4abd503" + integrity sha512-8XPvpAA8uyhfteu8pIvQxpJZ7SYYdpUivZpGy6sFsBuKRY/7rQGavedeB8aK+Zkyq6upMFVL/9AW6vOYzfRyLg== + dependencies: + chalk "^4.1.0" + is-unicode-supported "^0.1.0" + +long@^5.0.0, long@^5.2.3: + version "5.2.3" + resolved "https://registry.yarnpkg.com/long/-/long-5.2.3.tgz#a3ba97f3877cf1d778eccbcb048525ebb77499e1" + integrity sha512-lcHwpNoggQTObv5apGNCTdJrO69eHOZMi4BNC+rTLER8iHAqGrUVeLh/irVIM7zTw2bOXA8T6uNPeujwOLg/2Q== + +loose-envify@^1.0.0, loose-envify@^1.1.0, loose-envify@^1.4.0: + version "1.4.0" + resolved "https://registry.yarnpkg.com/loose-envify/-/loose-envify-1.4.0.tgz#71ee51fa7be4caec1a63839f7e682d8132d30caf" + integrity sha512-lyuxPGr/Wfhrlem2CL/UcnUc1zcqKAImBDzukY7Y5F/yQiNdko6+fRLevlw1HgMySw7f611UIY408EtxRSoK3Q== + dependencies: + js-tokens "^3.0.0 || ^4.0.0" + +lower-case@^2.0.2: + version "2.0.2" + resolved "https://registry.yarnpkg.com/lower-case/-/lower-case-2.0.2.tgz#6fa237c63dbdc4a82ca0fd882e4722dc5e634e28" + integrity sha512-7fm3l3NAF9WfN6W3JOmf5drwpVqX78JtoGJ3A6W0a6ZnldM41w2fV5D490psKFTpMds8TJse/eHLFFsNHHjHgg== + dependencies: + tslib "^2.0.3" + +lru-cache@^5.1.1: + version "5.1.1" + resolved "https://registry.yarnpkg.com/lru-cache/-/lru-cache-5.1.1.tgz#1da27e6710271947695daf6848e847f01d84b920" + integrity sha512-KpNARQA3Iwv+jTA0utUVVbrh+Jlrr1Fv0e56GGzAFOXN7dk/FviaDW8LHmK52DlcH4WP2n6gI8vN1aesBFgo9w== + dependencies: + yallist "^3.0.2" + +lru-cache@^6.0.0: + version "6.0.0" + resolved "https://registry.yarnpkg.com/lru-cache/-/lru-cache-6.0.0.tgz#6d6fe6570ebd96aaf90fcad1dafa3b2566db3a94" + integrity sha512-Jo6dJ04CmSjuznwJSS3pUeWmd/H0ffTlkXXgwZi+eq1UCmqQwCh+eLsYOYCwY991i2Fah4h1BEMCx4qThGbsiA== + dependencies: + yallist "^4.0.0" + +"lru-cache@^9.1.1 || ^10.0.0": + version "10.0.1" + resolved "https://registry.yarnpkg.com/lru-cache/-/lru-cache-10.0.1.tgz#0a3be479df549cca0e5d693ac402ff19537a6b7a" + integrity sha512-IJ4uwUTi2qCccrioU6g9g/5rvvVl13bsdczUUcqbciD9iLr095yj8DQKdObriEvuNSx325N1rV1O0sJFszx75g== + +lz-string@^1.5.0: + version "1.5.0" + resolved "https://registry.yarnpkg.com/lz-string/-/lz-string-1.5.0.tgz#c1ab50f77887b712621201ba9fd4e3a6ed099941" + integrity sha512-h5bgJWpxJNswbU7qCrV0tIKQCaS3blPDrqKWx+QxzuzL1zGUzij9XCWLrSLsJPu5t+eWA/ycetzYAO5IOMcWAQ== + +make-dir@^2.0.0, make-dir@^2.1.0: + version "2.1.0" + resolved "https://registry.yarnpkg.com/make-dir/-/make-dir-2.1.0.tgz#5f0310e18b8be898cc07009295a30ae41e91e6f5" + integrity sha512-LS9X+dc8KLxXCb8dni79fLIIUA5VyZoyjSMCwTluaXA0o27cCK0bhXkpgw+sTXVpPy/lSO57ilRixqk0vDmtRA== + dependencies: + pify "^4.0.1" + semver "^5.6.0" + +make-dir@^3.0.2: + version "3.1.0" + resolved "https://registry.yarnpkg.com/make-dir/-/make-dir-3.1.0.tgz#415e967046b3a7f1d185277d84aa58203726a13f" + integrity sha512-g3FeP20LNwhALb/6Cz6Dd4F2ngze0jz7tbzrD2wAV+o9FeNHe4rL+yK2md0J/fiSf1sa1ADhXqi5+oVwOM/eGw== + dependencies: + semver "^6.0.0" + +make-dir@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/make-dir/-/make-dir-4.0.0.tgz#c3c2307a771277cd9638305f915c29ae741b614e" + integrity sha512-hXdUTZYIVOt1Ex//jAQi+wTZZpUpwBj/0QsOzqegb3rGMMeJiSEu5xLHnYfBrRV4RH2+OCSOO95Is/7x1WJ4bw== + dependencies: + semver "^7.5.3" + +makeerror@1.0.12: + version "1.0.12" + resolved "https://registry.yarnpkg.com/makeerror/-/makeerror-1.0.12.tgz#3e5dd2079a82e812e983cc6610c4a2cb0eaa801a" + integrity sha512-JmqCvUhmt43madlpFzG4BQzG2Z3m6tvQDNKdClZnO3VbIudJYmxsT0FNJMeiB2+JTSlTQTSbU8QdesVmwJcmLg== + dependencies: + tmpl "1.0.5" + +map-or-similar@^1.5.0: + version "1.5.0" + resolved "https://registry.yarnpkg.com/map-or-similar/-/map-or-similar-1.5.0.tgz#6de2653174adfb5d9edc33c69d3e92a1b76faf08" + integrity sha512-0aF7ZmVon1igznGI4VS30yugpduQW3y3GkcgGJOp7d8x8QrizhigUxjI/m2UojsXXto+jLAH3KSz+xOJTiORjg== + +markdown-to-jsx@^7.1.8: + version "7.3.2" + resolved "https://registry.yarnpkg.com/markdown-to-jsx/-/markdown-to-jsx-7.3.2.tgz#f286b4d112dad3028acc1e77dfe1f653b347e131" + integrity sha512-B+28F5ucp83aQm+OxNrPkS8z0tMKaeHiy0lHJs3LqCyDQFtWuenaIrkaVTgAm1pf1AU85LXltva86hlaT17i8Q== + +material-symbols@0.11.0: + version "0.11.0" + resolved "https://registry.yarnpkg.com/material-symbols/-/material-symbols-0.11.0.tgz#b3bfa073946c61098a59108fd38a6c5e60efb8b0" + integrity sha512-YJ7EyB7KMZFPQAKi0x1A+tH4zXEYw8FSGy1aAheqLm/ELzdotKwYe1hXUpmMcz87g7OoqikDeKEjRB+uGv9LrQ== + +md5.js@^1.3.4: + version "1.3.5" + resolved "https://registry.yarnpkg.com/md5.js/-/md5.js-1.3.5.tgz#b5d07b8e3216e3e27cd728d72f70d1e6a342005f" + integrity sha512-xitP+WxNPcTTOgnTJcrhM0xvdPepipPSf3I8EIpGKeFLjt3PlJLIDG3u8EX53ZIubkb+5U2+3rELYpEhHhzdkg== + dependencies: + hash-base "^3.0.0" + inherits "^2.0.1" + safe-buffer "^5.1.2" + +mdast-util-definitions@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/mdast-util-definitions/-/mdast-util-definitions-4.0.0.tgz#c5c1a84db799173b4dcf7643cda999e440c24db2" + integrity sha512-k8AJ6aNnUkB7IE+5azR9h81O5EQ/cTDXtWdMq9Kk5KcEW/8ritU5CeLg/9HhOC++nALHBlaogJ5jz0Ybk3kPMQ== + dependencies: + unist-util-visit "^2.0.0" + +mdast-util-to-string@^1.0.0: + version "1.1.0" + resolved "https://registry.yarnpkg.com/mdast-util-to-string/-/mdast-util-to-string-1.1.0.tgz#27055500103f51637bd07d01da01eb1967a43527" + integrity sha512-jVU0Nr2B9X3MU4tSK7JP1CMkSvOj7X5l/GboG1tKRw52lLF1x2Ju92Ms9tNetCcbfX3hzlM73zYo2NKkWSfF/A== + +media-typer@0.3.0: + version "0.3.0" + resolved "https://registry.yarnpkg.com/media-typer/-/media-typer-0.3.0.tgz#8710d7af0aa626f8fffa1ce00168545263255748" + integrity sha512-dq+qelQ9akHpcOl/gUVRTxVIOkAJ1wR3QAvb4RsVjS8oVoFjDGTc679wJYmUmknUF5HwMLOgb5O+a3KxfWapPQ== + +memfs@^3.4.1, memfs@^3.4.12: + version "3.6.0" + resolved "https://registry.yarnpkg.com/memfs/-/memfs-3.6.0.tgz#d7a2110f86f79dd950a8b6df6d57bc984aa185f6" + integrity sha512-EGowvkkgbMcIChjMTMkESFDbZeSh8xZ7kNSF0hAiAN4Jh6jgHCRS0Ga/+C8y6Au+oqpezRHCfPsmJ2+DwAgiwQ== + dependencies: + fs-monkey "^1.0.4" + +memoizerific@^1.11.3: + version "1.11.3" + resolved "https://registry.yarnpkg.com/memoizerific/-/memoizerific-1.11.3.tgz#7c87a4646444c32d75438570905f2dbd1b1a805a" + integrity sha512-/EuHYwAPdLtXwAwSZkh/Gutery6pD2KYd44oQLhAvQp/50mpyduZh8Q7PYHXTCJ+wuXxt7oij2LXyIJOOYFPog== + dependencies: + map-or-similar "^1.5.0" + +merge-descriptors@1.0.1: + version "1.0.1" + resolved "https://registry.yarnpkg.com/merge-descriptors/-/merge-descriptors-1.0.1.tgz#b00aaa556dd8b44568150ec9d1b953f3f90cbb61" + integrity sha512-cCi6g3/Zr1iqQi6ySbseM1Xvooa98N0w31jzUYrXPX2xqObmFGHJ0tQ5u74H3mVh7wLouTseZyYIq39g8cNp1w== + +merge-stream@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/merge-stream/-/merge-stream-2.0.0.tgz#52823629a14dd00c9770fb6ad47dc6310f2c1f60" + integrity sha512-abv/qOcuPfk3URPfDzmZU1LKmuw8kT+0nIHvKrKgFrwifol/doWcdA4ZqsWQ8ENrFKkd67Mfpo/LovbIUsbt3w== + +merge2@^1.3.0, merge2@^1.4.1: + version "1.4.1" + resolved "https://registry.yarnpkg.com/merge2/-/merge2-1.4.1.tgz#4368892f885e907455a6fd7dc55c0c9d404990ae" + integrity sha512-8q7VEgMJW4J8tcfVPy8g09NcQwZdbwFEqhe/WZkoIzjn/3TGDwtOCYtXGxA3O8tPzpczCCDgv+P2P5y00ZJOOg== + +methods@~1.1.2: + version "1.1.2" + resolved "https://registry.yarnpkg.com/methods/-/methods-1.1.2.tgz#5529a4d67654134edcc5266656835b0f851afcee" + integrity sha512-iclAHeNqNm68zFtnZ0e+1L2yUIdvzNoauKU4WBA3VvH/vPFieF7qfRlwUZU+DA9P9bPXIS90ulxoUoCH23sV2w== + +micromatch@^4.0.2, micromatch@^4.0.4, micromatch@^4.0.5: + version "4.0.5" + resolved "https://registry.yarnpkg.com/micromatch/-/micromatch-4.0.5.tgz#bc8999a7cbbf77cdc89f132f6e467051b49090c6" + integrity sha512-DMy+ERcEW2q8Z2Po+WNXuw3c5YaUSFjAO5GsJqfEl7UjvtIuFKO6ZrKvcItdy98dwFI2N1tg3zNIdKaQT+aNdA== + dependencies: + braces "^3.0.2" + picomatch "^2.3.1" + +miller-rabin@^4.0.0: + version "4.0.1" + resolved "https://registry.yarnpkg.com/miller-rabin/-/miller-rabin-4.0.1.tgz#f080351c865b0dc562a8462966daa53543c78a4d" + integrity sha512-115fLhvZVqWwHPbClyntxEVfVDfl9DLLTuJvq3g2O/Oxi8AiNouAHvDSzHS0viUJc+V5vm3eq91Xwqn9dp4jRA== + dependencies: + bn.js "^4.0.0" + brorand "^1.0.1" + +mime-db@1.52.0, "mime-db@>= 1.43.0 < 2": + version "1.52.0" + resolved "https://registry.yarnpkg.com/mime-db/-/mime-db-1.52.0.tgz#bbabcdc02859f4987301c856e3387ce5ec43bf70" + integrity sha512-sPU4uV7dYlvtWJxwwxHD0PuihVNiE7TyAbQ5SWxDCB9mUYvOgroQOwYQQOKPJ8CIbE+1ETVlOoK1UC2nU3gYvg== + +mime-types@^2.1.12, mime-types@^2.1.25, mime-types@^2.1.27, mime-types@^2.1.31, mime-types@~2.1.24, mime-types@~2.1.34: + version "2.1.35" + resolved "https://registry.yarnpkg.com/mime-types/-/mime-types-2.1.35.tgz#381a871b62a734450660ae3deee44813f70d959a" + integrity sha512-ZDY+bPm5zTTF+YpCrAU9nK0UgICYPT0QtT1NZWFv4s++TNkcgVaT0g6+4R2uI4MjQjzysHB1zxuWL50hzaeXiw== + dependencies: + mime-db "1.52.0" + +mime@1.6.0, mime@^1.4.1: + version "1.6.0" + resolved "https://registry.yarnpkg.com/mime/-/mime-1.6.0.tgz#32cd9e5c64553bd58d19a568af452acff04981b1" + integrity sha512-x0Vn8spI+wuJ1O6S7gnbaQg8Pxh4NNHb7KSINmEWKiPE4RKOplvijn+NkmYmmRgP68mc70j2EbeTFRsrswaQeg== + +mime@^2.0.3: + version "2.6.0" + resolved "https://registry.yarnpkg.com/mime/-/mime-2.6.0.tgz#a2a682a95cd4d0cb1d6257e28f83da7e35800367" + integrity sha512-USPkMeET31rOMiarsBNIHZKLGgvKc/LrjofAnBlOttf5ajRvqiRA8QsenbcooctK6d6Ts6aqZXBA+XbkKthiQg== + +mimic-fn@^2.1.0: + version "2.1.0" + resolved "https://registry.yarnpkg.com/mimic-fn/-/mimic-fn-2.1.0.tgz#7ed2c2ccccaf84d3ffcb7a69b57711fc2083401b" + integrity sha512-OqbOk5oEQeAZ8WXWydlu9HJjz9WVdEIvamMCcXmuqUYjTknH/sqsWvhQ3vgwKFRR1HpjvNBKQ37nbJgYzGqGcg== + +min-indent@^1.0.0: + version "1.0.1" + resolved "https://registry.yarnpkg.com/min-indent/-/min-indent-1.0.1.tgz#a63f681673b30571fbe8bc25686ae746eefa9869" + integrity sha512-I9jwMn07Sy/IwOj3zVkVik2JTvgpaykDZEigL6Rx6N9LbMywwUSMtxET+7lVoDLLd3O3IXwJwvuuns8UB/HeAg== + +minimalistic-assert@^1.0.0, minimalistic-assert@^1.0.1: + version "1.0.1" + resolved "https://registry.yarnpkg.com/minimalistic-assert/-/minimalistic-assert-1.0.1.tgz#2e194de044626d4a10e7f7fbc00ce73e83e4d5c7" + integrity sha512-UtJcAD4yEaGtjPezWuO9wC4nwUnVH/8/Im3yEHQP4b67cXlD/Qr9hdITCU1xDbSEXg2XKNaP8jsReV7vQd00/A== + +minimalistic-crypto-utils@^1.0.1: + version "1.0.1" + resolved "https://registry.yarnpkg.com/minimalistic-crypto-utils/-/minimalistic-crypto-utils-1.0.1.tgz#f6c00c1c0b082246e5c4d99dfb8c7c083b2b582a" + integrity sha512-JIYlbt6g8i5jKfJ3xz7rF0LXmv2TkDxBLUkiBeZ7bAx4GnnNMr8xFpGnOxn6GhTEHx3SjRrZEoU+j04prX1ktg== + +minimatch@^3.0.2, minimatch@^3.0.4, minimatch@^3.0.5, minimatch@^3.1.1, minimatch@^3.1.2: + version "3.1.2" + resolved "https://registry.yarnpkg.com/minimatch/-/minimatch-3.1.2.tgz#19cd194bfd3e428f049a70817c038d89ab4be35b" + integrity sha512-J7p63hRiAjw1NDEww1W7i37+ByIrOWO5XQQAzZ3VOcL0PNybwpfmV/N05zFAzwQ9USyEcX6t3UO+K5aqBQOIHw== + dependencies: + brace-expansion "^1.1.7" + +minimatch@^5.0.1: + version "5.1.6" + resolved "https://registry.yarnpkg.com/minimatch/-/minimatch-5.1.6.tgz#1cfcb8cf5522ea69952cd2af95ae09477f122a96" + integrity sha512-lKwV/1brpG6mBUFHtb7NUmtABCb2WZZmm2wNiOA5hAb8VdCS4B3dtMWyvcoViccwAW/COERjXLt0zP1zXUN26g== + dependencies: + brace-expansion "^2.0.1" + +minimatch@^9.0.1: + version "9.0.3" + resolved "https://registry.yarnpkg.com/minimatch/-/minimatch-9.0.3.tgz#a6e00c3de44c3a542bfaae70abfc22420a6da825" + integrity sha512-RHiac9mvaRw0x3AYRgDC1CxAP7HTcNrrECeA8YYJeWnpo+2Q5CegtZjaotWTWxDG3UeGA1coE05iH1mPjT/2mg== + dependencies: + brace-expansion "^2.0.1" + +minimist@^1.2.0, minimist@^1.2.5, minimist@^1.2.6: + version "1.2.8" + resolved "https://registry.yarnpkg.com/minimist/-/minimist-1.2.8.tgz#c1a464e7693302e082a075cee0c057741ac4772c" + integrity sha512-2yyAR8qBkN3YuheJanUpWC5U3bb5osDywNB8RzDVlDwDHbocAJveqqj1u8+SVD7jkWT4yvsHCpWqqWqAxb0zCA== + +minipass@^3.0.0: + version "3.3.6" + resolved "https://registry.yarnpkg.com/minipass/-/minipass-3.3.6.tgz#7bba384db3a1520d18c9c0e5251c3444e95dd94a" + integrity sha512-DxiNidxSEK+tHG6zOIklvNOwm3hvCrbUrdtzY74U6HKTJxvIDfOUL5W5P2Ghd3DTkhhKPYGqeNUIh5qcM4YBfw== + dependencies: + yallist "^4.0.0" + +minipass@^5.0.0: + version "5.0.0" + resolved "https://registry.yarnpkg.com/minipass/-/minipass-5.0.0.tgz#3e9788ffb90b694a5d0ec94479a45b5d8738133d" + integrity sha512-3FnjYuehv9k6ovOEbyOswadCDPX1piCfhV8ncmYtHOjuPwylVWsghTLo7rabjC3Rx5xD4HDx8Wm1xnMF7S5qFQ== + +"minipass@^5.0.0 || ^6.0.2 || ^7.0.0": + version "7.0.3" + resolved "https://registry.yarnpkg.com/minipass/-/minipass-7.0.3.tgz#05ea638da44e475037ed94d1c7efcc76a25e1974" + integrity sha512-LhbbwCfz3vsb12j/WkWQPZfKTsgqIe1Nf/ti1pKjYESGLHIVjWU96G9/ljLH4F9mWNVhlQOm0VySdAWzf05dpg== + +minizlib@^2.1.1: + version "2.1.2" + resolved "https://registry.yarnpkg.com/minizlib/-/minizlib-2.1.2.tgz#e90d3466ba209b932451508a11ce3d3632145931" + integrity sha512-bAxsR8BVfj60DWXHE3u30oHzfl4G7khkSuPW+qvpd7jFRHm7dLxOjUk1EHACJ/hxLY8phGJ0YhYHZo7jil7Qdg== + dependencies: + minipass "^3.0.0" + yallist "^4.0.0" + +mkdirp-classic@^0.5.2: + version "0.5.3" + resolved "https://registry.yarnpkg.com/mkdirp-classic/-/mkdirp-classic-0.5.3.tgz#fa10c9115cc6d8865be221ba47ee9bed78601113" + integrity sha512-gKLcREMhtuZRwRAfqP3RFW+TK4JqApVBtOIftVgjuABpAtpxhPGaDcfvbhNvD0B8iD1oUr/txX35NjcaY6Ns/A== + +mkdirp@^0.5.4: + version "0.5.6" + resolved "https://registry.yarnpkg.com/mkdirp/-/mkdirp-0.5.6.tgz#7def03d2432dcae4ba1d611445c48396062255f6" + integrity sha512-FP+p8RB8OWpF3YZBCrP5gtADmtXApB5AMLn+vdyA+PyxCjrCs00mjyUozssO33cwDeT3wNGdLxJ5M//YqtHAJw== + dependencies: + minimist "^1.2.6" + +mkdirp@^1.0.3: + version "1.0.4" + resolved "https://registry.yarnpkg.com/mkdirp/-/mkdirp-1.0.4.tgz#3eb5ed62622756d79a5f0e2a221dfebad75c2f7e" + integrity sha512-vVqVZQyf3WLx2Shd0qJ9xuvqgAyKPLAiqITEtqW0oIUjzo3PePDd6fW9iFz30ef7Ysp/oiWqbhszeGWW2T6Gzw== + +mri@^1.2.0: + version "1.2.0" + resolved "https://registry.yarnpkg.com/mri/-/mri-1.2.0.tgz#6721480fec2a11a4889861115a48b6cbe7cc8f0b" + integrity sha512-tzzskb3bG8LvYGFF/mDTpq3jpI6Q9wc3LEmBaghu+DdCssd1FakN7Bc0hVNmEyGq1bq3RgfkCb3cmQLpNPOroA== + +ms@2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/ms/-/ms-2.0.0.tgz#5608aeadfc00be6c2901df5f9861788de0d597c8" + integrity sha512-Tpp60P6IUJDTuOq/5Z8cdskzJujfwqfOTkrwIwj7IRISpnkJnT6SyJ4PCPnGMoFjC9ddhal5KVIYtAt97ix05A== + +ms@2.1.1: + version "2.1.1" + resolved "https://registry.yarnpkg.com/ms/-/ms-2.1.1.tgz#30a5864eb3ebb0a66f2ebe6d727af06a09d86e0a" + integrity sha512-tgp+dl5cGk28utYktBsrFqA7HKgrhgPsg6Z/EfhWI4gl1Hwq8B/GmY/0oXZ6nF8hDVesS/FpnYaD/kOWhYQvyg== + +ms@2.1.2: + version "2.1.2" + resolved "https://registry.yarnpkg.com/ms/-/ms-2.1.2.tgz#d09d1f357b443f493382a8eb3ccd183872ae6009" + integrity sha512-sGkPx+VjMtmA6MX27oA4FBFELFCZZ4S4XqeGOXCv68tT+jb3vk/RyaKWP0PTKyWtmLSM0b+adUTEvbs1PEaH2w== + +ms@2.1.3, ms@^2.1.1: + version "2.1.3" + resolved "https://registry.yarnpkg.com/ms/-/ms-2.1.3.tgz#574c8138ce1d2b5861f0b44579dbadd60c6615b2" + integrity sha512-6FlzubTLZG3J2a/NVCAleEhjzq5oxgHyaCU9yYXvcLsvoVaHJq/s5xXI6/XXP6tz7R9xAOtHnSO/tXtF3WRTlA== + +mz@^2.7.0: + version "2.7.0" + resolved "https://registry.yarnpkg.com/mz/-/mz-2.7.0.tgz#95008057a56cafadc2bc63dde7f9ff6955948e32" + integrity sha512-z81GNO7nnYMEhrGh9LeymoE4+Yr0Wn5McHIZMK5cfQCl+NDX08sCZgUc9/6MHni9IWuFLm1Z3HTCXu2z9fN62Q== + dependencies: + any-promise "^1.0.0" + object-assign "^4.0.1" + thenify-all "^1.0.0" + +nanoid@^3.3.4, nanoid@^3.3.6: + version "3.3.6" + resolved "https://registry.yarnpkg.com/nanoid/-/nanoid-3.3.6.tgz#443380c856d6e9f9824267d960b4236ad583ea4c" + integrity sha512-BGcqMMJuToF7i1rt+2PWSNVnWIkGCU78jBG3RxO/bZlnZPK2Cmi2QaffxGO/2RvWi9sL+FAiRiXMgsyxQ1DIDA== + +natural-compare@^1.4.0: + version "1.4.0" + resolved "https://registry.yarnpkg.com/natural-compare/-/natural-compare-1.4.0.tgz#4abebfeed7541f2c27acfb29bdbbd15c8d5ba4f7" + integrity sha512-OWND8ei3VtNC9h7V60qff3SVobHr996CTwgxubgyQYEpg290h9J0buyECNNJexkFm5sOajh5G116RYA1c8ZMSw== + +needle@^3.1.0: + version "3.2.0" + resolved "https://registry.yarnpkg.com/needle/-/needle-3.2.0.tgz#07d240ebcabfd65c76c03afae7f6defe6469df44" + integrity sha512-oUvzXnyLiVyVGoianLijF9O/RecZUf7TkBfimjGrLM4eQhXyeJwM6GeAWccwfQ9aa4gMCZKqhAOuLaMIcQxajQ== + dependencies: + debug "^3.2.6" + iconv-lite "^0.6.3" + sax "^1.2.4" + +negotiator@0.6.3: + version "0.6.3" + resolved "https://registry.yarnpkg.com/negotiator/-/negotiator-0.6.3.tgz#58e323a72fedc0d6f9cd4d31fe49f51479590ccd" + integrity sha512-+EUsqGPLsM+j/zdChZjsnX51g4XrHFOIXwfnCVPGlQk/k5giakcKsuxCObBRu6DSm9opw/O6slWbJdghQM4bBg== + +neo-async@^2.5.0, neo-async@^2.6.1, neo-async@^2.6.2: + version "2.6.2" + resolved "https://registry.yarnpkg.com/neo-async/-/neo-async-2.6.2.tgz#b4aafb93e3aeb2d8174ca53cf163ab7d7308305f" + integrity sha512-Yd3UES5mWCSqR+qNT93S3UoYUkqAZ9lLg8a7g9rimsWmYGK8cVToA4/sF3RrshdyV3sAGMXVUmpMYOw+dLpOuw== + +next@13.4.16: + version "13.4.16" + resolved "https://registry.yarnpkg.com/next/-/next-13.4.16.tgz#327ef6885b22161ed001cd5943c20b5e409a9406" + integrity sha512-1xaA/5DrfpPu0eV31Iro7JfPeqO8uxQWb1zYNTe+KDKdzqkAGapLcDYHMLNKXKB7lHjZ7LfKUOf9dyuzcibrhA== + dependencies: + "@next/env" "13.4.16" + "@swc/helpers" "0.5.1" + busboy "1.6.0" + caniuse-lite "^1.0.30001406" + postcss "8.4.14" + styled-jsx "5.1.1" + watchpack "2.4.0" + zod "3.21.4" + optionalDependencies: + "@next/swc-darwin-arm64" "13.4.16" + "@next/swc-darwin-x64" "13.4.16" + "@next/swc-linux-arm64-gnu" "13.4.16" + "@next/swc-linux-arm64-musl" "13.4.16" + "@next/swc-linux-x64-gnu" "13.4.16" + "@next/swc-linux-x64-musl" "13.4.16" + "@next/swc-win32-arm64-msvc" "13.4.16" + "@next/swc-win32-ia32-msvc" "13.4.16" + "@next/swc-win32-x64-msvc" "13.4.16" + +no-case@^3.0.4: + version "3.0.4" + resolved "https://registry.yarnpkg.com/no-case/-/no-case-3.0.4.tgz#d361fd5c9800f558551a8369fc0dcd4662b6124d" + integrity sha512-fgAN3jGAh+RoxUGZHTSOLJIqUc2wmoBwGR4tbpNAKmmovFoWq0OdRkb0VkldReO2a2iBT/OEulG9XSUc10r3zg== + dependencies: + lower-case "^2.0.2" + tslib "^2.0.3" + +node-abort-controller@^3.0.1: + version "3.1.1" + resolved "https://registry.yarnpkg.com/node-abort-controller/-/node-abort-controller-3.1.1.tgz#a94377e964a9a37ac3976d848cb5c765833b8548" + integrity sha512-AGK2yQKIjRuqnc6VkX2Xj5d+QW8xZ87pa1UK6yA6ouUyuxfHuMP6umE5QK7UmTeOAymo+Zx1Fxiuw9rVx8taHQ== + +node-dir@^0.1.10, node-dir@^0.1.17: + version "0.1.17" + resolved "https://registry.yarnpkg.com/node-dir/-/node-dir-0.1.17.tgz#5f5665d93351335caabef8f1c554516cf5f1e4e5" + integrity sha512-tmPX422rYgofd4epzrNoOXiE8XFZYOcCq1vD7MAXCDO+O+zndlA2ztdKKMa+EeuBG5tHETpr4ml4RGgpqDCCAg== + dependencies: + minimatch "^3.0.2" + +node-fetch-native@^1.0.2: + version "1.4.0" + resolved "https://registry.yarnpkg.com/node-fetch-native/-/node-fetch-native-1.4.0.tgz#fbe8ac033cb6aa44bd106b5e4fd2b6277ba70fa1" + integrity sha512-F5kfEj95kX8tkDhUCYdV8dg3/8Olx/94zB8+ZNthFs6Bz31UpUi8Xh40TN3thLwXgrwXry1pEg9lJ++tLWTcqA== + +node-fetch@^2.0.0: + version "2.7.0" + resolved "https://registry.yarnpkg.com/node-fetch/-/node-fetch-2.7.0.tgz#d0f0fa6e3e2dc1d27efcd8ad99d550bda94d187d" + integrity sha512-c4FRfUm/dbcWZ7U+1Wq0AwCyFL+3nt2bEw05wfxSz+DWpWsitgmSgYmy2dQdWyKC1694ELPqMs/YzUSNozLt8A== + dependencies: + whatwg-url "^5.0.0" + +node-int64@^0.4.0: + version "0.4.0" + resolved "https://registry.yarnpkg.com/node-int64/-/node-int64-0.4.0.tgz#87a9065cdb355d3182d8f94ce11188b825c68a3b" + integrity sha512-O5lz91xSOeoXP6DulyHfllpq+Eg00MWitZIbtPfoSEvqIHdl5gfcY6hYzDWnj0qD5tz52PI08u9qUvSVeUBeHw== + +node-polyfill-webpack-plugin@^2.0.1: + version "2.0.1" + resolved "https://registry.yarnpkg.com/node-polyfill-webpack-plugin/-/node-polyfill-webpack-plugin-2.0.1.tgz#141d86f177103a8517c71d99b7c6a46edbb1bb58" + integrity sha512-ZUMiCnZkP1LF0Th2caY6J/eKKoA0TefpoVa68m/LQU1I/mE8rGt4fNYGgNuCcK+aG8P8P43nbeJ2RqJMOL/Y1A== + dependencies: + assert "^2.0.0" + browserify-zlib "^0.2.0" + buffer "^6.0.3" + console-browserify "^1.2.0" + constants-browserify "^1.0.0" + crypto-browserify "^3.12.0" + domain-browser "^4.22.0" + events "^3.3.0" + filter-obj "^2.0.2" + https-browserify "^1.0.0" + os-browserify "^0.3.0" + path-browserify "^1.0.1" + process "^0.11.10" + punycode "^2.1.1" + querystring-es3 "^0.2.1" + readable-stream "^4.0.0" + stream-browserify "^3.0.0" + stream-http "^3.2.0" + string_decoder "^1.3.0" + timers-browserify "^2.0.12" + tty-browserify "^0.0.1" + type-fest "^2.14.0" + url "^0.11.0" + util "^0.12.4" + vm-browserify "^1.1.2" + +node-releases@^2.0.13: + version "2.0.13" + resolved "https://registry.yarnpkg.com/node-releases/-/node-releases-2.0.13.tgz#d5ed1627c23e3461e819b02e57b75e4899b1c81d" + integrity sha512-uYr7J37ae/ORWdZeQ1xxMJe3NtdmqMC/JZK+geofDrkLUApKRHPd18/TxtBOJ4A0/+uUIliorNrfYV6s1b02eQ== + +normalize-package-data@^2.5.0: + version "2.5.0" + resolved "https://registry.yarnpkg.com/normalize-package-data/-/normalize-package-data-2.5.0.tgz#e66db1838b200c1dfc233225d12cb36520e234a8" + integrity sha512-/5CMN3T0R4XTj4DcGaexo+roZSdSFW/0AOOTROrjxzCG1wrWXEsGbRKevjlIL+ZDE4sZlJr5ED4YW0yqmkK+eA== + dependencies: + hosted-git-info "^2.1.4" + resolve "^1.10.0" + semver "2 || 3 || 4 || 5" + validate-npm-package-license "^3.0.1" + +normalize-path@^3.0.0, normalize-path@~3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/normalize-path/-/normalize-path-3.0.0.tgz#0dcd69ff23a1c9b11fd0978316644a0388216a65" + integrity sha512-6eZs5Ls3WtCisHWp9S2GUy8dqkpGi4BVSz3GaqiE6ezub0512ESztXUwUB6C6IKbQkY2Pnb/mD4WYojCRwcwLA== + +normalize-range@^0.1.2: + version "0.1.2" + resolved "https://registry.yarnpkg.com/normalize-range/-/normalize-range-0.1.2.tgz#2d10c06bdfd312ea9777695a4d28439456b75942" + integrity sha512-bdok/XvKII3nUpklnV6P2hxtMNrCboOjAcyBuQnWEhO665FwrSNRxU+AqpsyvO6LgGYPspN+lu5CLtw4jPRKNA== + +npm-run-path@^4.0.1: + version "4.0.1" + resolved "https://registry.yarnpkg.com/npm-run-path/-/npm-run-path-4.0.1.tgz#b7ecd1e5ed53da8e37a55e1c2269e0b97ed748ea" + integrity sha512-S48WzZW777zhNIrn7gxOlISNAqi9ZC/uQFnRdbeIHhZhCA6UqpkOT8T1G7BvfdgP4Er8gF4sUbaS0i7QvIfCWw== + dependencies: + path-key "^3.0.0" + +nth-check@^2.0.1: + version "2.1.1" + resolved "https://registry.yarnpkg.com/nth-check/-/nth-check-2.1.1.tgz#c9eab428effce36cd6b92c924bdb000ef1f1ed1d" + integrity sha512-lqjrjmaOoAnWfMmBPL+XNnynZh2+swxiX3WUE0s4yEHI6m+AwrK2UZOimIRl3X/4QctVqS8AiZjFqyOGrMXb/w== + dependencies: + boolbase "^1.0.0" + +object-assign@^4.0.1, object-assign@^4.1.1: + version "4.1.1" + resolved "https://registry.yarnpkg.com/object-assign/-/object-assign-4.1.1.tgz#2109adc7965887cfc05cbbd442cac8bfbb360863" + integrity sha512-rJgTQnkUnH1sFw8yT6VSU3zD3sWmu6sZhIseY8VX+GRu3P6F7Fu+JNDoXfklElbLJSnc3FUQHVe4cU5hj+BcUg== + +object-hash@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/object-hash/-/object-hash-3.0.0.tgz#73f97f753e7baffc0e2cc9d6e079079744ac82e9" + integrity sha512-RSn9F68PjH9HqtltsSnqYC1XXoWe9Bju5+213R98cNGttag9q9yAOTzdbsqvIa7aNm5WffBZFpWYr2aWrklWAw== + +object-inspect@^1.12.3, object-inspect@^1.9.0: + version "1.12.3" + resolved "https://registry.yarnpkg.com/object-inspect/-/object-inspect-1.12.3.tgz#ba62dffd67ee256c8c086dfae69e016cd1f198b9" + integrity sha512-geUvdk7c+eizMNUDkRpW1wJwgfOiOeHbxBR/hLXK1aT6zmVSO0jsQcs7fj6MGw89jC/cjGfLcNOrtMYtGqm81g== + +object-is@^1.0.1, object-is@^1.1.5: + version "1.1.5" + resolved "https://registry.yarnpkg.com/object-is/-/object-is-1.1.5.tgz#b9deeaa5fc7f1846a0faecdceec138e5778f53ac" + integrity sha512-3cyDsyHgtmi7I7DfSSI2LDp6SK2lwvtbg0p0R1e0RvTqF5ceGx+K2dfSjm1bKDMVCFEDAQvy+o8c6a7VujOddw== + dependencies: + call-bind "^1.0.2" + define-properties "^1.1.3" + +object-keys@^1.1.1: + version "1.1.1" + resolved "https://registry.yarnpkg.com/object-keys/-/object-keys-1.1.1.tgz#1c47f272df277f3b1daf061677d9c82e2322c60e" + integrity sha512-NuAESUOUMrlIXOfHKzD6bpPu3tYt3xvjNdRIQ+FeT0lNb4K8WR70CaDxhuNguS2XG+GjkyMwOzsN5ZktImfhLA== + +object.assign@^4.1.4: + version "4.1.4" + resolved "https://registry.yarnpkg.com/object.assign/-/object.assign-4.1.4.tgz#9673c7c7c351ab8c4d0b516f4343ebf4dfb7799f" + integrity sha512-1mxKf0e58bvyjSCtKYY4sRe9itRk3PJpquJOjeIkz885CczcI4IvJJDLPS72oowuSh+pBxUFROpX+TU++hxhZQ== + dependencies: + call-bind "^1.0.2" + define-properties "^1.1.4" + has-symbols "^1.0.3" + object-keys "^1.1.1" + +object.entries@^1.1.6: + version "1.1.7" + resolved "https://registry.yarnpkg.com/object.entries/-/object.entries-1.1.7.tgz#2b47760e2a2e3a752f39dd874655c61a7f03c131" + integrity sha512-jCBs/0plmPsOnrKAfFQXRG2NFjlhZgjjcBLSmTnEhU8U6vVTsVe8ANeQJCHTl3gSsI4J+0emOoCgoKlmQPMgmA== + dependencies: + call-bind "^1.0.2" + define-properties "^1.2.0" + es-abstract "^1.22.1" + +object.fromentries@^2.0.6: + version "2.0.7" + resolved "https://registry.yarnpkg.com/object.fromentries/-/object.fromentries-2.0.7.tgz#71e95f441e9a0ea6baf682ecaaf37fa2a8d7e616" + integrity sha512-UPbPHML6sL8PI/mOqPwsH4G6iyXcCGzLin8KvEPenOZN5lpCNBZZQ+V62vdjB1mQHrmqGQt5/OJzemUA+KJmEA== + dependencies: + call-bind "^1.0.2" + define-properties "^1.2.0" + es-abstract "^1.22.1" + +object.groupby@^1.0.0: + version "1.0.1" + resolved "https://registry.yarnpkg.com/object.groupby/-/object.groupby-1.0.1.tgz#d41d9f3c8d6c778d9cbac86b4ee9f5af103152ee" + integrity sha512-HqaQtqLnp/8Bn4GL16cj+CUYbnpe1bh0TtEaWvybszDG4tgxCJuRpV8VGuvNaI1fAnI4lUJzDG55MXcOH4JZcQ== + dependencies: + call-bind "^1.0.2" + define-properties "^1.2.0" + es-abstract "^1.22.1" + get-intrinsic "^1.2.1" + +object.hasown@^1.1.2: + version "1.1.3" + resolved "https://registry.yarnpkg.com/object.hasown/-/object.hasown-1.1.3.tgz#6a5f2897bb4d3668b8e79364f98ccf971bda55ae" + integrity sha512-fFI4VcYpRHvSLXxP7yiZOMAd331cPfd2p7PFDVbgUsYOfCT3tICVqXWngbjr4m49OvsBwUBQ6O2uQoJvy3RexA== + dependencies: + define-properties "^1.2.0" + es-abstract "^1.22.1" + +object.values@^1.1.6: + version "1.1.7" + resolved "https://registry.yarnpkg.com/object.values/-/object.values-1.1.7.tgz#617ed13272e7e1071b43973aa1655d9291b8442a" + integrity sha512-aU6xnDFYT3x17e/f0IiiwlGPTy2jzMySGfUB4fq6z7CV8l85CWHDk5ErhyhpfDHhrOMwGFhSQkhMGHaIotA6Ng== + dependencies: + call-bind "^1.0.2" + define-properties "^1.2.0" + es-abstract "^1.22.1" + +objectorarray@^1.0.5: + version "1.0.5" + resolved "https://registry.yarnpkg.com/objectorarray/-/objectorarray-1.0.5.tgz#2c05248bbefabd8f43ad13b41085951aac5e68a5" + integrity sha512-eJJDYkhJFFbBBAxeh8xW+weHlkI28n2ZdQV/J/DNfWfSKlGEf2xcfAbZTv3riEXHAhL9SVOTs2pRmXiSTf78xg== + +on-finished@2.4.1: + version "2.4.1" + resolved "https://registry.yarnpkg.com/on-finished/-/on-finished-2.4.1.tgz#58c8c44116e54845ad57f14ab10b03533184ac3f" + integrity sha512-oVlzkg3ENAhCk2zdv7IJwd/QUD4z2RxRwpkcGY8psCVcCYZNq4wYnVWALHM+brtuJjePWiYF/ClmuDr8Ch5+kg== + dependencies: + ee-first "1.1.1" + +on-headers@~1.0.2: + version "1.0.2" + resolved "https://registry.yarnpkg.com/on-headers/-/on-headers-1.0.2.tgz#772b0ae6aaa525c399e489adfad90c403eb3c28f" + integrity sha512-pZAE+FJLoyITytdqK0U5s+FIpjN0JP3OzFi/u8Rx+EV5/W+JTWGXG8xFzevE7AjBfDqHv/8vL8qQsIhHnqRkrA== + +once@^1.3.0, once@^1.3.1, once@^1.4.0: + version "1.4.0" + resolved "https://registry.yarnpkg.com/once/-/once-1.4.0.tgz#583b1aa775961d4b113ac17d9c50baef9dd76bd1" + integrity sha512-lNaJgI+2Q5URQBkccEKHTQOPaXdUxnZZElQTZY0MFUAuaEqe1E+Nyvgdz/aIyNi6Z9MzO5dv1H8n58/GELp3+w== + dependencies: + wrappy "1" + +onetime@^5.1.0, onetime@^5.1.2: + version "5.1.2" + resolved "https://registry.yarnpkg.com/onetime/-/onetime-5.1.2.tgz#d0e96ebb56b07476df1dd9c4806e5237985ca45e" + integrity sha512-kbpaSSGJTWdAY5KPVeMOKXSrPtr8C8C7wodJbcsd51jRnmD+GZu8Y0VoU6Dm5Z4vWr0Ig/1NKuWRKf7j5aaYSg== + dependencies: + mimic-fn "^2.1.0" + +open@^8.0.4, open@^8.4.0: + version "8.4.2" + resolved "https://registry.yarnpkg.com/open/-/open-8.4.2.tgz#5b5ffe2a8f793dcd2aad73e550cb87b59cb084f9" + integrity sha512-7x81NCL719oNbsq/3mh+hVrAWmFuEYUqrq/Iw3kUzH8ReypT9QQ0BLoJS7/G9k6N81XjW4qHWtjWwe/9eLy1EQ== + dependencies: + define-lazy-prop "^2.0.0" + is-docker "^2.1.1" + is-wsl "^2.2.0" + +optionator@^0.9.3: + version "0.9.3" + resolved "https://registry.yarnpkg.com/optionator/-/optionator-0.9.3.tgz#007397d44ed1872fdc6ed31360190f81814e2c64" + integrity sha512-JjCoypp+jKn1ttEFExxhetCKeJt9zhAgAve5FXHixTvFDW/5aEktX9bufBKLRRMdU7bNtpLfcGu94B3cdEJgjg== + dependencies: + "@aashutoshrathi/word-wrap" "^1.2.3" + deep-is "^0.1.3" + fast-levenshtein "^2.0.6" + levn "^0.4.1" + prelude-ls "^1.2.1" + type-check "^0.4.0" + +ora@^5.4.1: + version "5.4.1" + resolved "https://registry.yarnpkg.com/ora/-/ora-5.4.1.tgz#1b2678426af4ac4a509008e5e4ac9e9959db9e18" + integrity sha512-5b6Y85tPxZZ7QytO+BQzysW31HJku27cRIlkbAXaNx+BdcVi+LlRFmVXzeF6a7JCwJpyw5c4b+YSVImQIrBpuQ== + dependencies: + bl "^4.1.0" + chalk "^4.1.0" + cli-cursor "^3.1.0" + cli-spinners "^2.5.0" + is-interactive "^1.0.0" + is-unicode-supported "^0.1.0" + log-symbols "^4.1.0" + strip-ansi "^6.0.0" + wcwidth "^1.0.1" + +os-browserify@^0.3.0: + version "0.3.0" + resolved "https://registry.yarnpkg.com/os-browserify/-/os-browserify-0.3.0.tgz#854373c7f5c2315914fc9bfc6bd8238fdda1ec27" + integrity sha512-gjcpUc3clBf9+210TRaDWbf+rZZZEshZ+DlXMRCeAjp0xhTrnQsKHypIy1J3d5hKdUzj69t708EHtU8P6bUn0A== + +p-limit@^2.0.0, p-limit@^2.2.0: + version "2.3.0" + resolved "https://registry.yarnpkg.com/p-limit/-/p-limit-2.3.0.tgz#3dd33c647a214fdfffd835933eb086da0dc21db1" + integrity sha512-//88mFWSJx8lxCzwdAABTJL2MyWB12+eIY7MDL2SqLmAkeKU9qxRvWuSyTjm3FUmpBEMuFfckAIqEaVGUDxb6w== + dependencies: + p-try "^2.0.0" + +p-limit@^3.0.2: + version "3.1.0" + resolved "https://registry.yarnpkg.com/p-limit/-/p-limit-3.1.0.tgz#e1daccbe78d0d1388ca18c64fea38e3e57e3706b" + integrity sha512-TYOanM3wGwNGsZN2cVTYPArw454xnXj5qmWF1bEoAc4+cU/ol7GVh7odevjp1FNHduHc3KZMcFduxU5Xc6uJRQ== + dependencies: + yocto-queue "^0.1.0" + +p-limit@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/p-limit/-/p-limit-4.0.0.tgz#914af6544ed32bfa54670b061cafcbd04984b644" + integrity sha512-5b0R4txpzjPWVw/cXXUResoD4hb6U/x9BH08L7nw+GN1sezDzPdxeRvpc9c433fZhBan/wusjbCsqwqm4EIBIQ== + dependencies: + yocto-queue "^1.0.0" + +p-locate@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/p-locate/-/p-locate-3.0.0.tgz#322d69a05c0264b25997d9f40cd8a891ab0064a4" + integrity sha512-x+12w/To+4GFfgJhBEpiDcLozRJGegY+Ei7/z0tSLkMmxGZNybVMSfWj9aJn8Z5Fc7dBUNJOOVgPv2H7IwulSQ== + dependencies: + p-limit "^2.0.0" + +p-locate@^4.1.0: + version "4.1.0" + resolved "https://registry.yarnpkg.com/p-locate/-/p-locate-4.1.0.tgz#a3428bb7088b3a60292f66919278b7c297ad4f07" + integrity sha512-R79ZZ/0wAxKGu3oYMlz8jy/kbhsNrS7SKZ7PxEHBgJ5+F2mtFW2fK2cOtBh1cHYkQsbzFV7I+EoRKe6Yt0oK7A== + dependencies: + p-limit "^2.2.0" + +p-locate@^5.0.0: + version "5.0.0" + resolved "https://registry.yarnpkg.com/p-locate/-/p-locate-5.0.0.tgz#83c8315c6785005e3bd021839411c9e110e6d834" + integrity sha512-LaNjtRWUBY++zB5nE/NwcaoMylSPk+S+ZHNB1TzdbMJMny6dynpAGt7X/tl/QYq3TIeE6nxHppbo2LGymrG5Pw== + dependencies: + p-limit "^3.0.2" + +p-locate@^6.0.0: + version "6.0.0" + resolved "https://registry.yarnpkg.com/p-locate/-/p-locate-6.0.0.tgz#3da9a49d4934b901089dca3302fa65dc5a05c04f" + integrity sha512-wPrq66Llhl7/4AGC6I+cqxT07LhXvWL08LNXz1fENOw0Ap4sRZZ/gZpTTJ5jpurzzzfS2W/Ge9BY3LgLjCShcw== + dependencies: + p-limit "^4.0.0" + +p-map@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/p-map/-/p-map-4.0.0.tgz#bb2f95a5eda2ec168ec9274e06a747c3e2904d2b" + integrity sha512-/bjOqmgETBYB5BoEeGVea8dmvHb2m9GLy1E9W43yeyfP6QQCZGFNa+XRceJEuDB6zqr+gKpIAmlLebMpykw/MQ== + dependencies: + aggregate-error "^3.0.0" + +p-try@^2.0.0: + version "2.2.0" + resolved "https://registry.yarnpkg.com/p-try/-/p-try-2.2.0.tgz#cb2868540e313d61de58fafbe35ce9004d5540e6" + integrity sha512-R4nPAVTAU0B9D35/Gk3uJf/7XYbQcyohSKdvAxIRSNghFl4e71hVoGnBNQz9cWaXxO2I10KTC+3jMdvvoKw6dQ== + +pako@~0.2.0: + version "0.2.9" + resolved "https://registry.yarnpkg.com/pako/-/pako-0.2.9.tgz#f3f7522f4ef782348da8161bad9ecfd51bf83a75" + integrity sha512-NUcwaKxUxWrZLpDG+z/xZaCgQITkA/Dv4V/T6bw7VON6l1Xz/VnrBqrYjZQ12TamKHzITTfOEIYUj48y2KXImA== + +pako@~1.0.5: + version "1.0.11" + resolved "https://registry.yarnpkg.com/pako/-/pako-1.0.11.tgz#6c9599d340d54dfd3946380252a35705a6b992bf" + integrity sha512-4hLB8Py4zZce5s4yd9XzopqwVv/yGNhV1Bl8NTmCq1763HeK2+EwVTv+leGeL13Dnh2wfbqowVPXCIO0z4taYw== + +param-case@^3.0.4: + version "3.0.4" + resolved "https://registry.yarnpkg.com/param-case/-/param-case-3.0.4.tgz#7d17fe4aa12bde34d4a77d91acfb6219caad01c5" + integrity sha512-RXlj7zCYokReqWpOPH9oYivUzLYZ5vAPIfEmCTNViosC78F8F0H9y7T7gG2M39ymgutxF5gcFEsyZQSph9Bp3A== + dependencies: + dot-case "^3.0.4" + tslib "^2.0.3" + +parent-module@^1.0.0: + version "1.0.1" + resolved "https://registry.yarnpkg.com/parent-module/-/parent-module-1.0.1.tgz#691d2709e78c79fae3a156622452d00762caaaa2" + integrity sha512-GQ2EWRpQV8/o+Aw8YqtfZZPfNRWZYkbidE9k5rpl/hC3vtHHBfGm2Ifi6qWV+coDGkrUKZAxE3Lot5kcsRlh+g== + dependencies: + callsites "^3.0.0" + +parse-asn1@^5.0.0, parse-asn1@^5.1.5: + version "5.1.6" + resolved "https://registry.yarnpkg.com/parse-asn1/-/parse-asn1-5.1.6.tgz#385080a3ec13cb62a62d39409cb3e88844cdaed4" + integrity sha512-RnZRo1EPU6JBnra2vGHj0yhp6ebyjBZpmUCLHWiFhxlzvBCCpAuZ7elsBp1PVAbQN0/04VD/19rfzlBSwLstMw== + dependencies: + asn1.js "^5.2.0" + browserify-aes "^1.0.0" + evp_bytestokey "^1.0.0" + pbkdf2 "^3.0.3" + safe-buffer "^5.1.1" + +parse-json@^5.0.0, parse-json@^5.2.0: + version "5.2.0" + resolved "https://registry.yarnpkg.com/parse-json/-/parse-json-5.2.0.tgz#c76fc66dee54231c962b22bcc8a72cf2f99753cd" + integrity sha512-ayCKvm/phCGxOkYRSCM82iDwct8/EonSEgCSxWxD7ve6jHggsFl4fZVQBPRNgQoKiuV/odhFrGzQXZwbifC8Rg== + dependencies: + "@babel/code-frame" "^7.0.0" + error-ex "^1.3.1" + json-parse-even-better-errors "^2.3.0" + lines-and-columns "^1.1.6" + +parse-node-version@^1.0.1: + version "1.0.1" + resolved "https://registry.yarnpkg.com/parse-node-version/-/parse-node-version-1.0.1.tgz#e2b5dbede00e7fa9bc363607f53327e8b073189b" + integrity sha512-3YHlOa/JgH6Mnpr05jP9eDG254US9ek25LyIxZlDItp2iJtwyaXQb57lBYLdT3MowkUFYEV2XXNAYIPlESvJlA== + +parseurl@~1.3.2, parseurl@~1.3.3: + version "1.3.3" + resolved "https://registry.yarnpkg.com/parseurl/-/parseurl-1.3.3.tgz#9da19e7bee8d12dff0513ed5b76957793bc2e8d4" + integrity sha512-CiyeOxFT/JZyN5m0z9PfXw4SCBJ6Sygz1Dpl0wqjlhDEGGBP1GnsUVEL0p63hoG1fcj3fHynXi9NYO4nWOL+qQ== + +pascal-case@^3.1.2: + version "3.1.2" + resolved "https://registry.yarnpkg.com/pascal-case/-/pascal-case-3.1.2.tgz#b48e0ef2b98e205e7c1dae747d0b1508237660eb" + integrity sha512-uWlGT3YSnK9x3BQJaOdcZwrnV6hPpd8jFH1/ucpiLRPh/2zCVJKS19E4GvYHvaCcACn3foXZ0cLB9Wrx1KGe5g== + dependencies: + no-case "^3.0.4" + tslib "^2.0.3" + +path-browserify@^1.0.1: + version "1.0.1" + resolved "https://registry.yarnpkg.com/path-browserify/-/path-browserify-1.0.1.tgz#d98454a9c3753d5790860f16f68867b9e46be1fd" + integrity sha512-b7uo2UCUOYZcnF/3ID0lulOJi/bafxa1xPe7ZPsammBSpjSWQkjNxlt635YGS2MiR9GjvuXCtz2emr3jbsz98g== + +path-exists@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/path-exists/-/path-exists-3.0.0.tgz#ce0ebeaa5f78cb18925ea7d810d7b59b010fd515" + integrity sha512-bpC7GYwiDYQ4wYLe+FA8lhRjhQCMcQGuSgGGqDkg/QerRWw9CmGRT0iSOVRSZJ29NMLZgIzqaljJ63oaL4NIJQ== + +path-exists@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/path-exists/-/path-exists-4.0.0.tgz#513bdbe2d3b95d7762e8c1137efa195c6c61b5b3" + integrity sha512-ak9Qy5Q7jYb2Wwcey5Fpvg2KoAc/ZIhLSLOSBmRmygPsGwkVVt0fZa0qrtMz+m6tJTAHfZQ8FnmB4MG4LWy7/w== + +path-exists@^5.0.0: + version "5.0.0" + resolved "https://registry.yarnpkg.com/path-exists/-/path-exists-5.0.0.tgz#a6aad9489200b21fab31e49cf09277e5116fb9e7" + integrity sha512-RjhtfwJOxzcFmNOi6ltcbcu4Iu+FL3zEj83dk4kAS+fVpTxXLO1b38RvJgT/0QwvV/L3aY9TAnyv0EOqW4GoMQ== + +path-is-absolute@^1.0.0: + version "1.0.1" + resolved "https://registry.yarnpkg.com/path-is-absolute/-/path-is-absolute-1.0.1.tgz#174b9268735534ffbc7ace6bf53a5a9e1b5c5f5f" + integrity sha512-AVbw3UJ2e9bq64vSaS9Am0fje1Pa8pbGqTTsmXfaIiMpnr5DlDhfJOuLj9Sf95ZPVDAUerDfEk88MPmPe7UCQg== + +path-key@^3.0.0, path-key@^3.1.0: + version "3.1.1" + resolved "https://registry.yarnpkg.com/path-key/-/path-key-3.1.1.tgz#581f6ade658cbba65a0d3380de7753295054f375" + integrity sha512-ojmeN0qd+y0jszEtoY48r0Peq5dwMEkIlCOu6Q5f41lfkswXuKtYrhgoTpLnyIcHm24Uhqx+5Tqm2InSwLhE6Q== + +path-parse@^1.0.7: + version "1.0.7" + resolved "https://registry.yarnpkg.com/path-parse/-/path-parse-1.0.7.tgz#fbc114b60ca42b30d9daf5858e4bd68bbedb6735" + integrity sha512-LDJzPVEEEPR+y48z93A0Ed0yXb8pAByGWo/k5YYdYgpY2/2EsOsksJrq7lOHxryrVOn1ejG6oAp8ahvOIQD8sw== + +path-scurry@^1.10.1: + version "1.10.1" + resolved "https://registry.yarnpkg.com/path-scurry/-/path-scurry-1.10.1.tgz#9ba6bf5aa8500fe9fd67df4f0d9483b2b0bfc698" + integrity sha512-MkhCqzzBEpPvxxQ71Md0b1Kk51W01lrYvlMzSUaIzNsODdd7mqhiimSZlr+VegAz5Z6Vzt9Xg2ttE//XBhH3EQ== + dependencies: + lru-cache "^9.1.1 || ^10.0.0" + minipass "^5.0.0 || ^6.0.2 || ^7.0.0" + +path-to-regexp@0.1.7: + version "0.1.7" + resolved "https://registry.yarnpkg.com/path-to-regexp/-/path-to-regexp-0.1.7.tgz#df604178005f522f15eb4490e7247a1bfaa67f8c" + integrity sha512-5DFkuoqlv1uYQKxy8omFBeJPQcdoE07Kv2sferDCrAq1ohOU+MSDswDIbnx3YAM60qIOnYa53wBhXW0EbMonrQ== + +path-type@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/path-type/-/path-type-4.0.0.tgz#84ed01c0a7ba380afe09d90a8c180dcd9d03043b" + integrity sha512-gDKb8aZMDeD/tZWs9P6+q0J9Mwkdl6xMV8TjnGP3qJVJ06bdMgkbBlLU8IdfOsIsFz2BW1rNVT3XuNEl8zPAvw== + +pathe@^1.1.0: + version "1.1.1" + resolved "https://registry.yarnpkg.com/pathe/-/pathe-1.1.1.tgz#1dd31d382b974ba69809adc9a7a347e65d84829a" + integrity sha512-d+RQGp0MAYTIaDBIMmOfMwz3E+LOZnxx1HZd5R18mmCZY0QBlK0LDZfPc8FW8Ed2DlvsuE6PRjroDY+wg4+j/Q== + +pbkdf2@^3.0.3: + version "3.1.2" + resolved "https://registry.yarnpkg.com/pbkdf2/-/pbkdf2-3.1.2.tgz#dd822aa0887580e52f1a039dc3eda108efae3075" + integrity sha512-iuh7L6jA7JEGu2WxDwtQP1ddOpaJNC4KlDEFfdQajSGgGPNi4OyDc2R7QnbY2bR9QjBVGwgvTdNJZoE7RaxUMA== + dependencies: + create-hash "^1.1.2" + create-hmac "^1.1.4" + ripemd160 "^2.0.1" + safe-buffer "^5.0.1" + sha.js "^2.4.8" + +peek-stream@^1.1.0: + version "1.1.3" + resolved "https://registry.yarnpkg.com/peek-stream/-/peek-stream-1.1.3.tgz#3b35d84b7ccbbd262fff31dc10da56856ead6d67" + integrity sha512-FhJ+YbOSBb9/rIl2ZeE/QHEsWn7PqNYt8ARAY3kIgNGOk13g9FGyIY6JIl/xB/3TFRVoTv5as0l11weORrTekA== + dependencies: + buffer-from "^1.0.0" + duplexify "^3.5.0" + through2 "^2.0.3" + +pend@~1.2.0: + version "1.2.0" + resolved "https://registry.yarnpkg.com/pend/-/pend-1.2.0.tgz#7a57eb550a6783f9115331fcf4663d5c8e007a50" + integrity sha512-F3asv42UuXchdzt+xXqfW1OGlVBe+mxa2mqI0pg5yAHZPvFmY3Y6drSf/GQ1A86WgWEN9Kzh/WrgKa6iGcHXLg== + +picocolors@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/picocolors/-/picocolors-1.0.0.tgz#cb5bdc74ff3f51892236eaf79d68bc44564ab81c" + integrity sha512-1fygroTLlHu66zi26VoTDv8yRgm0Fccecssto+MhsZ0D/DGW2sm8E8AjW7NU5VVTRt5GxbeZ5qBuJr+HyLYkjQ== + +picomatch@^2.0.4, picomatch@^2.2.1, picomatch@^2.2.3, picomatch@^2.3.0, picomatch@^2.3.1: + version "2.3.1" + resolved "https://registry.yarnpkg.com/picomatch/-/picomatch-2.3.1.tgz#3ba3833733646d9d3e4995946c1365a67fb07a42" + integrity sha512-JU3teHTNjmE2VCGFzuY8EXzCDVwEqB2a8fsIvwaStHhAWJEeVd1o1QD80CU6+ZdEXXSLbSsuLwJjkCBWqRQUVA== + +pify@^2.0.0, pify@^2.3.0: + version "2.3.0" + resolved "https://registry.yarnpkg.com/pify/-/pify-2.3.0.tgz#ed141a6ac043a849ea588498e7dca8b15330e90c" + integrity sha512-udgsAY+fTnvv7kI7aaxbqwWNb0AHiB0qBO89PZKPkoTmGOgdbrHDKD+0B2X4uTfJ/FT1R09r9gTsjUjNJotuog== + +pify@^4.0.1: + version "4.0.1" + resolved "https://registry.yarnpkg.com/pify/-/pify-4.0.1.tgz#4b2cd25c50d598735c50292224fd8c6df41e3231" + integrity sha512-uB80kBFb/tfd68bVleG9T5GGsGPjJrLAUpR5PZIrhBnIaRTQRjqdJSsIKkOP6OAIFbj7GOrcudc5pNjZ+geV2g== + +pinkie-promise@^2.0.0: + version "2.0.1" + resolved "https://registry.yarnpkg.com/pinkie-promise/-/pinkie-promise-2.0.1.tgz#2135d6dfa7a358c069ac9b178776288228450ffa" + integrity sha512-0Gni6D4UcLTbv9c57DfxDGdr41XfgUjqWZu492f0cIGr16zDU06BWP/RAEvOuo7CQ0CNjHaLlM59YJJFm3NWlw== + dependencies: + pinkie "^2.0.0" + +pinkie@^2.0.0: + version "2.0.4" + resolved "https://registry.yarnpkg.com/pinkie/-/pinkie-2.0.4.tgz#72556b80cfa0d48a974e80e77248e80ed4f7f870" + integrity sha512-MnUuEycAemtSaeFSjXKW/aroV7akBbY+Sv+RkyqFjgAe73F+MR0TBWKBRDkmfWq/HiFmdavfZ1G7h4SPZXaCSg== + +pirates@^4.0.1, pirates@^4.0.4, pirates@^4.0.5: + version "4.0.6" + resolved "https://registry.yarnpkg.com/pirates/-/pirates-4.0.6.tgz#3018ae32ecfcff6c29ba2267cbf21166ac1f36b9" + integrity sha512-saLsH7WeYYPiD25LDuLRRY/i+6HaPYr6G1OUlN39otzkSTxKnubR9RTxS3/Kk50s1g2JTgFwWQDQyplC5/SHZg== + +pkg-dir@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/pkg-dir/-/pkg-dir-3.0.0.tgz#2749020f239ed990881b1f71210d51eb6523bea3" + integrity sha512-/E57AYkoeQ25qkxMj5PBOVgF8Kiu/h7cYS30Z5+R7WaiCCBfLq58ZI/dSeaEKb9WVJV5n/03QwrN3IeWIFllvw== + dependencies: + find-up "^3.0.0" + +pkg-dir@^4.1.0: + version "4.2.0" + resolved "https://registry.yarnpkg.com/pkg-dir/-/pkg-dir-4.2.0.tgz#f099133df7ede422e81d1d8448270eeb3e4261f3" + integrity sha512-HRDzbaKjC+AOWVXxAU/x54COGeIv9eb+6CkDSQoNTt4XyWoIJvuPsXizxu/Fr23EiekbtZwmh1IcIG/l/a10GQ== + dependencies: + find-up "^4.0.0" + +pkg-dir@^5.0.0: + version "5.0.0" + resolved "https://registry.yarnpkg.com/pkg-dir/-/pkg-dir-5.0.0.tgz#a02d6aebe6ba133a928f74aec20bafdfe6b8e760" + integrity sha512-NPE8TDbzl/3YQYY7CSS228s3g2ollTFnc+Qi3tqmqJp9Vg2ovUpixcJEo2HJScN2Ez+kEaal6y70c0ehqJBJeA== + dependencies: + find-up "^5.0.0" + +pkg-dir@^7.0.0: + version "7.0.0" + resolved "https://registry.yarnpkg.com/pkg-dir/-/pkg-dir-7.0.0.tgz#8f0c08d6df4476756c5ff29b3282d0bab7517d11" + integrity sha512-Ie9z/WINcxxLp27BKOCHGde4ITq9UklYKDzVo1nhk5sqGEXU3FpkwP5GM2voTGJkGd9B3Otl+Q4uwSOeSUtOBA== + dependencies: + find-up "^6.3.0" + +pnp-webpack-plugin@^1.7.0: + version "1.7.0" + resolved "https://registry.yarnpkg.com/pnp-webpack-plugin/-/pnp-webpack-plugin-1.7.0.tgz#65741384f6d8056f36e2255a8d67ffc20866f5c9" + integrity sha512-2Rb3vm+EXble/sMXNSu6eoBx8e79gKqhNq9F5ZWW6ERNCTE/Q0wQNne5541tE5vKjfM8hpNCYL+LGc1YTfI0dg== + dependencies: + ts-pnp "^1.1.6" + +polished@^4.2.2: + version "4.2.2" + resolved "https://registry.yarnpkg.com/polished/-/polished-4.2.2.tgz#2529bb7c3198945373c52e34618c8fe7b1aa84d1" + integrity sha512-Sz2Lkdxz6F2Pgnpi9U5Ng/WdWAUZxmHrNPoVlm3aAemxoy2Qy7LGjQg4uf8qKelDAUW94F4np3iH2YPf2qefcQ== + dependencies: + "@babel/runtime" "^7.17.8" + +postcss-import@^15.1.0: + version "15.1.0" + resolved "https://registry.yarnpkg.com/postcss-import/-/postcss-import-15.1.0.tgz#41c64ed8cc0e23735a9698b3249ffdbf704adc70" + integrity sha512-hpr+J05B2FVYUAXHeK1YyI267J/dDDhMU6B6civm8hSY1jYJnBXxzKDKDswzJmtLHryrjhnDjqqp/49t8FALew== + dependencies: + postcss-value-parser "^4.0.0" + read-cache "^1.0.0" + resolve "^1.1.7" + +postcss-js@^4.0.1: + version "4.0.1" + resolved "https://registry.yarnpkg.com/postcss-js/-/postcss-js-4.0.1.tgz#61598186f3703bab052f1c4f7d805f3991bee9d2" + integrity sha512-dDLF8pEO191hJMtlHFPRa8xsizHaM82MLfNkUHdUtVEV3tgTp5oj+8qbEqYM57SLfc74KSbw//4SeJma2LRVIw== + dependencies: + camelcase-css "^2.0.1" + +postcss-load-config@^4.0.1: + version "4.0.1" + resolved "https://registry.yarnpkg.com/postcss-load-config/-/postcss-load-config-4.0.1.tgz#152383f481c2758274404e4962743191d73875bd" + integrity sha512-vEJIc8RdiBRu3oRAI0ymerOn+7rPuMvRXslTvZUKZonDHFIczxztIyJ1urxM1x9JXEikvpWWTUUqal5j/8QgvA== + dependencies: + lilconfig "^2.0.5" + yaml "^2.1.1" + +postcss-loader@^7.0.2, postcss-loader@^7.2.4: + version "7.3.3" + resolved "https://registry.yarnpkg.com/postcss-loader/-/postcss-loader-7.3.3.tgz#6da03e71a918ef49df1bb4be4c80401df8e249dd" + integrity sha512-YgO/yhtevGO/vJePCQmTxiaEwER94LABZN0ZMT4A0vsak9TpO+RvKRs7EmJ8peIlB9xfXCsS7M8LjqncsUZ5HA== + dependencies: + cosmiconfig "^8.2.0" + jiti "^1.18.2" + semver "^7.3.8" + +postcss-modules-extract-imports@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/postcss-modules-extract-imports/-/postcss-modules-extract-imports-3.0.0.tgz#cda1f047c0ae80c97dbe28c3e76a43b88025741d" + integrity sha512-bdHleFnP3kZ4NYDhuGlVK+CMrQ/pqUm8bx/oGL93K6gVwiclvX5x0n76fYMKuIGKzlABOy13zsvqjb0f92TEXw== + +postcss-modules-local-by-default@^4.0.3: + version "4.0.3" + resolved "https://registry.yarnpkg.com/postcss-modules-local-by-default/-/postcss-modules-local-by-default-4.0.3.tgz#b08eb4f083050708998ba2c6061b50c2870ca524" + integrity sha512-2/u2zraspoACtrbFRnTijMiQtb4GW4BvatjaG/bCjYQo8kLTdevCUlwuBHx2sCnSyrI3x3qj4ZK1j5LQBgzmwA== + dependencies: + icss-utils "^5.0.0" + postcss-selector-parser "^6.0.2" + postcss-value-parser "^4.1.0" + +postcss-modules-scope@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/postcss-modules-scope/-/postcss-modules-scope-3.0.0.tgz#9ef3151456d3bbfa120ca44898dfca6f2fa01f06" + integrity sha512-hncihwFA2yPath8oZ15PZqvWGkWf+XUfQgUGamS4LqoP1anQLOsOJw0vr7J7IwLpoY9fatA2qiGUGmuZL0Iqlg== + dependencies: + postcss-selector-parser "^6.0.4" + +postcss-modules-values@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/postcss-modules-values/-/postcss-modules-values-4.0.0.tgz#d7c5e7e68c3bb3c9b27cbf48ca0bb3ffb4602c9c" + integrity sha512-RDxHkAiEGI78gS2ofyvCsu7iycRv7oqw5xMWn9iMoR0N/7mf9D50ecQqUo5BZ9Zh2vH4bCUR/ktCqbB9m8vJjQ== + dependencies: + icss-utils "^5.0.0" + +postcss-nested@^6.0.1: + version "6.0.1" + resolved "https://registry.yarnpkg.com/postcss-nested/-/postcss-nested-6.0.1.tgz#f83dc9846ca16d2f4fa864f16e9d9f7d0961662c" + integrity sha512-mEp4xPMi5bSWiMbsgoPfcP74lsWLHkQbZc3sY+jWYd65CUwXrUaTp0fmNpa01ZcETKlIgUdFN/MpS2xZtqL9dQ== + dependencies: + postcss-selector-parser "^6.0.11" + +postcss-selector-parser@^6.0.11, postcss-selector-parser@^6.0.2, postcss-selector-parser@^6.0.4: + version "6.0.13" + resolved "https://registry.yarnpkg.com/postcss-selector-parser/-/postcss-selector-parser-6.0.13.tgz#d05d8d76b1e8e173257ef9d60b706a8e5e99bf1b" + integrity sha512-EaV1Gl4mUEV4ddhDnv/xtj7sxwrwxdetHdWUGnT4VJQf+4d05v6lHYZr8N573k5Z0BViss7BDhfWtKS3+sfAqQ== + dependencies: + cssesc "^3.0.0" + util-deprecate "^1.0.2" + +postcss-value-parser@^3.3.0: + version "3.3.1" + resolved "https://registry.yarnpkg.com/postcss-value-parser/-/postcss-value-parser-3.3.1.tgz#9ff822547e2893213cf1c30efa51ac5fd1ba8281" + integrity sha512-pISE66AbVkp4fDQ7VHBwRNXzAAKJjw4Vw7nWI/+Q3vuly7SNfgYXvm6i5IgFylHGK5sP/xHAbB7N49OS4gWNyQ== + +postcss-value-parser@^4.0.0, postcss-value-parser@^4.0.2, postcss-value-parser@^4.1.0, postcss-value-parser@^4.2.0: + version "4.2.0" + resolved "https://registry.yarnpkg.com/postcss-value-parser/-/postcss-value-parser-4.2.0.tgz#723c09920836ba6d3e5af019f92bc0971c02e514" + integrity sha512-1NNCs6uurfkVbeXG4S8JFT9t19m45ICnif8zWLd5oPSZ50QnwMfK+H3jv408d4jw/7Bttv5axS5IiHoLaVNHeQ== + +postcss@8.4.14: + version "8.4.14" + resolved "https://registry.yarnpkg.com/postcss/-/postcss-8.4.14.tgz#ee9274d5622b4858c1007a74d76e42e56fd21caf" + integrity sha512-E398TUmfAYFPBSdzgeieK2Y1+1cpdxJx8yXbK/m57nRhKSmk1GB2tO4lbLBtlkfPQTDKfe4Xqv1ASWPpayPEig== + dependencies: + nanoid "^3.3.4" + picocolors "^1.0.0" + source-map-js "^1.0.2" + +postcss@^8.2.14, postcss@^8.4.21, postcss@^8.4.23, postcss@^8.4.28: + version "8.4.29" + resolved "https://registry.yarnpkg.com/postcss/-/postcss-8.4.29.tgz#33bc121cf3b3688d4ddef50be869b2a54185a1dd" + integrity sha512-cbI+jaqIeu/VGqXEarWkRCCffhjgXc0qjBtXpqJhTBohMUjUQnbBr0xqX3vEKudc4iviTewcJo5ajcec5+wdJw== + dependencies: + nanoid "^3.3.6" + picocolors "^1.0.0" + source-map-js "^1.0.2" + +prelude-ls@^1.2.1: + version "1.2.1" + resolved "https://registry.yarnpkg.com/prelude-ls/-/prelude-ls-1.2.1.tgz#debc6489d7a6e6b0e7611888cec880337d316396" + integrity sha512-vkcDPrRZo1QZLbn5RLGPpg/WmIQ65qoWWhcGKf/b5eplkkarX0m9z8ppCat4mlOqUsWpyNuYgO3VRyrYHSzX5g== + +prettier-plugin-organize-imports@^3.2.3: + version "3.2.3" + resolved "https://registry.yarnpkg.com/prettier-plugin-organize-imports/-/prettier-plugin-organize-imports-3.2.3.tgz#6b0141ac71f7ee9a673ce83e95456319e3a7cf0d" + integrity sha512-KFvk8C/zGyvUaE3RvxN2MhCLwzV6OBbFSkwZ2OamCrs9ZY4i5L77jQ/w4UmUr+lqX8qbaqVq6bZZkApn+IgJSg== + +prettier@^2.8.0: + version "2.8.8" + resolved "https://registry.yarnpkg.com/prettier/-/prettier-2.8.8.tgz#e8c5d7e98a4305ffe3de2e1fc4aca1a71c28b1da" + integrity sha512-tdN8qQGvNjw4CHbY+XXk0JgCXn9QiF21a55rBe5LJAU+kDyC4WQn4+awm2Xfk2lQMk5fKup9XgzTZtGkjBdP9Q== + +prettier@^3.0.1: + version "3.0.3" + resolved "https://registry.yarnpkg.com/prettier/-/prettier-3.0.3.tgz#432a51f7ba422d1469096c0fdc28e235db8f9643" + integrity sha512-L/4pUDMxcNa8R/EthV08Zt42WBO4h1rarVtK0K+QJG0X187OLo7l699jWw0GKuwzkPQ//jMFA/8Xm6Fh3J/DAg== + +pretty-error@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/pretty-error/-/pretty-error-4.0.0.tgz#90a703f46dd7234adb46d0f84823e9d1cb8f10d6" + integrity sha512-AoJ5YMAcXKYxKhuJGdcvse+Voc6v1RgnsR3nWcYU7q4t6z0Q6T86sv5Zq8VIRbOWWFpvdGE83LtdSMNd+6Y0xw== + dependencies: + lodash "^4.17.20" + renderkid "^3.0.0" + +pretty-format@^27.0.2: + version "27.5.1" + resolved "https://registry.yarnpkg.com/pretty-format/-/pretty-format-27.5.1.tgz#2181879fdea51a7a5851fb39d920faa63f01d88e" + integrity sha512-Qb1gy5OrP5+zDf2Bvnzdl3jsTf1qXVMazbvCoKhtKqVs4/YK4ozX4gKQJJVyNe+cajNPn0KoC0MC3FUmaHWEmQ== + dependencies: + ansi-regex "^5.0.1" + ansi-styles "^5.0.0" + react-is "^17.0.1" + +pretty-hrtime@^1.0.3: + version "1.0.3" + resolved "https://registry.yarnpkg.com/pretty-hrtime/-/pretty-hrtime-1.0.3.tgz#b7e3ea42435a4c9b2759d99e0f201eb195802ee1" + integrity sha512-66hKPCr+72mlfiSjlEB1+45IjXSqvVAIy6mocupoww4tBFE9R9IhwwUGoI4G++Tc9Aq+2rxOt0RFU6gPcrte0A== + +process-nextick-args@~2.0.0: + version "2.0.1" + resolved "https://registry.yarnpkg.com/process-nextick-args/-/process-nextick-args-2.0.1.tgz#7820d9b16120cc55ca9ae7792680ae7dba6d7fe2" + integrity sha512-3ouUOpQhtgrbOa17J7+uxOTpITYWaGP7/AhoR3+A+/1e9skrzelGi/dXzEYyvbxubEF6Wn2ypscTKiKJFFn1ag== + +process@^0.11.10: + version "0.11.10" + resolved "https://registry.yarnpkg.com/process/-/process-0.11.10.tgz#7332300e840161bda3e69a1d1d91a7d4bc16f182" + integrity sha512-cdGef/drWFoydD1JsMzuFf8100nZl+GT+yacc2bEced5f9Rjk4z+WtFUTBu9PhOi9j/jfmBPu0mMEY4wIdAF8A== + +progress@^2.0.1: + version "2.0.3" + resolved "https://registry.yarnpkg.com/progress/-/progress-2.0.3.tgz#7e8cf8d8f5b8f239c1bc68beb4eb78567d572ef8" + integrity sha512-7PiHtLll5LdnKIMw100I+8xJXR5gW2QwWYkT6iJva0bXitZKa/XMrSbdmg3r2Xnaidz9Qumd0VPaMrZlF9V9sA== + +prompts@^2.4.0: + version "2.4.2" + resolved "https://registry.yarnpkg.com/prompts/-/prompts-2.4.2.tgz#7b57e73b3a48029ad10ebd44f74b01722a4cb069" + integrity sha512-NxNv/kLguCA7p3jE8oL2aEBsrJWgAakBpgmgK6lpPWV+WuOmY6r2/zbAVnP+T8bQlA0nzHXSJSJW0Hq7ylaD2Q== + dependencies: + kleur "^3.0.3" + sisteransi "^1.0.5" + +prop-types@^15.6.2, prop-types@^15.7.2, prop-types@^15.8.1: + version "15.8.1" + resolved "https://registry.yarnpkg.com/prop-types/-/prop-types-15.8.1.tgz#67d87bf1a694f48435cf332c24af10214a3140b5" + integrity sha512-oj87CgZICdulUohogVAR7AjlC0327U4el4L6eAvOqCeudMDVU0NThNaV+b9Df4dXgSP1gXMTnPdhfe/2qDH5cg== + dependencies: + loose-envify "^1.4.0" + object-assign "^4.1.1" + react-is "^16.13.1" + +protobufjs@^7.2.4, protobufjs@^7.2.5: + version "7.2.5" + resolved "https://registry.yarnpkg.com/protobufjs/-/protobufjs-7.2.5.tgz#45d5c57387a6d29a17aab6846dcc283f9b8e7f2d" + integrity sha512-gGXRSXvxQ7UiPgfw8gevrfRWcTlSbOFg+p/N+JVJEK5VhueL2miT6qTymqAmjr1Q5WbOCyJbyrk6JfWKwlFn6A== + dependencies: + "@protobufjs/aspromise" "^1.1.2" + "@protobufjs/base64" "^1.1.2" + "@protobufjs/codegen" "^2.0.4" + "@protobufjs/eventemitter" "^1.1.0" + "@protobufjs/fetch" "^1.1.0" + "@protobufjs/float" "^1.0.2" + "@protobufjs/inquire" "^1.1.0" + "@protobufjs/path" "^1.1.2" + "@protobufjs/pool" "^1.1.0" + "@protobufjs/utf8" "^1.1.0" + "@types/node" ">=13.7.0" + long "^5.0.0" + +proxy-addr@~2.0.7: + version "2.0.7" + resolved "https://registry.yarnpkg.com/proxy-addr/-/proxy-addr-2.0.7.tgz#f19fe69ceab311eeb94b42e70e8c2070f9ba1025" + integrity sha512-llQsMLSUDUPT44jdrU/O37qlnifitDP+ZwrmmZcoSKyLKvtZxpyV0n2/bD/N4tBAAZ/gJEdZU7KMraoK1+XYAg== + dependencies: + forwarded "0.2.0" + ipaddr.js "1.9.1" + +proxy-from-env@^1.0.0: + version "1.1.0" + resolved "https://registry.yarnpkg.com/proxy-from-env/-/proxy-from-env-1.1.0.tgz#e102f16ca355424865755d2c9e8ea4f24d58c3e2" + integrity sha512-D+zkORCbA9f1tdWRK0RaCR3GPv50cMxcrz4X8k5LTSUD1Dkw47mKJEZQNunItRTkWwgtaUSo1RVFRIG9ZXiFYg== + +prr@~1.0.1: + version "1.0.1" + resolved "https://registry.yarnpkg.com/prr/-/prr-1.0.1.tgz#d3fc114ba06995a45ec6893f484ceb1d78f5f476" + integrity sha512-yPw4Sng1gWghHQWj0B3ZggWUm4qVbPwPFcRG8KyxiU7J2OHFSoEHKS+EZ3fv5l1t9CyCiop6l/ZYeWbrgoQejw== + +public-encrypt@^4.0.0: + version "4.0.3" + resolved "https://registry.yarnpkg.com/public-encrypt/-/public-encrypt-4.0.3.tgz#4fcc9d77a07e48ba7527e7cbe0de33d0701331e0" + integrity sha512-zVpa8oKZSz5bTMTFClc1fQOnyyEzpl5ozpi1B5YcvBrdohMjH2rfsBtyXcuNuwjsDIXmBYlF2N5FlJYhR29t8Q== + dependencies: + bn.js "^4.1.0" + browserify-rsa "^4.0.0" + create-hash "^1.1.0" + parse-asn1 "^5.0.0" + randombytes "^2.0.1" + safe-buffer "^5.1.2" + +pump@^2.0.0: + version "2.0.1" + resolved "https://registry.yarnpkg.com/pump/-/pump-2.0.1.tgz#12399add6e4cf7526d973cbc8b5ce2e2908b3909" + integrity sha512-ruPMNRkN3MHP1cWJc9OWr+T/xDP0jhXYCLfJcBuX54hhfIBnaQmAUMfDcG4DM5UMWByBbJY69QSphm3jtDKIkA== + dependencies: + end-of-stream "^1.1.0" + once "^1.3.1" + +pump@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/pump/-/pump-3.0.0.tgz#b4a2116815bde2f4e1ea602354e8c75565107a64" + integrity sha512-LwZy+p3SFs1Pytd/jYct4wpv49HiYCqd9Rlc5ZVdk0V+8Yzv6jR5Blk3TRmPL1ft69TxP0IMZGJ+WPFU2BFhww== + dependencies: + end-of-stream "^1.1.0" + once "^1.3.1" + +pumpify@^1.3.3: + version "1.5.1" + resolved "https://registry.yarnpkg.com/pumpify/-/pumpify-1.5.1.tgz#36513be246ab27570b1a374a5ce278bfd74370ce" + integrity sha512-oClZI37HvuUJJxSKKrC17bZ9Cu0ZYhEAGPsPUy9KlMUmv9dKX2o77RUmq7f3XjIxbwyGwYzbzQ1L2Ks8sIradQ== + dependencies: + duplexify "^3.6.0" + inherits "^2.0.3" + pump "^2.0.0" + +punycode@^1.4.1: + version "1.4.1" + resolved "https://registry.yarnpkg.com/punycode/-/punycode-1.4.1.tgz#c0d5a63b2718800ad8e1eb0fa5269c84dd41845e" + integrity sha512-jmYNElW7yvO7TV33CjSmvSiE2yco3bV2czu/OzDKdMNVZQWfxCblURLhf+47syQRBntjfLdd/H0egrzIG+oaFQ== + +punycode@^2.1.0, punycode@^2.1.1: + version "2.3.0" + resolved "https://registry.yarnpkg.com/punycode/-/punycode-2.3.0.tgz#f67fa67c94da8f4d0cfff981aee4118064199b8f" + integrity sha512-rRV+zQD8tVFys26lAGR9WUuS4iUAngJScM+ZRSKtvl5tKeZ2t5bvdNFdNHBW9FWR4guGHlgmsZ1G7BSm2wTbuA== + +puppeteer-core@^2.1.1: + version "2.1.1" + resolved "https://registry.yarnpkg.com/puppeteer-core/-/puppeteer-core-2.1.1.tgz#e9b3fbc1237b4f66e25999832229e9db3e0b90ed" + integrity sha512-n13AWriBMPYxnpbb6bnaY5YoY6rGj8vPLrz6CZF3o0qJNEwlcfJVxBzYZ0NJsQ21UbdJoijPCDrM++SUVEz7+w== + dependencies: + "@types/mime-types" "^2.1.0" + debug "^4.1.0" + extract-zip "^1.6.6" + https-proxy-agent "^4.0.0" + mime "^2.0.3" + mime-types "^2.1.25" + progress "^2.0.1" + proxy-from-env "^1.0.0" + rimraf "^2.6.1" + ws "^6.1.0" + +qs@6.11.0: + version "6.11.0" + resolved "https://registry.yarnpkg.com/qs/-/qs-6.11.0.tgz#fd0d963446f7a65e1367e01abd85429453f0c37a" + integrity sha512-MvjoMCJwEarSbUYk5O+nmoSzSutSsTwF85zcHPQ9OrlFoZOYIjaqBAJIqIXjptyD5vThxGq52Xu/MaJzRkIk4Q== + dependencies: + side-channel "^1.0.4" + +qs@^6.10.0, qs@^6.11.0: + version "6.11.2" + resolved "https://registry.yarnpkg.com/qs/-/qs-6.11.2.tgz#64bea51f12c1f5da1bc01496f48ffcff7c69d7d9" + integrity sha512-tDNIz22aBzCDxLtVH++VnTfzxlfeK5CbqohpSqpJgj1Wg/cQbStNAz3NuqCs5vV+pjBsK4x4pN9HlVh7rcYRiA== + dependencies: + side-channel "^1.0.4" + +querystring-es3@^0.2.1: + version "0.2.1" + resolved "https://registry.yarnpkg.com/querystring-es3/-/querystring-es3-0.2.1.tgz#9ec61f79049875707d69414596fd907a4d711e73" + integrity sha512-773xhDQnZBMFobEiztv8LIl70ch5MSF/jUQVlhwFyBILqq96anmoctVIYz+ZRp0qbCKATTn6ev02M3r7Ga5vqA== + +queue-microtask@^1.2.2: + version "1.2.3" + resolved "https://registry.yarnpkg.com/queue-microtask/-/queue-microtask-1.2.3.tgz#4929228bbc724dfac43e0efb058caf7b6cfb6243" + integrity sha512-NuaNSa6flKT5JaSYQzJok04JzTL1CA6aGhv5rfLW3PgqA+M2ChpZQnAC8h8i4ZFkBS8X5RqkDBHA7r4hej3K9A== + +queue@6.0.2: + version "6.0.2" + resolved "https://registry.yarnpkg.com/queue/-/queue-6.0.2.tgz#b91525283e2315c7553d2efa18d83e76432fed65" + integrity sha512-iHZWu+q3IdFZFX36ro/lKBkSvfkztY5Y7HMiPlOUjhupPcG2JMfst2KKEpu5XndviX/3UhFbRngUPNKtgvtZiA== + dependencies: + inherits "~2.0.3" + +ramda@0.29.0: + version "0.29.0" + resolved "https://registry.yarnpkg.com/ramda/-/ramda-0.29.0.tgz#fbbb67a740a754c8a4cbb41e2a6e0eb8507f55fb" + integrity sha512-BBea6L67bYLtdbOqfp8f58fPMqEwx0doL+pAi8TZyp2YWz8R9G8z9x75CZI8W+ftqhFHCpEX2cRnUUXK130iKA== + +randombytes@^2.0.0, randombytes@^2.0.1, randombytes@^2.0.5, randombytes@^2.1.0: + version "2.1.0" + resolved "https://registry.yarnpkg.com/randombytes/-/randombytes-2.1.0.tgz#df6f84372f0270dc65cdf6291349ab7a473d4f2a" + integrity sha512-vYl3iOX+4CKUWuxGi9Ukhie6fsqXqS9FE2Zaic4tNFD2N2QQaXOMFbuKK4QmDHC0JO6B1Zp41J0LpT0oR68amQ== + dependencies: + safe-buffer "^5.1.0" + +randomfill@^1.0.3: + version "1.0.4" + resolved "https://registry.yarnpkg.com/randomfill/-/randomfill-1.0.4.tgz#c92196fc86ab42be983f1bf31778224931d61458" + integrity sha512-87lcbR8+MhcWcUiQ+9e+Rwx8MyR2P7qnt15ynUlbm3TU/fjbgz4GsvfSUDTemtCCtVCqb4ZcEFlyPNTh9bBTLw== + dependencies: + randombytes "^2.0.5" + safe-buffer "^5.1.0" + +range-parser@^1.2.1, range-parser@~1.2.1: + version "1.2.1" + resolved "https://registry.yarnpkg.com/range-parser/-/range-parser-1.2.1.tgz#3cf37023d199e1c24d1a55b84800c2f3e6468031" + integrity sha512-Hrgsx+orqoygnmhFbKaHE6c296J+HTAQXoxEF6gNupROmmGJRoyzfG3ccAveqCBrwr/2yxQ5BVd/GTl5agOwSg== + +raw-body@2.5.1: + version "2.5.1" + resolved "https://registry.yarnpkg.com/raw-body/-/raw-body-2.5.1.tgz#fe1b1628b181b700215e5fd42389f98b71392857" + integrity sha512-qqJBtEyVgS0ZmPGdCFPWJ3FreoqvG4MVQln/kCgF7Olq95IbOp0/BWyMwbdtn4VTvkM8Y7khCQ2Xgk/tcrCXig== + dependencies: + bytes "3.1.2" + http-errors "2.0.0" + iconv-lite "0.4.24" + unpipe "1.0.0" + +react-colorful@^5.1.2: + version "5.6.1" + resolved "https://registry.yarnpkg.com/react-colorful/-/react-colorful-5.6.1.tgz#7dc2aed2d7c72fac89694e834d179e32f3da563b" + integrity sha512-1exovf0uGTGyq5mXQT0zgQ80uvj2PCwvF8zY1RN9/vbJVSjSo3fsB/4L3ObbF7u70NduSiK4xu4Y6q1MHoUGEw== + +react-day-picker@^8.7.1: + version "8.8.1" + resolved "https://registry.yarnpkg.com/react-day-picker/-/react-day-picker-8.8.1.tgz#44474c6f7f346ea28796974d7a065d5572f4cd38" + integrity sha512-U7RsRoRI5pyMXhKq54hS9yM11WEGkPf8hIdrxIM/sefgmQjuxazqgwcZFMiPZW/K9vtmzLZFf9bLW0wVsGYd5w== + +react-docgen-typescript@^2.2.2: + version "2.2.2" + resolved "https://registry.yarnpkg.com/react-docgen-typescript/-/react-docgen-typescript-2.2.2.tgz#4611055e569edc071204aadb20e1c93e1ab1659c" + integrity sha512-tvg2ZtOpOi6QDwsb3GZhOjDkkX0h8Z2gipvTg6OVMUyoYoURhEiRNePT8NZItTVCDh39JJHnLdfCOkzoLbFnTg== + +react-docgen@^5.0.0: + version "5.4.3" + resolved "https://registry.yarnpkg.com/react-docgen/-/react-docgen-5.4.3.tgz#7d297f73b977d0c7611402e5fc2a168acf332b26" + integrity sha512-xlLJyOlnfr8lLEEeaDZ+X2J/KJoe6Nr9AzxnkdQWush5hz2ZSu66w6iLMOScMmxoSHWpWMn+k3v5ZiyCfcWsOA== + dependencies: + "@babel/core" "^7.7.5" + "@babel/generator" "^7.12.11" + "@babel/runtime" "^7.7.6" + ast-types "^0.14.2" + commander "^2.19.0" + doctrine "^3.0.0" + estree-to-babel "^3.1.0" + neo-async "^2.6.1" + node-dir "^0.1.10" + strip-indent "^3.0.0" + +react-dom@18.2.0: + version "18.2.0" + resolved "https://registry.yarnpkg.com/react-dom/-/react-dom-18.2.0.tgz#22aaf38708db2674ed9ada224ca4aa708d821e3d" + integrity sha512-6IMTriUmvsjHUjNtEDudZfuDQUoWXVxKHhlEGSk81n4YFS+r/Kl99wXiwlVXtPBtJenozv2P+hxDsw9eA7Xo6g== + dependencies: + loose-envify "^1.1.0" + scheduler "^0.23.0" + +react-element-to-jsx-string@^15.0.0: + version "15.0.0" + resolved "https://registry.yarnpkg.com/react-element-to-jsx-string/-/react-element-to-jsx-string-15.0.0.tgz#1cafd5b6ad41946ffc8755e254da3fc752a01ac6" + integrity sha512-UDg4lXB6BzlobN60P8fHWVPX3Kyw8ORrTeBtClmIlGdkOOE+GYQSFvmEU5iLLpwp/6v42DINwNcwOhOLfQ//FQ== + dependencies: + "@base2/pretty-print-object" "1.0.1" + is-plain-object "5.0.0" + react-is "18.1.0" + +react-inspector@^6.0.0: + version "6.0.2" + resolved "https://registry.yarnpkg.com/react-inspector/-/react-inspector-6.0.2.tgz#aa3028803550cb6dbd7344816d5c80bf39d07e9d" + integrity sha512-x+b7LxhmHXjHoU/VrFAzw5iutsILRoYyDq97EDYdFpPLcvqtEzk4ZSZSQjnFPbr5T57tLXnHcqFYoN1pI6u8uQ== + +react-is@18.1.0: + version "18.1.0" + resolved "https://registry.yarnpkg.com/react-is/-/react-is-18.1.0.tgz#61aaed3096d30eacf2a2127118b5b41387d32a67" + integrity sha512-Fl7FuabXsJnV5Q1qIOQwx/sagGF18kogb4gpfcG4gjLBWO0WDiiz1ko/ExayuxE7InyQkBLkxRFG5oxY6Uu3Kg== + +react-is@^16.10.2, react-is@^16.13.1: + version "16.13.1" + resolved "https://registry.yarnpkg.com/react-is/-/react-is-16.13.1.tgz#789729a4dc36de2999dc156dd6c1d9c18cea56a4" + integrity sha512-24e6ynE2H+OKt4kqsOvNd8kBpV65zoxbA4BVsEOB3ARVWQki/DHzaUoC5KuON/BiccDaCCTZBuOcfZs70kR8bQ== + +react-is@^17.0.1: + version "17.0.2" + resolved "https://registry.yarnpkg.com/react-is/-/react-is-17.0.2.tgz#e691d4a8e9c789365655539ab372762b0efb54f0" + integrity sha512-w2GsyukL62IJnlaff/nRegPQR94C/XXamvMWmSHRJ4y7Ts/4ocGRmTHvOs8PSE6pB3dWOrD/nueuU5sduBsQ4w== + +react-lifecycles-compat@^3.0.4: + version "3.0.4" + resolved "https://registry.yarnpkg.com/react-lifecycles-compat/-/react-lifecycles-compat-3.0.4.tgz#4f1a273afdfc8f3488a8c516bfda78f872352362" + integrity sha512-fBASbA6LnOU9dOU2eW7aQ8xmYBSXUIWr+UmF9b1efZBazGNO+rcXT/icdKnYm2pTwcRylVUYwW7H1PHfLekVzA== + +react-refresh@^0.11.0: + version "0.11.0" + resolved "https://registry.yarnpkg.com/react-refresh/-/react-refresh-0.11.0.tgz#77198b944733f0f1f1a90e791de4541f9f074046" + integrity sha512-F27qZr8uUqwhWZboondsPx8tnC3Ct3SxZA3V5WyEvujRyyNv0VYPhoBg1gZ8/MV5tubQp76Trw8lTv9hzRBa+A== + +react-remove-scroll-bar@^2.3.3: + version "2.3.4" + resolved "https://registry.yarnpkg.com/react-remove-scroll-bar/-/react-remove-scroll-bar-2.3.4.tgz#53e272d7a5cb8242990c7f144c44d8bd8ab5afd9" + integrity sha512-63C4YQBUt0m6ALadE9XV56hV8BgJWDmmTPY758iIJjfQKt2nYwoUrPk0LXRXcB/yIj82T1/Ixfdpdk68LwIB0A== + dependencies: + react-style-singleton "^2.2.1" + tslib "^2.0.0" + +react-remove-scroll@2.5.5: + version "2.5.5" + resolved "https://registry.yarnpkg.com/react-remove-scroll/-/react-remove-scroll-2.5.5.tgz#1e31a1260df08887a8a0e46d09271b52b3a37e77" + integrity sha512-ImKhrzJJsyXJfBZ4bzu8Bwpka14c/fQt0k+cyFp/PBhTfyDnU5hjOtM4AG/0AMyy8oKzOTR0lDgJIM7pYXI0kw== + dependencies: + react-remove-scroll-bar "^2.3.3" + react-style-singleton "^2.2.1" + tslib "^2.1.0" + use-callback-ref "^1.3.0" + use-sidecar "^1.1.2" + +react-resize-detector@^8.0.4: + version "8.1.0" + resolved "https://registry.yarnpkg.com/react-resize-detector/-/react-resize-detector-8.1.0.tgz#1c7817db8bc886e2dbd3fbe3b26ea8e56be0524a" + integrity sha512-S7szxlaIuiy5UqLhLL1KY3aoyGHbZzsTpYal9eYMwCyKqoqoVLCmIgAgNyIM1FhnP2KyBygASJxdhejrzjMb+w== + dependencies: + lodash "^4.17.21" + +react-smooth@^2.0.2: + version "2.0.3" + resolved "https://registry.yarnpkg.com/react-smooth/-/react-smooth-2.0.3.tgz#2845fa8f22914f2e4445856d5688fb8a7d72f3ae" + integrity sha512-yl4y3XiMorss7ayF5QnBiSprig0+qFHui8uh7Hgg46QX5O+aRMRKlfGGNGLHno35JkQSvSYY8eCWkBfHfrSHfg== + dependencies: + fast-equals "^5.0.0" + react-transition-group "2.9.0" + +react-style-singleton@^2.2.1: + version "2.2.1" + resolved "https://registry.yarnpkg.com/react-style-singleton/-/react-style-singleton-2.2.1.tgz#f99e420492b2d8f34d38308ff660b60d0b1205b4" + integrity sha512-ZWj0fHEMyWkHzKYUr2Bs/4zU6XLmq9HsgBURm7g5pAVfyn49DgUiNgY2d4lXRlYSiCif9YBGpQleewkcqddc7g== + dependencies: + get-nonce "^1.0.0" + invariant "^2.2.4" + tslib "^2.0.0" + +react-transition-group@2.9.0: + version "2.9.0" + resolved "https://registry.yarnpkg.com/react-transition-group/-/react-transition-group-2.9.0.tgz#df9cdb025796211151a436c69a8f3b97b5b07c8d" + integrity sha512-+HzNTCHpeQyl4MJ/bdE0u6XRMe9+XG/+aL4mCxVN4DnPBQ0/5bfHWPDuOZUzYdMj94daZaZdCCc1Dzt9R/xSSg== + dependencies: + dom-helpers "^3.4.0" + loose-envify "^1.4.0" + prop-types "^15.6.2" + react-lifecycles-compat "^3.0.4" + +react-transition-group@^4.4.5: + version "4.4.5" + resolved "https://registry.yarnpkg.com/react-transition-group/-/react-transition-group-4.4.5.tgz#e53d4e3f3344da8521489fbef8f2581d42becdd1" + integrity sha512-pZcd1MCJoiKiBR2NRxeCRg13uCXbydPnmB4EOeRrY7480qNWO8IIgQG6zlDkm6uRMsURXPuKq0GWtiM59a5Q6g== + dependencies: + "@babel/runtime" "^7.5.5" + dom-helpers "^5.0.1" + loose-envify "^1.4.0" + prop-types "^15.6.2" + +react@18.2.0: + version "18.2.0" + resolved "https://registry.yarnpkg.com/react/-/react-18.2.0.tgz#555bd98592883255fa00de14f1151a917b5d77d5" + integrity sha512-/3IjMdb2L9QbBdWiW5e3P2/npwMBaU9mHCSCUzNln0ZCYbcfTsGbTJrU/kGemdH2IWmB2ioZ+zkxtmq6g09fGQ== + dependencies: + loose-envify "^1.1.0" + +read-cache@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/read-cache/-/read-cache-1.0.0.tgz#e664ef31161166c9751cdbe8dbcf86b5fb58f774" + integrity sha512-Owdv/Ft7IjOgm/i0xvNDZ1LrRANRfew4b2prF3OWMQLxLfu3bS8FVhCsrSCMK4lR56Y9ya+AThoTpDCTxCmpRA== + dependencies: + pify "^2.3.0" + +read-pkg-up@^7.0.1: + version "7.0.1" + resolved "https://registry.yarnpkg.com/read-pkg-up/-/read-pkg-up-7.0.1.tgz#f3a6135758459733ae2b95638056e1854e7ef507" + integrity sha512-zK0TB7Xd6JpCLmlLmufqykGE+/TlOePD6qKClNW7hHDKFh/J7/7gCWGR7joEQEW1bKq3a3yUZSObOoWLFQ4ohg== + dependencies: + find-up "^4.1.0" + read-pkg "^5.2.0" + type-fest "^0.8.1" + +read-pkg@^5.2.0: + version "5.2.0" + resolved "https://registry.yarnpkg.com/read-pkg/-/read-pkg-5.2.0.tgz#7bf295438ca5a33e56cd30e053b34ee7250c93cc" + integrity sha512-Ug69mNOpfvKDAc2Q8DRpMjjzdtrnv9HcSMX+4VsZxD1aZ6ZzrIE7rlzXBtWTyhULSMKg076AW6WR5iZpD0JiOg== + dependencies: + "@types/normalize-package-data" "^2.4.0" + normalize-package-data "^2.5.0" + parse-json "^5.0.0" + type-fest "^0.6.0" + +readable-stream@^2.0.0, readable-stream@^2.2.2, readable-stream@~2.3.6: + version "2.3.8" + resolved "https://registry.yarnpkg.com/readable-stream/-/readable-stream-2.3.8.tgz#91125e8042bba1b9887f49345f6277027ce8be9b" + integrity sha512-8p0AUk4XODgIewSi0l8Epjs+EVnWiK7NoDIEGU0HhE7+ZyY8D1IMY7odu5lRrFXGg71L15KG8QrPmum45RTtdA== + dependencies: + core-util-is "~1.0.0" + inherits "~2.0.3" + isarray "~1.0.0" + process-nextick-args "~2.0.0" + safe-buffer "~5.1.1" + string_decoder "~1.1.1" + util-deprecate "~1.0.1" + +readable-stream@^3.1.1, readable-stream@^3.4.0, readable-stream@^3.5.0, readable-stream@^3.6.0: + version "3.6.2" + resolved "https://registry.yarnpkg.com/readable-stream/-/readable-stream-3.6.2.tgz#56a9b36ea965c00c5a93ef31eb111a0f11056967" + integrity sha512-9u/sniCrY3D5WdsERHzHE4G2YCXqoG5FTHUiCC4SIbr6XcLZBY05ya9EKjYek9O5xOAwjGq+1JdGBAS7Q9ScoA== + dependencies: + inherits "^2.0.3" + string_decoder "^1.1.1" + util-deprecate "^1.0.1" + +readable-stream@^4.0.0: + version "4.4.2" + resolved "https://registry.yarnpkg.com/readable-stream/-/readable-stream-4.4.2.tgz#e6aced27ad3b9d726d8308515b9a1b98dc1b9d13" + integrity sha512-Lk/fICSyIhodxy1IDK2HazkeGjSmezAWX2egdtJnYhtzKEsBPJowlI6F6LPb5tqIQILrMbx22S5o3GuJavPusA== + dependencies: + abort-controller "^3.0.0" + buffer "^6.0.3" + events "^3.3.0" + process "^0.11.10" + string_decoder "^1.3.0" + +readdirp@~3.6.0: + version "3.6.0" + resolved "https://registry.yarnpkg.com/readdirp/-/readdirp-3.6.0.tgz#74a370bd857116e245b29cc97340cd431a02a6c7" + integrity sha512-hOS089on8RduqdbhvQ5Z37A0ESjsqz6qnRcffsMU3495FuTdqSm+7bhJ29JvIOsBDEEnan5DPu9t3To9VRlMzA== + dependencies: + picomatch "^2.2.1" + +recast@^0.21.0: + version "0.21.5" + resolved "https://registry.yarnpkg.com/recast/-/recast-0.21.5.tgz#e8cd22bb51bcd6130e54f87955d33a2b2e57b495" + integrity sha512-hjMmLaUXAm1hIuTqOdeYObMslq/q+Xff6QE3Y2P+uoHAg2nmVlLBps2hzh1UJDdMtDTMXOFewK6ky51JQIeECg== + dependencies: + ast-types "0.15.2" + esprima "~4.0.0" + source-map "~0.6.1" + tslib "^2.0.1" + +recast@^0.23.1: + version "0.23.4" + resolved "https://registry.yarnpkg.com/recast/-/recast-0.23.4.tgz#ca1bac7bfd3011ea5a28dfecb5df678559fb1ddf" + integrity sha512-qtEDqIZGVcSZCHniWwZWbRy79Dc6Wp3kT/UmDA2RJKBPg7+7k51aQBZirHmUGn5uvHf2rg8DkjizrN26k61ATw== + dependencies: + assert "^2.0.0" + ast-types "^0.16.1" + esprima "~4.0.0" + source-map "~0.6.1" + tslib "^2.0.1" + +recharts-scale@^0.4.4: + version "0.4.5" + resolved "https://registry.yarnpkg.com/recharts-scale/-/recharts-scale-0.4.5.tgz#0969271f14e732e642fcc5bd4ab270d6e87dd1d9" + integrity sha512-kivNFO+0OcUNu7jQquLXAxz1FIwZj8nrj+YkOKc5694NbjCvcT6aSZiIzNzd2Kul4o4rTto8QVR9lMNtxD4G1w== + dependencies: + decimal.js-light "^2.4.1" + +recharts@^2.7.1: + version "2.8.0" + resolved "https://registry.yarnpkg.com/recharts/-/recharts-2.8.0.tgz#90c95136e2cb6930224c94a51adce607701284fc" + integrity sha512-nciXqQDh3aW8abhwUlA4EBOBusRHLNiKHfpRZiG/yjups1x+auHb2zWPuEcTn/IMiN47vVMMuF8Sr+vcQJtsmw== + dependencies: + classnames "^2.2.5" + eventemitter3 "^4.0.1" + lodash "^4.17.19" + react-is "^16.10.2" + react-resize-detector "^8.0.4" + react-smooth "^2.0.2" + recharts-scale "^0.4.4" + reduce-css-calc "^2.1.8" + victory-vendor "^36.6.8" + +reduce-css-calc@^2.1.8: + version "2.1.8" + resolved "https://registry.yarnpkg.com/reduce-css-calc/-/reduce-css-calc-2.1.8.tgz#7ef8761a28d614980dc0c982f772c93f7a99de03" + integrity sha512-8liAVezDmUcH+tdzoEGrhfbGcP7nOV4NkGE3a74+qqvE7nt9i4sKLGBuZNOnpI4WiGksiNPklZxva80061QiPg== + dependencies: + css-unit-converter "^1.1.1" + postcss-value-parser "^3.3.0" + +reflect.getprototypeof@^1.0.3: + version "1.0.4" + resolved "https://registry.yarnpkg.com/reflect.getprototypeof/-/reflect.getprototypeof-1.0.4.tgz#aaccbf41aca3821b87bb71d9dcbc7ad0ba50a3f3" + integrity sha512-ECkTw8TmJwW60lOTR+ZkODISW6RQ8+2CL3COqtiJKLd6MmB45hN51HprHFziKLGkAuTGQhBb91V8cy+KHlaCjw== + dependencies: + call-bind "^1.0.2" + define-properties "^1.2.0" + es-abstract "^1.22.1" + get-intrinsic "^1.2.1" + globalthis "^1.0.3" + which-builtin-type "^1.1.3" + +regenerate-unicode-properties@^10.1.0: + version "10.1.0" + resolved "https://registry.yarnpkg.com/regenerate-unicode-properties/-/regenerate-unicode-properties-10.1.0.tgz#7c3192cab6dd24e21cb4461e5ddd7dd24fa8374c" + integrity sha512-d1VudCLoIGitcU/hEg2QqvyGZQmdC0Lf8BqdOMXGFSvJP4bNV1+XqbPQeHHLD51Jh4QJJ225dlIFvY4Ly6MXmQ== + dependencies: + regenerate "^1.4.2" + +regenerate@^1.4.2: + version "1.4.2" + resolved "https://registry.yarnpkg.com/regenerate/-/regenerate-1.4.2.tgz#b9346d8827e8f5a32f7ba29637d398b69014848a" + integrity sha512-zrceR/XhGYU/d/opr2EKO7aRHUeiBI8qjtfHqADTwZd6Szfy16la6kqD0MIUs5z5hx6AaKa+PixpPrR289+I0A== + +regenerator-runtime@^0.14.0: + version "0.14.0" + resolved "https://registry.yarnpkg.com/regenerator-runtime/-/regenerator-runtime-0.14.0.tgz#5e19d68eb12d486f797e15a3c6a918f7cec5eb45" + integrity sha512-srw17NI0TUWHuGa5CFGGmhfNIeja30WMBfbslPNhf6JrqQlLN5gcrvig1oqPxiVaXb0oW0XRKtH6Nngs5lKCIA== + +regenerator-transform@^0.15.2: + version "0.15.2" + resolved "https://registry.yarnpkg.com/regenerator-transform/-/regenerator-transform-0.15.2.tgz#5bbae58b522098ebdf09bca2f83838929001c7a4" + integrity sha512-hfMp2BoF0qOk3uc5V20ALGDS2ddjQaLrdl7xrGXvAIow7qeWRM2VA2HuCHkUKk9slq3VwEwLNK3DFBqDfPGYtg== + dependencies: + "@babel/runtime" "^7.8.4" + +regex-parser@^2.2.11: + version "2.2.11" + resolved "https://registry.yarnpkg.com/regex-parser/-/regex-parser-2.2.11.tgz#3b37ec9049e19479806e878cabe7c1ca83ccfe58" + integrity sha512-jbD/FT0+9MBU2XAZluI7w2OBs1RBi6p9M83nkoZayQXXU9e8Robt69FcZc7wU4eJD/YFTjn1JdCk3rbMJajz8Q== + +regexp.prototype.flags@^1.5.0: + version "1.5.0" + resolved "https://registry.yarnpkg.com/regexp.prototype.flags/-/regexp.prototype.flags-1.5.0.tgz#fe7ce25e7e4cca8db37b6634c8a2c7009199b9cb" + integrity sha512-0SutC3pNudRKgquxGoRGIz946MZVHqbNfPjBdxeOhBrdgDKlRoXmYLQN9xRbrR09ZXWeGAdPuif7egofn6v5LA== + dependencies: + call-bind "^1.0.2" + define-properties "^1.2.0" + functions-have-names "^1.2.3" + +regexpu-core@^5.3.1: + version "5.3.2" + resolved "https://registry.yarnpkg.com/regexpu-core/-/regexpu-core-5.3.2.tgz#11a2b06884f3527aec3e93dbbf4a3b958a95546b" + integrity sha512-RAM5FlZz+Lhmo7db9L298p2vHP5ZywrVXmVXpmAD9GuL5MPH6t9ROw1iA/wfHkQ76Qe7AaPF0nGuim96/IrQMQ== + dependencies: + "@babel/regjsgen" "^0.8.0" + regenerate "^1.4.2" + regenerate-unicode-properties "^10.1.0" + regjsparser "^0.9.1" + unicode-match-property-ecmascript "^2.0.0" + unicode-match-property-value-ecmascript "^2.1.0" + +regjsparser@^0.9.1: + version "0.9.1" + resolved "https://registry.yarnpkg.com/regjsparser/-/regjsparser-0.9.1.tgz#272d05aa10c7c1f67095b1ff0addae8442fc5709" + integrity sha512-dQUtn90WanSNl+7mQKcXAgZxvUe7Z0SqXlgzv0za4LwiUhyzBC58yQO3liFoUgu8GiJVInAhJjkj1N0EtQ5nkQ== + dependencies: + jsesc "~0.5.0" + +relateurl@^0.2.7: + version "0.2.7" + resolved "https://registry.yarnpkg.com/relateurl/-/relateurl-0.2.7.tgz#54dbf377e51440aca90a4cd274600d3ff2d888a9" + integrity sha512-G08Dxvm4iDN3MLM0EsP62EDV9IuhXPR6blNz6Utcp7zyV3tr4HVNINt6MpaRWbxoOHT3Q7YN2P+jaHX8vUbgog== + +remark-external-links@^8.0.0: + version "8.0.0" + resolved "https://registry.yarnpkg.com/remark-external-links/-/remark-external-links-8.0.0.tgz#308de69482958b5d1cd3692bc9b725ce0240f345" + integrity sha512-5vPSX0kHoSsqtdftSHhIYofVINC8qmp0nctkeU9YoJwV3YfiBRiI6cbFRJ0oI/1F9xS+bopXG0m2KS8VFscuKA== + dependencies: + extend "^3.0.0" + is-absolute-url "^3.0.0" + mdast-util-definitions "^4.0.0" + space-separated-tokens "^1.0.0" + unist-util-visit "^2.0.0" + +remark-slug@^6.0.0: + version "6.1.0" + resolved "https://registry.yarnpkg.com/remark-slug/-/remark-slug-6.1.0.tgz#0503268d5f0c4ecb1f33315c00465ccdd97923ce" + integrity sha512-oGCxDF9deA8phWvxFuyr3oSJsdyUAxMFbA0mZ7Y1Sas+emILtO+e5WutF9564gDsEN4IXaQXm5pFo6MLH+YmwQ== + dependencies: + github-slugger "^1.0.0" + mdast-util-to-string "^1.0.0" + unist-util-visit "^2.0.0" + +renderkid@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/renderkid/-/renderkid-3.0.0.tgz#5fd823e4d6951d37358ecc9a58b1f06836b6268a" + integrity sha512-q/7VIQA8lmM1hF+jn+sFSPWGlMkSAeNYcPLmDQx2zzuiDfaLrOmumR8iaUKlenFgh0XRPIUeSPlH3A+AW3Z5pg== + dependencies: + css-select "^4.1.3" + dom-converter "^0.2.0" + htmlparser2 "^6.1.0" + lodash "^4.17.21" + strip-ansi "^6.0.1" + +require-directory@^2.1.1: + version "2.1.1" + resolved "https://registry.yarnpkg.com/require-directory/-/require-directory-2.1.1.tgz#8c64ad5fd30dab1c976e2344ffe7f792a6a6df42" + integrity sha512-fGxEI7+wsG9xrvdjsrlmL22OMTTiHRwAMroiEeMgq8gzoLC/PQr7RsRDSTLUg/bZAZtF+TVIkHc6/4RIKrui+Q== + +require-from-string@^2.0.2: + version "2.0.2" + resolved "https://registry.yarnpkg.com/require-from-string/-/require-from-string-2.0.2.tgz#89a7fdd938261267318eafe14f9c32e598c36909" + integrity sha512-Xf0nWe6RseziFMu+Ap9biiUbmplq6S9/p+7w7YXP/JBHhrUDDUhwa+vANyubuqfZWTveU//DYVGsDG7RKL/vEw== + +requireindex@^1.1.0: + version "1.2.0" + resolved "https://registry.yarnpkg.com/requireindex/-/requireindex-1.2.0.tgz#3463cdb22ee151902635aa6c9535d4de9c2ef1ef" + integrity sha512-L9jEkOi3ASd9PYit2cwRfyppc9NoABujTP8/5gFcbERmo5jUoAKovIC3fsF17pkTnGsrByysqX+Kxd2OTNI1ww== + +resolve-from@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/resolve-from/-/resolve-from-4.0.0.tgz#4abcd852ad32dd7baabfe9b40e00a36db5f392e6" + integrity sha512-pb/MYmXstAkysRFx8piNI1tGFNQIFA3vkE3Gq4EuA1dF6gHp/+vgZqsCGJapvy8N3Q+4o7FwvquPJcnZ7RYy4g== + +resolve-from@^5.0.0: + version "5.0.0" + resolved "https://registry.yarnpkg.com/resolve-from/-/resolve-from-5.0.0.tgz#c35225843df8f776df21c57557bc087e9dfdfc69" + integrity sha512-qYg9KP24dD5qka9J47d0aVky0N+b4fTU89LN9iDnjB5waksiC49rvMB0PrUJQGoTmH50XPiqOvAjDfaijGxYZw== + +resolve-pkg-maps@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/resolve-pkg-maps/-/resolve-pkg-maps-1.0.0.tgz#616b3dc2c57056b5588c31cdf4b3d64db133720f" + integrity sha512-seS2Tj26TBVOC2NIc2rOe2y2ZO7efxITtLZcGSOnHHNOQ7CkiUBfw0Iw2ck6xkIhPwLhKNLS8BO+hEpngQlqzw== + +resolve-url-loader@^5.0.0: + version "5.0.0" + resolved "https://registry.yarnpkg.com/resolve-url-loader/-/resolve-url-loader-5.0.0.tgz#ee3142fb1f1e0d9db9524d539cfa166e9314f795" + integrity sha512-uZtduh8/8srhBoMx//5bwqjQ+rfYOUq8zC9NrMUGtjBiGTtFJM42s58/36+hTqeqINcnYe08Nj3LkK9lW4N8Xg== + dependencies: + adjust-sourcemap-loader "^4.0.0" + convert-source-map "^1.7.0" + loader-utils "^2.0.0" + postcss "^8.2.14" + source-map "0.6.1" + +resolve@^1.1.7, resolve@^1.10.0, resolve@^1.14.2, resolve@^1.22.2, resolve@^1.22.4: + version "1.22.4" + resolved "https://registry.yarnpkg.com/resolve/-/resolve-1.22.4.tgz#1dc40df46554cdaf8948a486a10f6ba1e2026c34" + integrity sha512-PXNdCiPqDqeUou+w1C2eTQbNfxKSuMxqTCuvlmmMsk1NWHL5fRrhY6Pl0qEYYc6+QqGClco1Qj8XnjPego4wfg== + dependencies: + is-core-module "^2.13.0" + path-parse "^1.0.7" + supports-preserve-symlinks-flag "^1.0.0" + +resolve@^2.0.0-next.4: + version "2.0.0-next.4" + resolved "https://registry.yarnpkg.com/resolve/-/resolve-2.0.0-next.4.tgz#3d37a113d6429f496ec4752d2a2e58efb1fd4660" + integrity sha512-iMDbmAWtfU+MHpxt/I5iWI7cY6YVEZUQ3MBgPQ++XD1PELuJHIl82xBmObyP2KyQmkNB2dsqF7seoQQiAn5yDQ== + dependencies: + is-core-module "^2.9.0" + path-parse "^1.0.7" + supports-preserve-symlinks-flag "^1.0.0" + +restore-cursor@^3.1.0: + version "3.1.0" + resolved "https://registry.yarnpkg.com/restore-cursor/-/restore-cursor-3.1.0.tgz#39f67c54b3a7a58cea5236d95cf0034239631f7e" + integrity sha512-l+sSefzHpj5qimhFSE5a8nufZYAM3sBSVMAPtYkmC+4EH2anSGaEMXSD0izRQbu9nfyQ9y5JrVmp7E8oZrUjvA== + dependencies: + onetime "^5.1.0" + signal-exit "^3.0.2" + +reusify@^1.0.4: + version "1.0.4" + resolved "https://registry.yarnpkg.com/reusify/-/reusify-1.0.4.tgz#90da382b1e126efc02146e90845a88db12925d76" + integrity sha512-U9nH88a3fc/ekCF1l0/UP1IosiuIjyTh7hBvXVMHYgVcfGvt897Xguj2UOLDeI5BG2m7/uwyaLVT6fbtCwTyzw== + +rimraf@^2.6.1: + version "2.7.1" + resolved "https://registry.yarnpkg.com/rimraf/-/rimraf-2.7.1.tgz#35797f13a7fdadc566142c29d4f07ccad483e3ec" + integrity sha512-uWjbaKIK3T1OSVptzX7Nl6PvQ3qAGtKEtVRjRuazjfL3Bx5eI409VZSqgND+4UNnmzLVdPj9FqFJNPqBZFve4w== + dependencies: + glob "^7.1.3" + +rimraf@^3.0.2: + version "3.0.2" + resolved "https://registry.yarnpkg.com/rimraf/-/rimraf-3.0.2.tgz#f1a5402ba6220ad52cc1282bac1ae3aa49fd061a" + integrity sha512-JZkJMZkAGFFPP2YqXZXPbMlMBgsxzE8ILs4lMIX/2o0L9UBw9O/Y3o6wFw/i9YLapcUJWwqbi3kdxIPdC62TIA== + dependencies: + glob "^7.1.3" + +rimraf@~2.6.2: + version "2.6.3" + resolved "https://registry.yarnpkg.com/rimraf/-/rimraf-2.6.3.tgz#b2d104fe0d8fb27cf9e0a1cda8262dd3833c6cab" + integrity sha512-mwqeW5XsA2qAejG46gYdENaxXjx9onRNCfn7L0duuP4hCuTIi/QO7PDK07KJfp1d+izWPrzEJDcSqBa0OZQriA== + dependencies: + glob "^7.1.3" + +ripemd160@^2.0.0, ripemd160@^2.0.1: + version "2.0.2" + resolved "https://registry.yarnpkg.com/ripemd160/-/ripemd160-2.0.2.tgz#a1c1a6f624751577ba5d07914cbc92850585890c" + integrity sha512-ii4iagi25WusVoiC4B4lq7pbXfAp3D9v5CwfkY33vffw2+pkDjY1D8GaN7spsxvCSx8dkPqOZCEZyfxcmJG2IA== + dependencies: + hash-base "^3.0.0" + inherits "^2.0.1" + +run-parallel@^1.1.9: + version "1.2.0" + resolved "https://registry.yarnpkg.com/run-parallel/-/run-parallel-1.2.0.tgz#66d1368da7bdf921eb9d95bd1a9229e7f21a43ee" + integrity sha512-5l4VyZR86LZ/lDxZTR6jqL8AFE2S0IFLMP26AbjsLVADxHdhB/c0GUsH+y39UfCi3dzz8OlQuPmnaJOMoDHQBA== + dependencies: + queue-microtask "^1.2.2" + +safe-array-concat@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/safe-array-concat/-/safe-array-concat-1.0.0.tgz#2064223cba3c08d2ee05148eedbc563cd6d84060" + integrity sha512-9dVEFruWIsnie89yym+xWTAYASdpw3CJV7Li/6zBewGf9z2i1j31rP6jnY0pHEO4QZh6N0K11bFjWmdR8UGdPQ== + dependencies: + call-bind "^1.0.2" + get-intrinsic "^1.2.0" + has-symbols "^1.0.3" + isarray "^2.0.5" + +safe-buffer@5.1.1: + version "5.1.1" + resolved "https://registry.yarnpkg.com/safe-buffer/-/safe-buffer-5.1.1.tgz#893312af69b2123def71f57889001671eeb2c853" + integrity sha512-kKvNJn6Mm93gAczWVJg7wH+wGYWNrDHdWvpUmHyEsgCtIwwo3bqPtV4tR5tuPaUhTOo/kvhVwd8XwwOllGYkbg== + +safe-buffer@5.1.2, safe-buffer@~5.1.0, safe-buffer@~5.1.1: + version "5.1.2" + resolved "https://registry.yarnpkg.com/safe-buffer/-/safe-buffer-5.1.2.tgz#991ec69d296e0313747d59bdfd2b745c35f8828d" + integrity sha512-Gd2UZBJDkXlY7GbJxfsE8/nvKkUEU1G38c1siN6QP6a9PT9MmHB8GnpscSmMJSoF8LOIrt8ud/wPtojys4G6+g== + +safe-buffer@5.2.1, safe-buffer@^5.0.1, safe-buffer@^5.1.0, safe-buffer@^5.1.1, safe-buffer@^5.1.2, safe-buffer@^5.2.0, safe-buffer@~5.2.0: + version "5.2.1" + resolved "https://registry.yarnpkg.com/safe-buffer/-/safe-buffer-5.2.1.tgz#1eaf9fa9bdb1fdd4ec75f58f9cdb4e6b7827eec6" + integrity sha512-rp3So07KcdmmKbGvgaNxQSJr7bGVSVk5S9Eq1F+ppbRo70+YeaDxkw5Dd8NPN+GD6bjnYm2VuPuCXmpuYvmCXQ== + +safe-regex-test@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/safe-regex-test/-/safe-regex-test-1.0.0.tgz#793b874d524eb3640d1873aad03596db2d4f2295" + integrity sha512-JBUUzyOgEwXQY1NuPtvcj/qcBDbDmEvWufhlnXZIm75DEHp+afM1r1ujJpJsV/gSM4t59tpDyPi1sd6ZaPFfsA== + dependencies: + call-bind "^1.0.2" + get-intrinsic "^1.1.3" + is-regex "^1.1.4" + +"safer-buffer@>= 2.1.2 < 3", "safer-buffer@>= 2.1.2 < 3.0.0", safer-buffer@^2.1.0: + version "2.1.2" + resolved "https://registry.yarnpkg.com/safer-buffer/-/safer-buffer-2.1.2.tgz#44fa161b0187b9549dd84bb91802f9bd8385cd6a" + integrity sha512-YZo3K82SD7Riyi0E1EQPojLz7kpepnSQI9IyPbHHg1XXXevb5dJI7tpyN2ADxGcQbHG7vcyRHk0cbwqcQriUtg== + +sass-loader@^12.4.0: + version "12.6.0" + resolved "https://registry.yarnpkg.com/sass-loader/-/sass-loader-12.6.0.tgz#5148362c8e2cdd4b950f3c63ac5d16dbfed37bcb" + integrity sha512-oLTaH0YCtX4cfnJZxKSLAyglED0naiYfNG1iXfU5w1LNZ+ukoA5DtyDIN5zmKVZwYNJP4KRc5Y3hkWga+7tYfA== + dependencies: + klona "^2.0.4" + neo-async "^2.6.2" + +sass-loader@^13.2.2: + version "13.3.2" + resolved "https://registry.yarnpkg.com/sass-loader/-/sass-loader-13.3.2.tgz#460022de27aec772480f03de17f5ba88fa7e18c6" + integrity sha512-CQbKl57kdEv+KDLquhC+gE3pXt74LEAzm+tzywcA0/aHZuub8wTErbjAoNI57rPUWRYRNC5WUnNl8eGJNbDdwg== + dependencies: + neo-async "^2.6.2" + +sax@^1.2.4: + version "1.2.4" + resolved "https://registry.yarnpkg.com/sax/-/sax-1.2.4.tgz#2816234e2378bddc4e5354fab5caa895df7100d9" + integrity sha512-NqVDv9TpANUjFm0N8uM5GxL36UgKi9/atZw+x7YFnQ8ckwFGKrl4xX4yWtrey3UJm5nP1kUbnYgLopqWNSRhWw== + +scheduler@^0.23.0: + version "0.23.0" + resolved "https://registry.yarnpkg.com/scheduler/-/scheduler-0.23.0.tgz#ba8041afc3d30eb206a487b6b384002e4e61fdfe" + integrity sha512-CtuThmgHNg7zIZWAXi3AsyIzA3n4xx7aNyjwC2VJldO2LMVDhFK+63xGqq6CsJH4rTAt6/M+N4GhZiDYPx9eUw== + dependencies: + loose-envify "^1.1.0" + +schema-utils@^3.0.0, schema-utils@^3.1.1, schema-utils@^3.2.0: + version "3.3.0" + resolved "https://registry.yarnpkg.com/schema-utils/-/schema-utils-3.3.0.tgz#f50a88877c3c01652a15b622ae9e9795df7a60fe" + integrity sha512-pN/yOAvcC+5rQ5nERGuwrjLlYvLTbCibnZ1I7B1LaiAz9BRBlE9GMgE/eqV30P7aJQUf7Ddimy/RsbYO/GrVGg== + dependencies: + "@types/json-schema" "^7.0.8" + ajv "^6.12.5" + ajv-keywords "^3.5.2" + +schema-utils@^4.0.0: + version "4.2.0" + resolved "https://registry.yarnpkg.com/schema-utils/-/schema-utils-4.2.0.tgz#70d7c93e153a273a805801882ebd3bff20d89c8b" + integrity sha512-L0jRsrPpjdckP3oPug3/VxNKt2trR8TcabrM6FOAAlvC/9Phcmm+cuAgTlxBqdBR1WJx7Naj9WHw+aOmheSVbw== + dependencies: + "@types/json-schema" "^7.0.9" + ajv "^8.9.0" + ajv-formats "^2.1.1" + ajv-keywords "^5.1.0" + +"semver@2 || 3 || 4 || 5", semver@^5.6.0: + version "5.7.2" + resolved "https://registry.yarnpkg.com/semver/-/semver-5.7.2.tgz#48d55db737c3287cd4835e17fa13feace1c41ef8" + integrity sha512-cBznnQ9KjJqU67B52RMC65CMarK2600WFnbkcaiwWq3xy/5haFJlshgnpjovMVJ+Hff49d8GEn0b87C5pDQ10g== + +semver@^6.0.0, semver@^6.3.0, semver@^6.3.1: + version "6.3.1" + resolved "https://registry.yarnpkg.com/semver/-/semver-6.3.1.tgz#556d2ef8689146e46dcea4bfdd095f3434dffcb4" + integrity sha512-BR7VvDCVHO+q2xBEWskxS6DJE1qRnb7DxzUrogb71CWoSficBxYsiAGd+Kl0mmq/MprG9yArRkyrQxTO6XjMzA== + +semver@^7.3.5, semver@^7.3.7, semver@^7.3.8, semver@^7.5.3, semver@^7.5.4: + version "7.5.4" + resolved "https://registry.yarnpkg.com/semver/-/semver-7.5.4.tgz#483986ec4ed38e1c6c48c34894a9182dbff68a6e" + integrity sha512-1bCSESV6Pv+i21Hvpxp3Dx+pSD8lIPt8uVjRrxAUt/nbswYc+tK6Y2btiULjd4+fnq15PX+nqQDC7Oft7WkwcA== + dependencies: + lru-cache "^6.0.0" + +send@0.18.0: + version "0.18.0" + resolved "https://registry.yarnpkg.com/send/-/send-0.18.0.tgz#670167cc654b05f5aa4a767f9113bb371bc706be" + integrity sha512-qqWzuOjSFOuqPjFe4NOsMLafToQQwBSOEpS+FwEt3A2V3vKubTquT3vmLTQpFgMXp8AlFWFuP1qKaJZOtPpVXg== + dependencies: + debug "2.6.9" + depd "2.0.0" + destroy "1.2.0" + encodeurl "~1.0.2" + escape-html "~1.0.3" + etag "~1.8.1" + fresh "0.5.2" + http-errors "2.0.0" + mime "1.6.0" + ms "2.1.3" + on-finished "2.4.1" + range-parser "~1.2.1" + statuses "2.0.1" + +serialize-javascript@^6.0.0, serialize-javascript@^6.0.1: + version "6.0.1" + resolved "https://registry.yarnpkg.com/serialize-javascript/-/serialize-javascript-6.0.1.tgz#b206efb27c3da0b0ab6b52f48d170b7996458e5c" + integrity sha512-owoXEFjWRllis8/M1Q+Cw5k8ZH40e3zhp/ovX+Xr/vi1qj6QesbyXXViFbpNvWvPNAD62SutwEXavefrLJWj7w== + dependencies: + randombytes "^2.1.0" + +serve-favicon@^2.5.0: + version "2.5.0" + resolved "https://registry.yarnpkg.com/serve-favicon/-/serve-favicon-2.5.0.tgz#935d240cdfe0f5805307fdfe967d88942a2cbcf0" + integrity sha512-FMW2RvqNr03x+C0WxTyu6sOv21oOjkq5j8tjquWccwa6ScNyGFOGJVpuS1NmTVGBAHS07xnSKotgf2ehQmf9iA== + dependencies: + etag "~1.8.1" + fresh "0.5.2" + ms "2.1.1" + parseurl "~1.3.2" + safe-buffer "5.1.1" + +serve-static@1.15.0: + version "1.15.0" + resolved "https://registry.yarnpkg.com/serve-static/-/serve-static-1.15.0.tgz#faaef08cffe0a1a62f60cad0c4e513cff0ac9540" + integrity sha512-XGuRDNjXUijsUL0vl6nSD7cwURuzEgglbOaFuZM9g3kwDXOWVTck0jLzjPzGD+TazWbboZYu52/9/XPdUgne9g== + dependencies: + encodeurl "~1.0.2" + escape-html "~1.0.3" + parseurl "~1.3.3" + send "0.18.0" + +setimmediate@^1.0.4: + version "1.0.5" + resolved "https://registry.yarnpkg.com/setimmediate/-/setimmediate-1.0.5.tgz#290cbb232e306942d7d7ea9b83732ab7856f8285" + integrity sha512-MATJdZp8sLqDl/68LfQmbP8zKPLQNV6BIZoIgrscFDQ+RsvK/BxeDQOgyxKKoh0y/8h3BqVFnCqQ/gd+reiIXA== + +setprototypeof@1.2.0: + version "1.2.0" + resolved "https://registry.yarnpkg.com/setprototypeof/-/setprototypeof-1.2.0.tgz#66c9a24a73f9fc28cbe66b09fed3d33dcaf1b424" + integrity sha512-E5LDX7Wrp85Kil5bhZv46j8jOeboKq5JMmYM3gVGdGH8xFpPWXUMsNrlODCrkoxMEeNi/XZIwuRvY4XNwYMJpw== + +sha.js@^2.4.0, sha.js@^2.4.8: + version "2.4.11" + resolved "https://registry.yarnpkg.com/sha.js/-/sha.js-2.4.11.tgz#37a5cf0b81ecbc6943de109ba2960d1b26584ae7" + integrity sha512-QMEp5B7cftE7APOjk5Y6xgrbWu+WkLVQwk8JNjZ8nKRciZaByEW6MubieAiToS7+dwvrjGhH8jRXz3MVd0AYqQ== + dependencies: + inherits "^2.0.1" + safe-buffer "^5.0.1" + +shallow-clone@^3.0.0: + version "3.0.1" + resolved "https://registry.yarnpkg.com/shallow-clone/-/shallow-clone-3.0.1.tgz#8f2981ad92531f55035b01fb230769a40e02efa3" + integrity sha512-/6KqX+GVUdqPuPPd2LxDDxzX6CAbjJehAAOKlNpqqUpAqPM6HeL8f+o3a+JsyGjn2lv0WY8UsTgUJjU9Ok55NA== + dependencies: + kind-of "^6.0.2" + +shallowequal@^1.1.0: + version "1.1.0" + resolved "https://registry.yarnpkg.com/shallowequal/-/shallowequal-1.1.0.tgz#188d521de95b9087404fd4dcb68b13df0ae4e7f8" + integrity sha512-y0m1JoUZSlPAjXVtPPW70aZWfIL/dSP7AFkRnniLCrK/8MDKog3TySTBmckD+RObVxH0v4Tox67+F14PdED2oQ== + +shebang-command@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/shebang-command/-/shebang-command-2.0.0.tgz#ccd0af4f8835fbdc265b82461aaf0c36663f34ea" + integrity sha512-kHxr2zZpYtdmrN1qDjrrX/Z1rR1kG8Dx+gkpK1G4eXmvXswmcE1hTWBWYUzlraYw1/yZp6YuDY77YtvbN0dmDA== + dependencies: + shebang-regex "^3.0.0" + +shebang-regex@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/shebang-regex/-/shebang-regex-3.0.0.tgz#ae16f1644d873ecad843b0307b143362d4c42172" + integrity sha512-7++dFhtcx3353uBaq8DDR4NuxBetBzC7ZQOhmTQInHEd6bSrXdiEyzCvG07Z44UYdLShWUyXt5M/yhz8ekcb1A== + +side-channel@^1.0.4: + version "1.0.4" + resolved "https://registry.yarnpkg.com/side-channel/-/side-channel-1.0.4.tgz#efce5c8fdc104ee751b25c58d4290011fa5ea2cf" + integrity sha512-q5XPytqFEIKHkGdiMIrY10mvLRvnQh42/+GoBlFW3b2LXLE2xxJpZFdm94we0BaoV3RwJyGqg5wS7epxTv0Zvw== + dependencies: + call-bind "^1.0.0" + get-intrinsic "^1.0.2" + object-inspect "^1.9.0" + +signal-exit@^3.0.2, signal-exit@^3.0.3, signal-exit@^3.0.7: + version "3.0.7" + resolved "https://registry.yarnpkg.com/signal-exit/-/signal-exit-3.0.7.tgz#a9a1767f8af84155114eaabd73f99273c8f59ad9" + integrity sha512-wnD2ZE+l+SPC/uoS0vXeE9L1+0wuaMqKlfz9AMUo38JsyLSBWSFcHR1Rri62LZc12vLr1gb3jl7iwQhgwpAbGQ== + +signal-exit@^4.0.1: + version "4.1.0" + resolved "https://registry.yarnpkg.com/signal-exit/-/signal-exit-4.1.0.tgz#952188c1cbd546070e2dd20d0f41c0ae0530cb04" + integrity sha512-bzyZ1e88w9O1iNJbKnOlvYTrWPDl46O1bG0D3XInv+9tkPrxrN8jUUTiFlDkkmKWgn1M6CfIA13SuGqOa9Korw== + +simple-update-notifier@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/simple-update-notifier/-/simple-update-notifier-2.0.0.tgz#d70b92bdab7d6d90dfd73931195a30b6e3d7cebb" + integrity sha512-a2B9Y0KlNXl9u/vsW6sTIu9vGEpfKu2wRV6l1H3XEas/0gUIzGzBoP/IouTcUQbm9JWZLH3COxyn03TYlFax6w== + dependencies: + semver "^7.5.3" + +sisteransi@^1.0.5: + version "1.0.5" + resolved "https://registry.yarnpkg.com/sisteransi/-/sisteransi-1.0.5.tgz#134d681297756437cc05ca01370d3a7a571075ed" + integrity sha512-bLGGlR1QxBcynn2d5YmDX4MGjlZvy2MRBDRNHLJ8VI6l6+9FUiyTFNJ0IveOSP0bcXgVDPRcfGqA0pjaqUpfVg== + +slash@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/slash/-/slash-2.0.0.tgz#de552851a1759df3a8f206535442f5ec4ddeab44" + integrity sha512-ZYKh3Wh2z1PpEXWr0MpSBZ0V6mZHAQfYevttO11c51CaWjGTaadiKZ+wVt1PbMlDV5qhMFslpZCemhwOK7C89A== + +slash@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/slash/-/slash-3.0.0.tgz#6539be870c165adbd5240220dbe361f1bc4d4634" + integrity sha512-g9Q1haeby36OSStwb4ntCGGGaKsaVSjQ68fBxoQcutl5fS1vuY18H3wSt3jFyFtrkx+Kz0V1G85A4MyAdDMi2Q== + +slash@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/slash/-/slash-4.0.0.tgz#2422372176c4c6c5addb5e2ada885af984b396a7" + integrity sha512-3dOsAHXXUkQTpOYcoAxLIorMTp4gIQr5IW3iVb7A7lFIp0VHhnynm9izx6TssdrIcVIESAlVjtnO2K8bg+Coew== + +source-map-js@^1.0.2: + version "1.0.2" + resolved "https://registry.yarnpkg.com/source-map-js/-/source-map-js-1.0.2.tgz#adbc361d9c62df380125e7f161f71c826f1e490c" + integrity sha512-R0XvVJ9WusLiqTCEiGCmICCMplcCkIwwR11mOSD9CR5u+IXYdiseeEuXCVAjS54zqwkLcPNnmU4OeJ6tUrWhDw== + +source-map-support@^0.5.16, source-map-support@~0.5.20: + version "0.5.21" + resolved "https://registry.yarnpkg.com/source-map-support/-/source-map-support-0.5.21.tgz#04fe7c7f9e1ed2d662233c28cb2b35b9f63f6e4f" + integrity sha512-uBHU3L3czsIyYXKX88fdrGovxdSCoTGDRZ6SYXtSRxLZUzHg5P/66Ht6uoUlHu9EZod+inXhKo3qQgwXUT/y1w== + dependencies: + buffer-from "^1.0.0" + source-map "^0.6.0" + +source-map@0.6.1, source-map@^0.6.0, source-map@^0.6.1, source-map@~0.6.0, source-map@~0.6.1: + version "0.6.1" + resolved "https://registry.yarnpkg.com/source-map/-/source-map-0.6.1.tgz#74722af32e9614e9c287a8d0bbde48b5e2f1a263" + integrity sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g== + +source-map@^0.7.3: + version "0.7.4" + resolved "https://registry.yarnpkg.com/source-map/-/source-map-0.7.4.tgz#a9bbe705c9d8846f4e08ff6765acf0f1b0898656" + integrity sha512-l3BikUxvPOcn5E74dZiq5BGsTb5yEwhaTSzccU6t4sDOH8NWJCstKO5QT2CvtFoK6F0saL7p9xHAqHOlCPJygA== + +space-separated-tokens@^1.0.0: + version "1.1.5" + resolved "https://registry.yarnpkg.com/space-separated-tokens/-/space-separated-tokens-1.1.5.tgz#85f32c3d10d9682007e917414ddc5c26d1aa6899" + integrity sha512-q/JSVd1Lptzhf5bkYm4ob4iWPjx0KiRe3sRFBNrVqbJkFaBm5vbbowy1mymoPNLRa52+oadOhJ+K49wsSeSjTA== + +spdx-correct@^3.0.0: + version "3.2.0" + resolved "https://registry.yarnpkg.com/spdx-correct/-/spdx-correct-3.2.0.tgz#4f5ab0668f0059e34f9c00dce331784a12de4e9c" + integrity sha512-kN9dJbvnySHULIluDHy32WHRUu3Og7B9sbY7tsFLctQkIqnMh3hErYgdMjTYuqmcXX+lK5T1lnUt3G7zNswmZA== + dependencies: + spdx-expression-parse "^3.0.0" + spdx-license-ids "^3.0.0" + +spdx-exceptions@^2.1.0: + version "2.3.0" + resolved "https://registry.yarnpkg.com/spdx-exceptions/-/spdx-exceptions-2.3.0.tgz#3f28ce1a77a00372683eade4a433183527a2163d" + integrity sha512-/tTrYOC7PPI1nUAgx34hUpqXuyJG+DTHJTnIULG4rDygi4xu/tfgmq1e1cIRwRzwZgo4NLySi+ricLkZkw4i5A== + +spdx-expression-parse@^3.0.0: + version "3.0.1" + resolved "https://registry.yarnpkg.com/spdx-expression-parse/-/spdx-expression-parse-3.0.1.tgz#cf70f50482eefdc98e3ce0a6833e4a53ceeba679" + integrity sha512-cbqHunsQWnJNE6KhVSMsMeH5H/L9EpymbzqTQ3uLwNCLZ1Q481oWaofqH7nO6V07xlXwY6PhQdQ2IedWx/ZK4Q== + dependencies: + spdx-exceptions "^2.1.0" + spdx-license-ids "^3.0.0" + +spdx-license-ids@^3.0.0: + version "3.0.13" + resolved "https://registry.yarnpkg.com/spdx-license-ids/-/spdx-license-ids-3.0.13.tgz#7189a474c46f8d47c7b0da4b987bb45e908bd2d5" + integrity sha512-XkD+zwiqXHikFZm4AX/7JSCXA98U5Db4AFd5XUg/+9UNtnH75+Z9KxtpYiJZx36mUDVOwH83pl7yvCer6ewM3w== + +sprintf-js@~1.0.2: + version "1.0.3" + resolved "https://registry.yarnpkg.com/sprintf-js/-/sprintf-js-1.0.3.tgz#04e6926f662895354f3dd015203633b857297e2c" + integrity sha512-D9cPgkvLlV3t3IzL0D0YLvGA9Ahk4PcvVwUbN0dSGr1aP0Nrt4AEnTUbuGvquEC0mA64Gqt1fzirlRs5ibXx8g== + +stackframe@^1.3.4: + version "1.3.4" + resolved "https://registry.yarnpkg.com/stackframe/-/stackframe-1.3.4.tgz#b881a004c8c149a5e8efef37d51b16e412943310" + integrity sha512-oeVtt7eWQS+Na6F//S4kJ2K2VbRlS9D43mAlMyVpVWovy9o+jfgH8O9agzANzaiLjclA0oYzUXEM4PurhSUChw== + +statuses@2.0.1: + version "2.0.1" + resolved "https://registry.yarnpkg.com/statuses/-/statuses-2.0.1.tgz#55cb000ccf1d48728bd23c685a063998cf1a1b63" + integrity sha512-RwNA9Z/7PrK06rYLIzFMlaF+l73iwpzsqRIFgbMLbTcLD6cOao82TaWefPXQvB2fOC4AjuYSEndS7N/mTCbkdQ== + +stop-iteration-iterator@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/stop-iteration-iterator/-/stop-iteration-iterator-1.0.0.tgz#6a60be0b4ee757d1ed5254858ec66b10c49285e4" + integrity sha512-iCGQj+0l0HOdZ2AEeBADlsRC+vsnDsZsbdSiH1yNSjcfKM7fdpCMfqAL/dwF5BLiw/XhRft/Wax6zQbhq2BcjQ== + dependencies: + internal-slot "^1.0.4" + +store2@^2.14.2: + version "2.14.2" + resolved "https://registry.yarnpkg.com/store2/-/store2-2.14.2.tgz#56138d200f9fe5f582ad63bc2704dbc0e4a45068" + integrity sha512-siT1RiqlfQnGqgT/YzXVUNsom9S0H1OX+dpdGN1xkyYATo4I6sep5NmsRD/40s3IIOvlCq6akxkqG82urIZW1w== + +storybook@^7.3.0: + version "7.4.0" + resolved "https://registry.yarnpkg.com/storybook/-/storybook-7.4.0.tgz#f1b64222e3d474bc6e258eb7e48c675685829873" + integrity sha512-jSwbyxHlr2dTY51Pv0mzenjrMDJNZH7DQhHu4ZezpjV+QK/rLCnD+Gt/7iDSaNlsmZJejQcmURDoEybWggMOqw== + dependencies: + "@storybook/cli" "7.4.0" + +stream-browserify@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/stream-browserify/-/stream-browserify-3.0.0.tgz#22b0a2850cdf6503e73085da1fc7b7d0c2122f2f" + integrity sha512-H73RAHsVBapbim0tU2JwwOiXUj+fikfiaoYAKHF3VJfA0pe2BCzkhAHBlLG6REzE+2WNZcxOXjK7lkso+9euLA== + dependencies: + inherits "~2.0.4" + readable-stream "^3.5.0" + +stream-http@^3.2.0: + version "3.2.0" + resolved "https://registry.yarnpkg.com/stream-http/-/stream-http-3.2.0.tgz#1872dfcf24cb15752677e40e5c3f9cc1926028b5" + integrity sha512-Oq1bLqisTyK3TSCXpPbT4sdeYNdmyZJv1LxpEm2vu1ZhK89kSE5YXwZc3cWk0MagGaKriBh9mCFbVGtO+vY29A== + dependencies: + builtin-status-codes "^3.0.0" + inherits "^2.0.4" + readable-stream "^3.6.0" + xtend "^4.0.2" + +stream-shift@^1.0.0: + version "1.0.1" + resolved "https://registry.yarnpkg.com/stream-shift/-/stream-shift-1.0.1.tgz#d7088281559ab2778424279b0877da3c392d5a3d" + integrity sha512-AiisoFqQ0vbGcZgQPY1cdP2I76glaVA/RauYR4G4thNFgkTqr90yXTo4LYX60Jl+sIlPNHHdGSwo01AvbKUSVQ== + +streamsearch@^1.1.0: + version "1.1.0" + resolved "https://registry.yarnpkg.com/streamsearch/-/streamsearch-1.1.0.tgz#404dd1e2247ca94af554e841a8ef0eaa238da764" + integrity sha512-Mcc5wHehp9aXz1ax6bZUyY5afg9u2rv5cqQI3mRrYkGC8rW2hM02jWuwjtL++LS5qinSyhj2QfLyNsuc+VsExg== + +"string-width-cjs@npm:string-width@^4.2.0", string-width@^4.1.0, string-width@^4.2.0, string-width@^4.2.3: + version "4.2.3" + resolved "https://registry.yarnpkg.com/string-width/-/string-width-4.2.3.tgz#269c7117d27b05ad2e536830a8ec895ef9c6d010" + integrity sha512-wKyQRQpjJ0sIp62ErSZdGsjMJWsap5oRNihHhu6G7JVO/9jIB6UyevL+tXuOqrng8j/cxKTWyWUwvSTriiZz/g== + dependencies: + emoji-regex "^8.0.0" + is-fullwidth-code-point "^3.0.0" + strip-ansi "^6.0.1" + +string-width@^5.0.1, string-width@^5.1.2: + version "5.1.2" + resolved "https://registry.yarnpkg.com/string-width/-/string-width-5.1.2.tgz#14f8daec6d81e7221d2a357e668cab73bdbca794" + integrity sha512-HnLOCR3vjcY8beoNLtcjZ5/nxn2afmME6lhrDrebokqMap+XbeW8n9TXpPDOqdGK5qcI3oT0GKTW6wC7EMiVqA== + dependencies: + eastasianwidth "^0.2.0" + emoji-regex "^9.2.2" + strip-ansi "^7.0.1" + +string-width@^6.1.0: + version "6.1.0" + resolved "https://registry.yarnpkg.com/string-width/-/string-width-6.1.0.tgz#96488d6ed23f9ad5d82d13522af9e4c4c3fd7518" + integrity sha512-k01swCJAgQmuADB0YIc+7TuatfNvTBVOoaUWJjTB9R4VJzR5vNWzf5t42ESVZFPS8xTySF7CAdV4t/aaIm3UnQ== + dependencies: + eastasianwidth "^0.2.0" + emoji-regex "^10.2.1" + strip-ansi "^7.0.1" + +string.prototype.matchall@^4.0.8: + version "4.0.9" + resolved "https://registry.yarnpkg.com/string.prototype.matchall/-/string.prototype.matchall-4.0.9.tgz#148779de0f75d36b13b15885fec5cadde994520d" + integrity sha512-6i5hL3MqG/K2G43mWXWgP+qizFW/QH/7kCNN13JrJS5q48FN5IKksLDscexKP3dnmB6cdm9jlNgAsWNLpSykmA== + dependencies: + call-bind "^1.0.2" + define-properties "^1.2.0" + es-abstract "^1.22.1" + get-intrinsic "^1.2.1" + has-symbols "^1.0.3" + internal-slot "^1.0.5" + regexp.prototype.flags "^1.5.0" + side-channel "^1.0.4" + +string.prototype.trim@^1.2.7: + version "1.2.7" + resolved "https://registry.yarnpkg.com/string.prototype.trim/-/string.prototype.trim-1.2.7.tgz#a68352740859f6893f14ce3ef1bb3037f7a90533" + integrity sha512-p6TmeT1T3411M8Cgg9wBTMRtY2q9+PNy9EV1i2lIXUN/btt763oIfxwN3RR8VU6wHX8j/1CFy0L+YuThm6bgOg== + dependencies: + call-bind "^1.0.2" + define-properties "^1.1.4" + es-abstract "^1.20.4" + +string.prototype.trimend@^1.0.6: + version "1.0.6" + resolved "https://registry.yarnpkg.com/string.prototype.trimend/-/string.prototype.trimend-1.0.6.tgz#c4a27fa026d979d79c04f17397f250a462944533" + integrity sha512-JySq+4mrPf9EsDBEDYMOb/lM7XQLulwg5R/m1r0PXEFqrV0qHvl58sdTilSXtKOflCsK2E8jxf+GKC0T07RWwQ== + dependencies: + call-bind "^1.0.2" + define-properties "^1.1.4" + es-abstract "^1.20.4" + +string.prototype.trimstart@^1.0.6: + version "1.0.7" + resolved "https://registry.yarnpkg.com/string.prototype.trimstart/-/string.prototype.trimstart-1.0.7.tgz#d4cdb44b83a4737ffbac2d406e405d43d0184298" + integrity sha512-NGhtDFu3jCEm7B4Fy0DpLewdJQOZcQ0rGbwQ/+stjnrp2i+rlKeCvos9hOIeCmqwratM47OBxY7uFZzjxHXmrg== + dependencies: + call-bind "^1.0.2" + define-properties "^1.2.0" + es-abstract "^1.22.1" + +string_decoder@^1.1.1, string_decoder@^1.3.0: + version "1.3.0" + resolved "https://registry.yarnpkg.com/string_decoder/-/string_decoder-1.3.0.tgz#42f114594a46cf1a8e30b0a84f56c78c3edac21e" + integrity sha512-hkRX8U1WjJFd8LsDJ2yQ/wWWxaopEsABU1XfkM8A+j0+85JAGppt16cr1Whg6KIbb4okU6Mql6BOj+uup/wKeA== + dependencies: + safe-buffer "~5.2.0" + +string_decoder@~1.1.1: + version "1.1.1" + resolved "https://registry.yarnpkg.com/string_decoder/-/string_decoder-1.1.1.tgz#9cf1611ba62685d7030ae9e4ba34149c3af03fc8" + integrity sha512-n/ShnvDi6FHbbVfviro+WojiFzv+s8MPMHBczVePfUpDJLwoLT0ht1l4YwBCbi8pJAveEEdnkHyPyTP/mzRfwg== + dependencies: + safe-buffer "~5.1.0" + +"strip-ansi-cjs@npm:strip-ansi@^6.0.1", strip-ansi@^6.0.0, strip-ansi@^6.0.1: + name strip-ansi-cjs + version "6.0.1" + resolved "https://registry.yarnpkg.com/strip-ansi/-/strip-ansi-6.0.1.tgz#9e26c63d30f53443e9489495b2105d37b67a85d9" + integrity sha512-Y38VPSHcqkFrCpFnQ9vuSXmquuv5oXOKpGeT6aGrr3o3Gc9AlVa6JBfUSOCnbxGGZF+/0ooI7KrPuUSztUdU5A== + dependencies: + ansi-regex "^5.0.1" + +strip-ansi@^7.0.1: + version "7.1.0" + resolved "https://registry.yarnpkg.com/strip-ansi/-/strip-ansi-7.1.0.tgz#d5b6568ca689d8561370b0707685d22434faff45" + integrity sha512-iq6eVVI64nQQTRYq2KtEg2d2uU7LElhTJwsH4YzIHZshxlgZms/wIc4VoDQTlG/IvVIrBKG06CrZnp0qv7hkcQ== + dependencies: + ansi-regex "^6.0.1" + +strip-bom@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/strip-bom/-/strip-bom-3.0.0.tgz#2334c18e9c759f7bdd56fdef7e9ae3d588e68ed3" + integrity sha512-vavAMRXOgBVNF6nyEEmL3DBK19iRpDcoIwW+swQ+CbGiu7lju6t+JklA1MHweoWtadgt4ISVUsXLyDq34ddcwA== + +strip-final-newline@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/strip-final-newline/-/strip-final-newline-2.0.0.tgz#89b852fb2fcbe936f6f4b3187afb0a12c1ab58ad" + integrity sha512-BrpvfNAE3dcvq7ll3xVumzjKjZQ5tI1sEUIKr3Uoks0XUl45St3FlatVqef9prk4jRDzhW6WZg+3bk93y6pLjA== + +strip-indent@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/strip-indent/-/strip-indent-3.0.0.tgz#c32e1cee940b6b3432c771bc2c54bcce73cd3001" + integrity sha512-laJTa3Jb+VQpaC6DseHhF7dXVqHTfJPCRDaEbid/drOhgitgYku/letMUqOXFoWV0zIIUbjpdH2t+tYj4bQMRQ== + dependencies: + min-indent "^1.0.0" + +strip-json-comments@^3.0.1, strip-json-comments@^3.1.1: + version "3.1.1" + resolved "https://registry.yarnpkg.com/strip-json-comments/-/strip-json-comments-3.1.1.tgz#31f1281b3832630434831c310c01cccda8cbe006" + integrity sha512-6fPc+R4ihwqP6N/aIv2f1gMH8lOVtWQHoqC4yK6oSDVVocumAsfCqjkXnqiYMhmMwS/mEHLp7Vehlt3ql6lEig== + +strip-outer@^1.0.1: + version "1.0.1" + resolved "https://registry.yarnpkg.com/strip-outer/-/strip-outer-1.0.1.tgz#b2fd2abf6604b9d1e6013057195df836b8a9d631" + integrity sha512-k55yxKHwaXnpYGsOzg4Vl8+tDrWylxDEpknGjhTiZB8dFRU5rTo9CAzeycivxV3s+zlTKwrs6WxMxR95n26kwg== + dependencies: + escape-string-regexp "^1.0.2" + +style-loader@^3.3.1, style-loader@^3.3.2: + version "3.3.3" + resolved "https://registry.yarnpkg.com/style-loader/-/style-loader-3.3.3.tgz#bba8daac19930169c0c9c96706749a597ae3acff" + integrity sha512-53BiGLXAcll9maCYtZi2RCQZKa8NQQai5C4horqKyRmHj9H7QmcUyucrH+4KW/gBQbXM2AsB0axoEcFZPlfPcw== + +styled-components@^6.0.7: + version "6.0.7" + resolved "https://registry.yarnpkg.com/styled-components/-/styled-components-6.0.7.tgz#1cf4a5e6b6181b29f941934df54af19b7ef05ab0" + integrity sha512-xIwWuiRMYR43mskVsW9MGTRjSo7ol4bcVjT595fGUp3OLBJOlOgaiKaxsHdC4a2HqWKqKnh0CmcRbk5ogyDjTg== + dependencies: + "@babel/cli" "^7.21.0" + "@babel/core" "^7.21.0" + "@babel/helper-module-imports" "^7.18.6" + "@babel/plugin-external-helpers" "^7.18.6" + "@babel/plugin-proposal-class-properties" "^7.18.6" + "@babel/plugin-proposal-object-rest-spread" "^7.20.7" + "@babel/preset-env" "^7.20.2" + "@babel/preset-react" "^7.18.6" + "@babel/preset-typescript" "^7.21.0" + "@babel/traverse" "^7.21.2" + "@emotion/is-prop-valid" "^1.2.1" + "@emotion/unitless" "^0.8.0" + "@types/stylis" "^4.0.2" + css-to-react-native "^3.2.0" + csstype "^3.1.2" + postcss "^8.4.23" + shallowequal "^1.1.0" + stylis "^4.3.0" + tslib "^2.5.0" + +styled-jsx@5.1.1: + version "5.1.1" + resolved "https://registry.yarnpkg.com/styled-jsx/-/styled-jsx-5.1.1.tgz#839a1c3aaacc4e735fed0781b8619ea5d0009d1f" + integrity sha512-pW7uC1l4mBZ8ugbiZrcIsiIvVx1UmTfw7UkC3Um2tmfUq9Bhk8IiyEIPl6F8agHgjzku6j0xQEZbfA5uSgSaCw== + dependencies: + client-only "0.0.1" + +stylis@^4.3.0: + version "4.3.0" + resolved "https://registry.yarnpkg.com/stylis/-/stylis-4.3.0.tgz#abe305a669fc3d8777e10eefcfc73ad861c5588c" + integrity sha512-E87pIogpwUsUwXw7dNyU4QDjdgVMy52m+XEOPEKUn161cCzWjjhPSQhByfd1CcNvrOLnXQ6OnnZDwnJrz/Z4YQ== + +sucrase@^3.32.0: + version "3.34.0" + resolved "https://registry.yarnpkg.com/sucrase/-/sucrase-3.34.0.tgz#1e0e2d8fcf07f8b9c3569067d92fbd8690fb576f" + integrity sha512-70/LQEZ07TEcxiU2dz51FKaE6hCTWC6vr7FOk3Gr0U60C3shtAN+H+BFr9XlYe5xqf3RA8nrc+VIwzCfnxuXJw== + dependencies: + "@jridgewell/gen-mapping" "^0.3.2" + commander "^4.0.0" + glob "7.1.6" + lines-and-columns "^1.1.6" + mz "^2.7.0" + pirates "^4.0.1" + ts-interface-checker "^0.1.9" + +supports-color@^5.3.0: + version "5.5.0" + resolved "https://registry.yarnpkg.com/supports-color/-/supports-color-5.5.0.tgz#e2e69a44ac8772f78a1ec0b35b689df6530efc8f" + integrity sha512-QjVjwdXIt408MIiAqCX4oUKsgU2EqAGzs2Ppkm4aQYbjm+ZEWEcW4SfFNTr4uMNZma0ey4f5lgLrkB0aX0QMow== + dependencies: + has-flag "^3.0.0" + +supports-color@^7.1.0: + version "7.2.0" + resolved "https://registry.yarnpkg.com/supports-color/-/supports-color-7.2.0.tgz#1b7dcdcb32b8138801b3e478ba6a51caa89648da" + integrity sha512-qpCAvRl9stuOHveKsn7HncJRvv501qIacKzQlO/+Lwxc9+0q2wLyv4Dfvt80/DPn2pqOBsJdDiogXGR9+OvwRw== + dependencies: + has-flag "^4.0.0" + +supports-color@^8.0.0: + version "8.1.1" + resolved "https://registry.yarnpkg.com/supports-color/-/supports-color-8.1.1.tgz#cd6fc17e28500cff56c1b86c0a7fd4a54a73005c" + integrity sha512-MpUEN2OodtUzxvKQl72cUF7RQ5EiHsGvSsVG0ia9c5RbWGL2CI4C7EpPS8UTBIplnlzZiNuV56w+FuNxy3ty2Q== + dependencies: + has-flag "^4.0.0" + +supports-preserve-symlinks-flag@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/supports-preserve-symlinks-flag/-/supports-preserve-symlinks-flag-1.0.0.tgz#6eda4bd344a3c94aea376d4cc31bc77311039e09" + integrity sha512-ot0WnXS9fgdkgIcePe6RHNk1WA8+muPa6cSjeR3V8K27q9BB1rTE3R1p7Hv0z1ZyAc8s6Vvv8DIyWf681MAt0w== + +swc-loader@^0.2.3: + version "0.2.3" + resolved "https://registry.yarnpkg.com/swc-loader/-/swc-loader-0.2.3.tgz#6792f1c2e4c9ae9bf9b933b3e010210e270c186d" + integrity sha512-D1p6XXURfSPleZZA/Lipb3A8pZ17fP4NObZvFCDjK/OKljroqDpPmsBdTraWhVBqUNpcWBQY1imWdoPScRlQ7A== + +synchronous-promise@^2.0.15: + version "2.0.17" + resolved "https://registry.yarnpkg.com/synchronous-promise/-/synchronous-promise-2.0.17.tgz#38901319632f946c982152586f2caf8ddc25c032" + integrity sha512-AsS729u2RHUfEra9xJrE39peJcc2stq2+poBXX8bcM08Y6g9j/i/PUzwNQqkaJde7Ntg1TO7bSREbR5sdosQ+g== + +tabbable@^6.0.1: + version "6.2.0" + resolved "https://registry.yarnpkg.com/tabbable/-/tabbable-6.2.0.tgz#732fb62bc0175cfcec257330be187dcfba1f3b97" + integrity sha512-Cat63mxsVJlzYvN51JmVXIgNoUokrIaT2zLclCXjRd8boZ0004U4KCs/sToJ75C6sdlByWxpYnb5Boif1VSFew== + +tailwind-merge@^1.9.1: + version "1.14.0" + resolved "https://registry.yarnpkg.com/tailwind-merge/-/tailwind-merge-1.14.0.tgz#e677f55d864edc6794562c63f5001f45093cdb8b" + integrity sha512-3mFKyCo/MBcgyOTlrY8T7odzZFx+w+qKSMAmdFzRvqBfLlSigU6TZnlFHK0lkMwj9Bj8OYU+9yW9lmGuS0QEnQ== + +tailwindcss@^3.3.3: + version "3.3.3" + resolved "https://registry.yarnpkg.com/tailwindcss/-/tailwindcss-3.3.3.tgz#90da807393a2859189e48e9e7000e6880a736daf" + integrity sha512-A0KgSkef7eE4Mf+nKJ83i75TMyq8HqY3qmFIJSWy8bNt0v1lG7jUcpGpoTFxAwYcWOphcTBLPPJg+bDfhDf52w== + dependencies: + "@alloc/quick-lru" "^5.2.0" + arg "^5.0.2" + chokidar "^3.5.3" + didyoumean "^1.2.2" + dlv "^1.1.3" + fast-glob "^3.2.12" + glob-parent "^6.0.2" + is-glob "^4.0.3" + jiti "^1.18.2" + lilconfig "^2.1.0" + micromatch "^4.0.5" + normalize-path "^3.0.0" + object-hash "^3.0.0" + picocolors "^1.0.0" + postcss "^8.4.23" + postcss-import "^15.1.0" + postcss-js "^4.0.1" + postcss-load-config "^4.0.1" + postcss-nested "^6.0.1" + postcss-selector-parser "^6.0.11" + resolve "^1.22.2" + sucrase "^3.32.0" + +tapable@^2.0.0, tapable@^2.1.1, tapable@^2.2.0, tapable@^2.2.1: + version "2.2.1" + resolved "https://registry.yarnpkg.com/tapable/-/tapable-2.2.1.tgz#1967a73ef4060a82f12ab96af86d52fdb76eeca0" + integrity sha512-GNzQvQTOIP6RyTfE2Qxb8ZVlNmw0n88vp1szwWRimP02mnTsx3Wtn5qRdqY9w2XduFNUgvOwhNnQsjwCp+kqaQ== + +tar-fs@^2.1.1: + version "2.1.1" + resolved "https://registry.yarnpkg.com/tar-fs/-/tar-fs-2.1.1.tgz#489a15ab85f1f0befabb370b7de4f9eb5cbe8784" + integrity sha512-V0r2Y9scmbDRLCNex/+hYzvp/zyYjvFbHPNgVTKfQvVrb6guiE/fxP+XblDNR011utopbkex2nM4dHNV6GDsng== + dependencies: + chownr "^1.1.1" + mkdirp-classic "^0.5.2" + pump "^3.0.0" + tar-stream "^2.1.4" + +tar-stream@^2.1.4: + version "2.2.0" + resolved "https://registry.yarnpkg.com/tar-stream/-/tar-stream-2.2.0.tgz#acad84c284136b060dc3faa64474aa9aebd77287" + integrity sha512-ujeqbceABgwMZxEJnk2HDY2DlnUZ+9oEcb1KzTVfYHio0UE6dG71n60d8D2I4qNvleWrrXpmjpt7vZeF1LnMZQ== + dependencies: + bl "^4.0.3" + end-of-stream "^1.4.1" + fs-constants "^1.0.0" + inherits "^2.0.3" + readable-stream "^3.1.1" + +tar@^6.1.13: + version "6.2.0" + resolved "https://registry.yarnpkg.com/tar/-/tar-6.2.0.tgz#b14ce49a79cb1cd23bc9b016302dea5474493f73" + integrity sha512-/Wo7DcT0u5HUV486xg675HtjNd3BXZ6xDbzsCUZPt5iw8bTQ63bP0Raut3mvro9u+CUyq7YQd8Cx55fsZXxqLQ== + dependencies: + chownr "^2.0.0" + fs-minipass "^2.0.0" + minipass "^5.0.0" + minizlib "^2.1.1" + mkdirp "^1.0.3" + yallist "^4.0.0" + +telejson@^7.2.0: + version "7.2.0" + resolved "https://registry.yarnpkg.com/telejson/-/telejson-7.2.0.tgz#3994f6c9a8f8d7f2dba9be2c7c5bbb447e876f32" + integrity sha512-1QTEcJkJEhc8OnStBx/ILRu5J2p0GjvWsBx56bmZRqnrkdBMUe+nX92jxV+p3dB4CP6PZCdJMQJwCggkNBMzkQ== + dependencies: + memoizerific "^1.11.3" + +temp-dir@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/temp-dir/-/temp-dir-2.0.0.tgz#bde92b05bdfeb1516e804c9c00ad45177f31321e" + integrity sha512-aoBAniQmmwtcKp/7BzsH8Cxzv8OL736p7v1ihGb5e9DJ9kTwGWHrQrVB5+lfVDzfGrdRzXch+ig7LHaY1JTOrg== + +temp@^0.8.4: + version "0.8.4" + resolved "https://registry.yarnpkg.com/temp/-/temp-0.8.4.tgz#8c97a33a4770072e0a05f919396c7665a7dd59f2" + integrity sha512-s0ZZzd0BzYv5tLSptZooSjK8oj6C+c19p7Vqta9+6NPOf7r+fxq0cJe6/oN4LTC79sy5NY8ucOJNgwsKCSbfqg== + dependencies: + rimraf "~2.6.2" + +tempy@^1.0.1: + version "1.0.1" + resolved "https://registry.yarnpkg.com/tempy/-/tempy-1.0.1.tgz#30fe901fd869cfb36ee2bd999805aa72fbb035de" + integrity sha512-biM9brNqxSc04Ee71hzFbryD11nX7VPhQQY32AdDmjFvodsRFz/3ufeoTZ6uYkRFfGo188tENcASNs3vTdsM0w== + dependencies: + del "^6.0.0" + is-stream "^2.0.0" + temp-dir "^2.0.0" + type-fest "^0.16.0" + unique-string "^2.0.0" + +terser-webpack-plugin@^5.3.1, terser-webpack-plugin@^5.3.7: + version "5.3.9" + resolved "https://registry.yarnpkg.com/terser-webpack-plugin/-/terser-webpack-plugin-5.3.9.tgz#832536999c51b46d468067f9e37662a3b96adfe1" + integrity sha512-ZuXsqE07EcggTWQjXUj+Aot/OMcD0bMKGgF63f7UxYcu5/AJF53aIpK1YoP5xR9l6s/Hy2b+t1AM0bLNPRuhwA== + dependencies: + "@jridgewell/trace-mapping" "^0.3.17" + jest-worker "^27.4.5" + schema-utils "^3.1.1" + serialize-javascript "^6.0.1" + terser "^5.16.8" + +terser@^5.10.0, terser@^5.16.8: + version "5.19.4" + resolved "https://registry.yarnpkg.com/terser/-/terser-5.19.4.tgz#941426fa482bf9b40a0308ab2b3cd0cf7c775ebd" + integrity sha512-6p1DjHeuluwxDXcuT9VR8p64klWJKo1ILiy19s6C9+0Bh2+NWTX6nD9EPppiER4ICkHDVB1RkVpin/YW2nQn/g== + dependencies: + "@jridgewell/source-map" "^0.3.3" + acorn "^8.8.2" + commander "^2.20.0" + source-map-support "~0.5.20" + +test-exclude@^6.0.0: + version "6.0.0" + resolved "https://registry.yarnpkg.com/test-exclude/-/test-exclude-6.0.0.tgz#04a8698661d805ea6fa293b6cb9e63ac044ef15e" + integrity sha512-cAGWPIyOHU6zlmg88jwm7VRyXnMN7iV68OGAbYDk/Mh/xC/pzVPlQtY6ngoIH/5/tciuhGfvESU8GrHrcxD56w== + dependencies: + "@istanbuljs/schema" "^0.1.2" + glob "^7.1.4" + minimatch "^3.0.4" + +text-table@^0.2.0: + version "0.2.0" + resolved "https://registry.yarnpkg.com/text-table/-/text-table-0.2.0.tgz#7f5ee823ae805207c00af2df4a84ec3fcfa570b4" + integrity sha512-N+8UisAXDGk8PFXP4HAzVR9nbfmVJ3zYLAWiTIoqC5v5isinhr+r5uaO8+7r3BMfuNIufIsA7RdpVgacC2cSpw== + +thenify-all@^1.0.0: + version "1.6.0" + resolved "https://registry.yarnpkg.com/thenify-all/-/thenify-all-1.6.0.tgz#1a1918d402d8fc3f98fbf234db0bcc8cc10e9726" + integrity sha512-RNxQH/qI8/t3thXJDwcstUO4zeqo64+Uy/+sNVRBx4Xn2OX+OZ9oP+iJnNFqplFra2ZUVeKCSa2oVWi3T4uVmA== + dependencies: + thenify ">= 3.1.0 < 4" + +"thenify@>= 3.1.0 < 4": + version "3.3.1" + resolved "https://registry.yarnpkg.com/thenify/-/thenify-3.3.1.tgz#8932e686a4066038a016dd9e2ca46add9838a95f" + integrity sha512-RVZSIV5IG10Hk3enotrhvz0T9em6cyHBLkH/YAZuKqd8hRkKhSfCGIcP2KUY0EPxndzANBmNllzWPwak+bheSw== + dependencies: + any-promise "^1.0.0" + +through2@^2.0.3: + version "2.0.5" + resolved "https://registry.yarnpkg.com/through2/-/through2-2.0.5.tgz#01c1e39eb31d07cb7d03a96a70823260b23132cd" + integrity sha512-/mrRod8xqpA+IHSLyGCQ2s8SPHiCDEeQJSep1jqLYeEUClOFG2Qsh+4FU6G9VeqpZnGW/Su8LQGc4YKni5rYSQ== + dependencies: + readable-stream "~2.3.6" + xtend "~4.0.1" + +timers-browserify@^2.0.12: + version "2.0.12" + resolved "https://registry.yarnpkg.com/timers-browserify/-/timers-browserify-2.0.12.tgz#44a45c11fbf407f34f97bccd1577c652361b00ee" + integrity sha512-9phl76Cqm6FhSX9Xe1ZUAMLtm1BLkKj2Qd5ApyWkXzsMRaA7dgr81kf4wJmQf/hAvg8EEyJxDo3du/0KlhPiKQ== + dependencies: + setimmediate "^1.0.4" + +tiny-invariant@^1.3.1: + version "1.3.1" + resolved "https://registry.yarnpkg.com/tiny-invariant/-/tiny-invariant-1.3.1.tgz#8560808c916ef02ecfd55e66090df23a4b7aa642" + integrity sha512-AD5ih2NlSssTCwsMznbvwMZpJ1cbhkGd2uueNxzv2jDlEeZdU04JQfRnggJQ8DrcVBGjAsCKwFBbDlVNtEMlzw== + +tmpl@1.0.5: + version "1.0.5" + resolved "https://registry.yarnpkg.com/tmpl/-/tmpl-1.0.5.tgz#8683e0b902bb9c20c4f726e3c0b69f36518c07cc" + integrity sha512-3f0uOEAQwIqGuWW2MVzYg8fV/QNnc/IpuJNG837rLuczAaLVHslWHZQj4IGiEl5Hs3kkbhwL9Ab7Hrsmuj+Smw== + +to-fast-properties@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/to-fast-properties/-/to-fast-properties-2.0.0.tgz#dc5e698cbd079265bc73e0377681a4e4e83f616e" + integrity sha512-/OaKK0xYrs3DmxRYqL/yDc+FxFUVYhDlXMhRmv3z915w2HF1tnN1omB354j8VUGO/hbRzyD6Y3sA7v7GS/ceog== + +to-regex-range@^5.0.1: + version "5.0.1" + resolved "https://registry.yarnpkg.com/to-regex-range/-/to-regex-range-5.0.1.tgz#1648c44aae7c8d988a326018ed72f5b4dd0392e4" + integrity sha512-65P7iz6X5yEr1cwcgvQxbbIw7Uk3gOy5dIdtZ4rDveLqhrdJP+Li/Hx6tyK0NEb+2GCyneCMJiGqrADCSNk8sQ== + dependencies: + is-number "^7.0.0" + +tocbot@^4.20.1: + version "4.21.1" + resolved "https://registry.yarnpkg.com/tocbot/-/tocbot-4.21.1.tgz#7b667bef1c3ea1a07e4f400b742aa71e7e7e5ba0" + integrity sha512-IfajhBTeg0HlMXu1f+VMbPef05QpDTsZ9X2Yn1+8npdaXsXg/+wrm9Ze1WG5OS1UDC3qJ5EQN/XOZ3gfXjPFCw== + +toidentifier@1.0.1: + version "1.0.1" + resolved "https://registry.yarnpkg.com/toidentifier/-/toidentifier-1.0.1.tgz#3be34321a88a820ed1bd80dfaa33e479fbb8dd35" + integrity sha512-o5sSPKEkg/DIQNmH43V0/uerLrpzVedkUh8tGNvaeXpfpuwjKenlSox/2O/BTlZUtEe+JG7s5YhEz608PlAHRA== + +tr46@~0.0.3: + version "0.0.3" + resolved "https://registry.yarnpkg.com/tr46/-/tr46-0.0.3.tgz#8184fd347dac9cdc185992f3a6622e14b9d9ab6a" + integrity sha512-N3WMsuqV66lT30CrXNbEjx4GEwlow3v6rr4mCcv6prnfwhS01rkgyFdjPNBYd9br7LpXV1+Emh01fHnq2Gdgrw== + +trim-repeated@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/trim-repeated/-/trim-repeated-1.0.0.tgz#e3646a2ea4e891312bf7eace6cfb05380bc01c21" + integrity sha512-pkonvlKk8/ZuR0D5tLW8ljt5I8kmxp2XKymhepUeOdCEfKpZaktSArkLHZt76OB1ZvO9bssUsDty4SWhLvZpLg== + dependencies: + escape-string-regexp "^1.0.2" + +ts-api-utils@^1.0.1: + version "1.0.2" + resolved "https://registry.yarnpkg.com/ts-api-utils/-/ts-api-utils-1.0.2.tgz#7c094f753b6705ee4faee25c3c684ade52d66d99" + integrity sha512-Cbu4nIqnEdd+THNEsBdkolnOXhg0I8XteoHaEKgvsxpsbWda4IsUut2c187HxywQCvveojow0Dgw/amxtSKVkQ== + +ts-dedent@^2.0.0, ts-dedent@^2.2.0: + version "2.2.0" + resolved "https://registry.yarnpkg.com/ts-dedent/-/ts-dedent-2.2.0.tgz#39e4bd297cd036292ae2394eb3412be63f563bb5" + integrity sha512-q5W7tVM71e2xjHZTlgfTDoPF/SmqKG5hddq9SzR49CH2hayqRKJtQ4mtRlSxKaJlR/+9rEM+mnBHf7I2/BQcpQ== + +ts-interface-checker@^0.1.9: + version "0.1.13" + resolved "https://registry.yarnpkg.com/ts-interface-checker/-/ts-interface-checker-0.1.13.tgz#784fd3d679722bc103b1b4b8030bcddb5db2a699" + integrity sha512-Y/arvbn+rrz3JCKl9C4kVNfTfSm2/mEp5FSz5EsZSANGPSlQrpRI5M4PKF+mJnE52jOO90PnPSc3Ur3bTQw0gA== + +ts-pnp@^1.1.6: + version "1.2.0" + resolved "https://registry.yarnpkg.com/ts-pnp/-/ts-pnp-1.2.0.tgz#a500ad084b0798f1c3071af391e65912c86bca92" + integrity sha512-csd+vJOb/gkzvcCHgTGSChYpy5f1/XKNsmvBGO4JXS+z1v2HobugDz4s1IeFXM3wZB44uczs+eazB5Q/ccdhQw== + +tsconfig-paths-webpack-plugin@^4.0.1: + version "4.1.0" + resolved "https://registry.yarnpkg.com/tsconfig-paths-webpack-plugin/-/tsconfig-paths-webpack-plugin-4.1.0.tgz#3c6892c5e7319c146eee1e7302ed9e6f2be4f763" + integrity sha512-xWFISjviPydmtmgeUAuXp4N1fky+VCtfhOkDUFIv5ea7p4wuTomI4QTrXvFBX2S4jZsmyTSrStQl+E+4w+RzxA== + dependencies: + chalk "^4.1.0" + enhanced-resolve "^5.7.0" + tsconfig-paths "^4.1.2" + +tsconfig-paths@^3.14.2: + version "3.14.2" + resolved "https://registry.yarnpkg.com/tsconfig-paths/-/tsconfig-paths-3.14.2.tgz#6e32f1f79412decd261f92d633a9dc1cfa99f088" + integrity sha512-o/9iXgCYc5L/JxCHPe3Hvh8Q/2xm5Z+p18PESBU6Ff33695QnCHBEjcytY2q19ua7Mbl/DavtBOLq+oG0RCL+g== + dependencies: + "@types/json5" "^0.0.29" + json5 "^1.0.2" + minimist "^1.2.6" + strip-bom "^3.0.0" + +tsconfig-paths@^4.0.0, tsconfig-paths@^4.1.2: + version "4.2.0" + resolved "https://registry.yarnpkg.com/tsconfig-paths/-/tsconfig-paths-4.2.0.tgz#ef78e19039133446d244beac0fd6a1632e2d107c" + integrity sha512-NoZ4roiN7LnbKn9QqE1amc9DJfzvZXxF4xDavcOWt1BPkdx+m+0gJuPM+S0vCe7zTJMYUP0R8pO2XMr+Y8oLIg== + dependencies: + json5 "^2.2.2" + minimist "^1.2.6" + strip-bom "^3.0.0" + +tslib@^1.13.0, tslib@^1.8.1: + version "1.14.1" + resolved "https://registry.yarnpkg.com/tslib/-/tslib-1.14.1.tgz#cf2d38bdc34a134bcaf1091c41f6619e2f672d00" + integrity sha512-Xni35NKzjgMrwevysHTCArtLDpPvye8zV/0E4EyYn43P7/7qvQwPh9BGkHewbMulVntbigmcT7rdX3BNo9wRJg== + +tslib@^2.0.0, tslib@^2.0.1, tslib@^2.0.3, tslib@^2.1.0, tslib@^2.3.0, tslib@^2.4.0, tslib@^2.5.0: + version "2.6.2" + resolved "https://registry.yarnpkg.com/tslib/-/tslib-2.6.2.tgz#703ac29425e7b37cd6fd456e92404d46d1f3e4ae" + integrity sha512-AEYxH93jGFPn/a2iVAwW87VuUIkR1FVUKB77NwMF7nBTDkDrrT/Hpt/IrCJ0QXhW27jTBDcf5ZY7w6RiqTMw2Q== + +tsutils@^3.21.0: + version "3.21.0" + resolved "https://registry.yarnpkg.com/tsutils/-/tsutils-3.21.0.tgz#b48717d394cea6c1e096983eed58e9d61715b623" + integrity sha512-mHKK3iUXL+3UF6xL5k0PEhKRUBKPBCv/+RkEOpjRWxxx27KKRBmmA60A9pgOUvMi8GKhRMPEmjBRPzs2W7O1OA== + dependencies: + tslib "^1.8.1" + +tty-browserify@^0.0.1: + version "0.0.1" + resolved "https://registry.yarnpkg.com/tty-browserify/-/tty-browserify-0.0.1.tgz#3f05251ee17904dfd0677546670db9651682b811" + integrity sha512-C3TaO7K81YvjCgQH9Q1S3R3P3BtN3RIM8n+OvX4il1K1zgE8ZhI0op7kClgkxtutIE8hQrcrHBXvIheqKUUCxw== + +type-check@^0.4.0, type-check@~0.4.0: + version "0.4.0" + resolved "https://registry.yarnpkg.com/type-check/-/type-check-0.4.0.tgz#07b8203bfa7056c0657050e3ccd2c37730bab8f1" + integrity sha512-XleUoc9uwGXqjWwXaUTZAmzMcFZ5858QA2vvx1Ur5xIcixXIP+8LnFDgRplU30us6teqdlskFfu+ae4K79Ooew== + dependencies: + prelude-ls "^1.2.1" + +type-fest@^0.16.0: + version "0.16.0" + resolved "https://registry.yarnpkg.com/type-fest/-/type-fest-0.16.0.tgz#3240b891a78b0deae910dbeb86553e552a148860" + integrity sha512-eaBzG6MxNzEn9kiwvtre90cXaNLkmadMWa1zQMs3XORCXNbsH/OewwbxC5ia9dCxIxnTAsSxXJaa/p5y8DlvJg== + +type-fest@^0.20.2: + version "0.20.2" + resolved "https://registry.yarnpkg.com/type-fest/-/type-fest-0.20.2.tgz#1bf207f4b28f91583666cb5fbd327887301cd5f4" + integrity sha512-Ne+eE4r0/iWnpAxD852z3A+N0Bt5RN//NjJwRd2VFHEmrywxf5vsZlh4R6lixl6B+wz/8d+maTSAkN1FIkI3LQ== + +type-fest@^0.6.0: + version "0.6.0" + resolved "https://registry.yarnpkg.com/type-fest/-/type-fest-0.6.0.tgz#8d2a2370d3df886eb5c90ada1c5bf6188acf838b" + integrity sha512-q+MB8nYR1KDLrgr4G5yemftpMC7/QLqVndBmEEdqzmNj5dcFOO4Oo8qlwZE3ULT3+Zim1F8Kq4cBnikNhlCMlg== + +type-fest@^0.8.1: + version "0.8.1" + resolved "https://registry.yarnpkg.com/type-fest/-/type-fest-0.8.1.tgz#09e249ebde851d3b1e48d27c105444667f17b83d" + integrity sha512-4dbzIzqvjtgiM5rw1k5rEHtBANKmdudhGyBEajN01fEyhaAIhsoKNy6y7+IN93IfpFtwY9iqi7kD+xwKhQsNJA== + +type-fest@^2.14.0, type-fest@^2.19.0, type-fest@~2.19: + version "2.19.0" + resolved "https://registry.yarnpkg.com/type-fest/-/type-fest-2.19.0.tgz#88068015bb33036a598b952e55e9311a60fd3a9b" + integrity sha512-RAH822pAdBgcNMAfWnCBU3CFZcfZ/i1eZjwFU/dsLKumyuuP3niueg2UAukXYF0E2AAoc82ZSSf9J0WQBinzHA== + +type-is@~1.6.18: + version "1.6.18" + resolved "https://registry.yarnpkg.com/type-is/-/type-is-1.6.18.tgz#4e552cd05df09467dcbc4ef739de89f2cf37c131" + integrity sha512-TkRKr9sUTxEH8MdfuCSP7VizJyzRNMjj2J2do2Jr3Kym598JVdEksuzPQCnlFPW4ky9Q+iA+ma9BGm06XQBy8g== + dependencies: + media-typer "0.3.0" + mime-types "~2.1.24" + +typed-array-buffer@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/typed-array-buffer/-/typed-array-buffer-1.0.0.tgz#18de3e7ed7974b0a729d3feecb94338d1472cd60" + integrity sha512-Y8KTSIglk9OZEr8zywiIHG/kmQ7KWyjseXs1CbSo8vC42w7hg2HgYTxSWwP0+is7bWDc1H+Fo026CpHFwm8tkw== + dependencies: + call-bind "^1.0.2" + get-intrinsic "^1.2.1" + is-typed-array "^1.1.10" + +typed-array-byte-length@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/typed-array-byte-length/-/typed-array-byte-length-1.0.0.tgz#d787a24a995711611fb2b87a4052799517b230d0" + integrity sha512-Or/+kvLxNpeQ9DtSydonMxCx+9ZXOswtwJn17SNLvhptaXYDJvkFFP5zbfU/uLmvnBJlI4yrnXRxpdWH/M5tNA== + dependencies: + call-bind "^1.0.2" + for-each "^0.3.3" + has-proto "^1.0.1" + is-typed-array "^1.1.10" + +typed-array-byte-offset@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/typed-array-byte-offset/-/typed-array-byte-offset-1.0.0.tgz#cbbe89b51fdef9cd6aaf07ad4707340abbc4ea0b" + integrity sha512-RD97prjEt9EL8YgAgpOkf3O4IF9lhJFr9g0htQkm0rchFp/Vx7LW5Q8fSXXub7BXAODyUQohRMyOc3faCPd0hg== + dependencies: + available-typed-arrays "^1.0.5" + call-bind "^1.0.2" + for-each "^0.3.3" + has-proto "^1.0.1" + is-typed-array "^1.1.10" + +typed-array-length@^1.0.4: + version "1.0.4" + resolved "https://registry.yarnpkg.com/typed-array-length/-/typed-array-length-1.0.4.tgz#89d83785e5c4098bec72e08b319651f0eac9c1bb" + integrity sha512-KjZypGq+I/H7HI5HlOoGHkWUUGq+Q0TPhQurLbyrVrvnKTBgzLhIJ7j6J/XTQOi0d1RjyZ0wdas8bKs2p0x3Ng== + dependencies: + call-bind "^1.0.2" + for-each "^0.3.3" + is-typed-array "^1.1.9" + +typedarray@^0.0.6: + version "0.0.6" + resolved "https://registry.yarnpkg.com/typedarray/-/typedarray-0.0.6.tgz#867ac74e3864187b1d3d47d996a78ec5c8830777" + integrity sha512-/aCDEGatGvZ2BIk+HmLf4ifCJFwvKFNb9/JeZPMulfgFracn9QFcAf5GO8B/mweUjSoblS5In0cWhqpfs/5PQA== + +typescript@^5.2.2: + version "5.2.2" + resolved "https://registry.yarnpkg.com/typescript/-/typescript-5.2.2.tgz#5ebb5e5a5b75f085f22bc3f8460fba308310fa78" + integrity sha512-mI4WrpHsbCIcwT9cF4FZvr80QUeKvsUsUvKDoR+X/7XHQH98xYD8YHZg7ANtz2GtZt/CBq2QJ0thkGJMHfqc1w== + +uglify-js@^3.1.4: + version "3.17.4" + resolved "https://registry.yarnpkg.com/uglify-js/-/uglify-js-3.17.4.tgz#61678cf5fa3f5b7eb789bb345df29afb8257c22c" + integrity sha512-T9q82TJI9e/C1TAxYvfb16xO120tMVFZrGA3f9/P4424DNu6ypK103y0GPFVa17yotwSyZW5iYXgjYHkGrJW/g== + +unbox-primitive@^1.0.2: + version "1.0.2" + resolved "https://registry.yarnpkg.com/unbox-primitive/-/unbox-primitive-1.0.2.tgz#29032021057d5e6cdbd08c5129c226dff8ed6f9e" + integrity sha512-61pPlCD9h51VoreyJ0BReideM3MDKMKnh6+V9L08331ipq6Q8OFXZYiqP6n/tbHx4s5I9uRhcye6BrbkizkBDw== + dependencies: + call-bind "^1.0.2" + has-bigints "^1.0.2" + has-symbols "^1.0.3" + which-boxed-primitive "^1.0.2" + +unicode-canonical-property-names-ecmascript@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/unicode-canonical-property-names-ecmascript/-/unicode-canonical-property-names-ecmascript-2.0.0.tgz#301acdc525631670d39f6146e0e77ff6bbdebddc" + integrity sha512-yY5PpDlfVIU5+y/BSCxAJRBIS1Zc2dDG3Ujq+sR0U+JjUevW2JhocOF+soROYDSaAezOzOKuyyixhD6mBknSmQ== + +unicode-match-property-ecmascript@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/unicode-match-property-ecmascript/-/unicode-match-property-ecmascript-2.0.0.tgz#54fd16e0ecb167cf04cf1f756bdcc92eba7976c3" + integrity sha512-5kaZCrbp5mmbz5ulBkDkbY0SsPOjKqVS35VpL9ulMPfSl0J0Xsm+9Evphv9CoIZFwre7aJoa94AY6seMKGVN5Q== + dependencies: + unicode-canonical-property-names-ecmascript "^2.0.0" + unicode-property-aliases-ecmascript "^2.0.0" + +unicode-match-property-value-ecmascript@^2.1.0: + version "2.1.0" + resolved "https://registry.yarnpkg.com/unicode-match-property-value-ecmascript/-/unicode-match-property-value-ecmascript-2.1.0.tgz#cb5fffdcd16a05124f5a4b0bf7c3770208acbbe0" + integrity sha512-qxkjQt6qjg/mYscYMC0XKRn3Rh0wFPlfxB0xkt9CfyTvpX1Ra0+rAmdX2QyAobptSEvuy4RtpPRui6XkV+8wjA== + +unicode-property-aliases-ecmascript@^2.0.0: + version "2.1.0" + resolved "https://registry.yarnpkg.com/unicode-property-aliases-ecmascript/-/unicode-property-aliases-ecmascript-2.1.0.tgz#43d41e3be698bd493ef911077c9b131f827e8ccd" + integrity sha512-6t3foTQI9qne+OZoVQB/8x8rk2k1eVy1gRXhV3oFQ5T6R1dqQ1xtin3XqSlx3+ATBkliTaR/hHyJBm+LVPNM8w== + +unique-string@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/unique-string/-/unique-string-2.0.0.tgz#39c6451f81afb2749de2b233e3f7c5e8843bd89d" + integrity sha512-uNaeirEPvpZWSgzwsPGtU2zVSTrn/8L5q/IexZmH0eH6SA73CmAA5U4GwORTxQAZs95TAXLNqeLoPPNO5gZfWg== + dependencies: + crypto-random-string "^2.0.0" + +unist-util-is@^4.0.0: + version "4.1.0" + resolved "https://registry.yarnpkg.com/unist-util-is/-/unist-util-is-4.1.0.tgz#976e5f462a7a5de73d94b706bac1b90671b57797" + integrity sha512-ZOQSsnce92GrxSqlnEEseX0gi7GH9zTJZ0p9dtu87WRb/37mMPO2Ilx1s/t9vBHrFhbgweUwb+t7cIn5dxPhZg== + +unist-util-visit-parents@^3.0.0: + version "3.1.1" + resolved "https://registry.yarnpkg.com/unist-util-visit-parents/-/unist-util-visit-parents-3.1.1.tgz#65a6ce698f78a6b0f56aa0e88f13801886cdaef6" + integrity sha512-1KROIZWo6bcMrZEwiH2UrXDyalAa0uqzWCxCJj6lPOvTve2WkfgCytoDTPaMnodXh1WrXOq0haVYHj99ynJlsg== + dependencies: + "@types/unist" "^2.0.0" + unist-util-is "^4.0.0" + +unist-util-visit@^2.0.0: + version "2.0.3" + resolved "https://registry.yarnpkg.com/unist-util-visit/-/unist-util-visit-2.0.3.tgz#c3703893146df47203bb8a9795af47d7b971208c" + integrity sha512-iJ4/RczbJMkD0712mGktuGpm/U4By4FfDonL7N/9tATGIF4imikjOuagyMY53tnZq3NP6BcmlrHhEKAfGWjh7Q== + dependencies: + "@types/unist" "^2.0.0" + unist-util-is "^4.0.0" + unist-util-visit-parents "^3.0.0" + +universalify@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/universalify/-/universalify-2.0.0.tgz#75a4984efedc4b08975c5aeb73f530d02df25717" + integrity sha512-hAZsKq7Yy11Zu1DE0OzWjw7nnLZmJZYTDZZyEFHZdUhV8FkH5MCfoU1XMaxXovpyW5nq5scPqq0ZDP9Zyl04oQ== + +unpipe@1.0.0, unpipe@~1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/unpipe/-/unpipe-1.0.0.tgz#b2bf4ee8514aae6165b4817829d21b2ef49904ec" + integrity sha512-pjy2bYhSsufwWlKwPc+l3cN7+wuJlK6uz0YdJEOlQDbl6jo/YlPi4mb8agUkVC8BF7V8NuzeyPNqRksA3hztKQ== + +unplugin@^1.3.1: + version "1.4.0" + resolved "https://registry.yarnpkg.com/unplugin/-/unplugin-1.4.0.tgz#b771373aa1bc664f50a044ee8009bd3a7aa04d85" + integrity sha512-5x4eIEL6WgbzqGtF9UV8VEC/ehKptPXDS6L2b0mv4FRMkJxRtjaJfOWDd6a8+kYbqsjklix7yWP0N3SUepjXcg== + dependencies: + acorn "^8.9.0" + chokidar "^3.5.3" + webpack-sources "^3.2.3" + webpack-virtual-modules "^0.5.0" + +untildify@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/untildify/-/untildify-4.0.0.tgz#2bc947b953652487e4600949fb091e3ae8cd919b" + integrity sha512-KK8xQ1mkzZeg9inewmFVDNkg3l5LUhoq9kN6iWYB/CC9YMG8HA+c1Q8HwDe6dEX7kErrEVNVBO3fWsVq5iDgtw== + +update-browserslist-db@^1.0.11: + version "1.0.11" + resolved "https://registry.yarnpkg.com/update-browserslist-db/-/update-browserslist-db-1.0.11.tgz#9a2a641ad2907ae7b3616506f4b977851db5b940" + integrity sha512-dCwEFf0/oT85M1fHBg4F0jtLwJrutGoHSQXCh7u4o2t1drG+c0a9Flnqww6XUKSfQMPpJBRjU8d4RXB09qtvaA== + dependencies: + escalade "^3.1.1" + picocolors "^1.0.0" + +uri-js@^4.2.2: + version "4.4.1" + resolved "https://registry.yarnpkg.com/uri-js/-/uri-js-4.4.1.tgz#9b1a52595225859e55f669d928f88c6c57f2a77e" + integrity sha512-7rKUyy33Q1yc98pQ1DAmLtwX109F7TIfWlW1Ydo8Wl1ii1SeHieeh0HHfPeL2fMXK6z0s8ecKs9frCuLJvndBg== + dependencies: + punycode "^2.1.0" + +url@^0.11.0: + version "0.11.1" + resolved "https://registry.yarnpkg.com/url/-/url-0.11.1.tgz#26f90f615427eca1b9f4d6a28288c147e2302a32" + integrity sha512-rWS3H04/+mzzJkv0eZ7vEDGiQbgquI1fGfOad6zKvgYQi1SzMmhl7c/DdRGxhaWrVH6z0qWITo8rpnxK/RfEhA== + dependencies: + punycode "^1.4.1" + qs "^6.11.0" + +use-callback-ref@^1.3.0: + version "1.3.0" + resolved "https://registry.yarnpkg.com/use-callback-ref/-/use-callback-ref-1.3.0.tgz#772199899b9c9a50526fedc4993fc7fa1f7e32d5" + integrity sha512-3FT9PRuRdbB9HfXhEq35u4oZkvpJ5kuYbpqhCfmiZyReuRgpnhDlbr2ZEnnuS0RrJAPn6l23xjFg9kpDM+Ms7w== + dependencies: + tslib "^2.0.0" + +use-resize-observer@^9.1.0: + version "9.1.0" + resolved "https://registry.yarnpkg.com/use-resize-observer/-/use-resize-observer-9.1.0.tgz#14735235cf3268569c1ea468f8a90c5789fc5c6c" + integrity sha512-R25VqO9Wb3asSD4eqtcxk8sJalvIOYBqS8MNZlpDSQ4l4xMQxC/J7Id9HoTqPq8FwULIn0PVW+OAqF2dyYbjow== + dependencies: + "@juggle/resize-observer" "^3.3.1" + +use-sidecar@^1.1.2: + version "1.1.2" + resolved "https://registry.yarnpkg.com/use-sidecar/-/use-sidecar-1.1.2.tgz#2f43126ba2d7d7e117aa5855e5d8f0276dfe73c2" + integrity sha512-epTbsLuzZ7lPClpz2TyryBfztm7m+28DlEv2ZCQ3MDr5ssiwyOwGH/e5F9CkfWjJ1t4clvI58yF822/GUkjjhw== + dependencies: + detect-node-es "^1.1.0" + tslib "^2.0.0" + +util-deprecate@^1.0.1, util-deprecate@^1.0.2, util-deprecate@~1.0.1: + version "1.0.2" + resolved "https://registry.yarnpkg.com/util-deprecate/-/util-deprecate-1.0.2.tgz#450d4dc9fa70de732762fbd2d4a28981419a0ccf" + integrity sha512-EPD5q1uXyFxJpCrLnCc1nHnq3gOa6DZBocAIiI2TaSCA7VCJ1UJDMagCzIkXNsUYfD1daK//LTEQ8xiIbrHtcw== + +util@^0.12.0, util@^0.12.4: + version "0.12.5" + resolved "https://registry.yarnpkg.com/util/-/util-0.12.5.tgz#5f17a6059b73db61a875668781a1c2b136bd6fbc" + integrity sha512-kZf/K6hEIrWHI6XqOFUiiMa+79wE/D8Q+NCNAWclkyg3b4d2k7s0QGepNjiABc+aR3N1PAyHL7p6UcLY6LmrnA== + dependencies: + inherits "^2.0.3" + is-arguments "^1.0.4" + is-generator-function "^1.0.7" + is-typed-array "^1.1.3" + which-typed-array "^1.1.2" + +utila@~0.4: + version "0.4.0" + resolved "https://registry.yarnpkg.com/utila/-/utila-0.4.0.tgz#8a16a05d445657a3aea5eecc5b12a4fa5379772c" + integrity sha512-Z0DbgELS9/L/75wZbro8xAnT50pBVFQZ+hUEueGDU5FN51YSCYM+jdxsfCiHjwNP/4LCDD0i/graKpeBnOXKRA== + +utils-merge@1.0.1: + version "1.0.1" + resolved "https://registry.yarnpkg.com/utils-merge/-/utils-merge-1.0.1.tgz#9f95710f50a267947b2ccc124741c1028427e713" + integrity sha512-pMZTvIkT1d+TFGvDOqodOclx0QWkkgi6Tdoa8gC8ffGAAqz9pzPTZWAybbsHHoED/ztMtkv/VoYTYyShUn81hA== + +uuid@^9.0.0: + version "9.0.0" + resolved "https://registry.yarnpkg.com/uuid/-/uuid-9.0.0.tgz#592f550650024a38ceb0c562f2f6aa435761efb5" + integrity sha512-MXcSTerfPa4uqyzStbRoTgt5XIe3x5+42+q1sDuy3R5MDk66URdLMOZe5aPX/SQd+kuYAh0FdP/pO28IkQyTeg== + +v8-to-istanbul@^9.0.0: + version "9.1.0" + resolved "https://registry.yarnpkg.com/v8-to-istanbul/-/v8-to-istanbul-9.1.0.tgz#1b83ed4e397f58c85c266a570fc2558b5feb9265" + integrity sha512-6z3GW9x8G1gd+JIIgQQQxXuiJtCXeAjp6RaPEPLv62mH3iPHPxV6W3robxtCzNErRo6ZwTmzWhsbNvjyEBKzKA== + dependencies: + "@jridgewell/trace-mapping" "^0.3.12" + "@types/istanbul-lib-coverage" "^2.0.1" + convert-source-map "^1.6.0" + +validate-npm-package-license@^3.0.1: + version "3.0.4" + resolved "https://registry.yarnpkg.com/validate-npm-package-license/-/validate-npm-package-license-3.0.4.tgz#fc91f6b9c7ba15c857f4cb2c5defeec39d4f410a" + integrity sha512-DpKm2Ui/xN7/HQKCtpZxoRWBhZ9Z0kqtygG8XCgNQ8ZlDnxuQmWhj566j8fN4Cu3/JmbhsDo7fcAJq4s9h27Ew== + dependencies: + spdx-correct "^3.0.0" + spdx-expression-parse "^3.0.0" + +vary@~1.1.2: + version "1.1.2" + resolved "https://registry.yarnpkg.com/vary/-/vary-1.1.2.tgz#2299f02c6ded30d4a5961b0b9f74524a18f634fc" + integrity sha512-BNGbWLfd0eUPabhkXUVm0j8uuvREyTh5ovRa/dyow/BqAbZJyC+5fU+IzQOzmAKzYqYRAISoRhdQr3eIZ/PXqg== + +victory-vendor@^36.6.8: + version "36.6.11" + resolved "https://registry.yarnpkg.com/victory-vendor/-/victory-vendor-36.6.11.tgz#acae770717c2dae541a54929c304ecab5ab6ac2a" + integrity sha512-nT8kCiJp8dQh8g991J/R5w5eE2KnO8EAIP0xocWlh9l2okngMWglOPoMZzJvek8Q1KUc4XE/mJxTZnvOB1sTYg== + dependencies: + "@types/d3-array" "^3.0.3" + "@types/d3-ease" "^3.0.0" + "@types/d3-interpolate" "^3.0.1" + "@types/d3-scale" "^4.0.2" + "@types/d3-shape" "^3.1.0" + "@types/d3-time" "^3.0.0" + "@types/d3-timer" "^3.0.0" + d3-array "^3.1.6" + d3-ease "^3.0.1" + d3-interpolate "^3.0.1" + d3-scale "^4.0.2" + d3-shape "^3.1.0" + d3-time "^3.0.0" + d3-timer "^3.0.1" + +vm-browserify@^1.1.2: + version "1.1.2" + resolved "https://registry.yarnpkg.com/vm-browserify/-/vm-browserify-1.1.2.tgz#78641c488b8e6ca91a75f511e7a3b32a86e5dda0" + integrity sha512-2ham8XPWTONajOR0ohOKOHXkm3+gaBmGut3SRuu75xLd/RRaY6vqgh8NBYYk7+RW3u5AtzPQZG8F10LHkl0lAQ== + +walker@^1.0.8: + version "1.0.8" + resolved "https://registry.yarnpkg.com/walker/-/walker-1.0.8.tgz#bd498db477afe573dc04185f011d3ab8a8d7653f" + integrity sha512-ts/8E8l5b7kY0vlWLewOkDXMmPdLcVV4GmOQLyxuSswIJsweeFZtAsMF7k1Nszz+TYBQrlYRmzOnr398y1JemQ== + dependencies: + makeerror "1.0.12" + +watchpack@2.4.0, watchpack@^2.2.0, watchpack@^2.4.0: + version "2.4.0" + resolved "https://registry.yarnpkg.com/watchpack/-/watchpack-2.4.0.tgz#fa33032374962c78113f93c7f2fb4c54c9862a5d" + integrity sha512-Lcvm7MGST/4fup+ifyKi2hjyIAwcdI4HRgtvTpIUxBRhB+RFtUh8XtDOxUfctVCnhVi+QQj49i91OyvzkJl6cg== + dependencies: + glob-to-regexp "^0.4.1" + graceful-fs "^4.1.2" + +wcwidth@^1.0.1: + version "1.0.1" + resolved "https://registry.yarnpkg.com/wcwidth/-/wcwidth-1.0.1.tgz#f0b0dcf915bc5ff1528afadb2c0e17b532da2fe8" + integrity sha512-XHPEwS0q6TaxcvG85+8EYkbiCux2XtWG2mkc47Ng2A77BQu9+DqIOJldST4HgPkuea7dvKSj5VgX3P1d4rW8Tg== + dependencies: + defaults "^1.0.3" + +webidl-conversions@^3.0.0: + version "3.0.1" + resolved "https://registry.yarnpkg.com/webidl-conversions/-/webidl-conversions-3.0.1.tgz#24534275e2a7bc6be7bc86611cc16ae0a5654871" + integrity sha512-2JAn3z8AR6rjK8Sm8orRC0h/bcl/DqL7tRPdGZ4I1CjdF+EaMLmYxBHyXuKL849eucPFhvBoxMsflfOb8kxaeQ== + +webpack-dev-middleware@^6.1.1: + version "6.1.1" + resolved "https://registry.yarnpkg.com/webpack-dev-middleware/-/webpack-dev-middleware-6.1.1.tgz#6bbc257ec83ae15522de7a62f995630efde7cc3d" + integrity sha512-y51HrHaFeeWir0YO4f0g+9GwZawuigzcAdRNon6jErXy/SqV/+O6eaVAzDqE6t3e3NpGeR5CS+cCDaTC+V3yEQ== + dependencies: + colorette "^2.0.10" + memfs "^3.4.12" + mime-types "^2.1.31" + range-parser "^1.2.1" + schema-utils "^4.0.0" + +webpack-hot-middleware@^2.25.1: + version "2.25.4" + resolved "https://registry.yarnpkg.com/webpack-hot-middleware/-/webpack-hot-middleware-2.25.4.tgz#d8bc9e9cb664fc3105c8e83d2b9ed436bee4e193" + integrity sha512-IRmTspuHM06aZh98OhBJtqLpeWFM8FXJS5UYpKYxCJzyFoyWj1w6VGFfomZU7OPA55dMLrQK0pRT1eQ3PACr4w== + dependencies: + ansi-html-community "0.0.8" + html-entities "^2.1.0" + strip-ansi "^6.0.0" + +webpack-sources@^3.2.3: + version "3.2.3" + resolved "https://registry.yarnpkg.com/webpack-sources/-/webpack-sources-3.2.3.tgz#2d4daab8451fd4b240cc27055ff6a0c2ccea0cde" + integrity sha512-/DyMEOrDgLKKIG0fmvtz+4dUX/3Ghozwgm6iPp8KRhvn+eQf9+Q7GWxVNMk3+uCPWfdXYC4ExGBckIXdFEfH1w== + +webpack-virtual-modules@^0.5.0: + version "0.5.0" + resolved "https://registry.yarnpkg.com/webpack-virtual-modules/-/webpack-virtual-modules-0.5.0.tgz#362f14738a56dae107937ab98ea7062e8bdd3b6c" + integrity sha512-kyDivFZ7ZM0BVOUteVbDFhlRt7Ah/CSPwJdi8hBpkK7QLumUqdLtVfm/PX/hkcnrvr0i77fO5+TjZ94Pe+C9iw== + +webpack@5, webpack@^5.76.0: + version "5.88.2" + resolved "https://registry.yarnpkg.com/webpack/-/webpack-5.88.2.tgz#f62b4b842f1c6ff580f3fcb2ed4f0b579f4c210e" + integrity sha512-JmcgNZ1iKj+aiR0OvTYtWQqJwq37Pf683dY9bVORwVbUrDhLhdn/PlO2sHsFHPkj7sHNQF3JwaAkp49V+Sq1tQ== + dependencies: + "@types/eslint-scope" "^3.7.3" + "@types/estree" "^1.0.0" + "@webassemblyjs/ast" "^1.11.5" + "@webassemblyjs/wasm-edit" "^1.11.5" + "@webassemblyjs/wasm-parser" "^1.11.5" + acorn "^8.7.1" + acorn-import-assertions "^1.9.0" + browserslist "^4.14.5" + chrome-trace-event "^1.0.2" + enhanced-resolve "^5.15.0" + es-module-lexer "^1.2.1" + eslint-scope "5.1.1" + events "^3.2.0" + glob-to-regexp "^0.4.1" + graceful-fs "^4.2.9" + json-parse-even-better-errors "^2.3.1" + loader-runner "^4.2.0" + mime-types "^2.1.27" + neo-async "^2.6.2" + schema-utils "^3.2.0" + tapable "^2.1.1" + terser-webpack-plugin "^5.3.7" + watchpack "^2.4.0" + webpack-sources "^3.2.3" + +whatwg-url@^5.0.0: + version "5.0.0" + resolved "https://registry.yarnpkg.com/whatwg-url/-/whatwg-url-5.0.0.tgz#966454e8765462e37644d3626f6742ce8b70965d" + integrity sha512-saE57nupxk6v3HY35+jzBwYa0rKSy0XR8JSxZPwgLr7ys0IBzhGviA1/TUGJLmSVqs8pb9AnvICXEuOHLprYTw== + dependencies: + tr46 "~0.0.3" + webidl-conversions "^3.0.0" + +which-boxed-primitive@^1.0.2: + version "1.0.2" + resolved "https://registry.yarnpkg.com/which-boxed-primitive/-/which-boxed-primitive-1.0.2.tgz#13757bc89b209b049fe5d86430e21cf40a89a8e6" + integrity sha512-bwZdv0AKLpplFY2KZRX6TvyuN7ojjr7lwkg6ml0roIy9YeuSr7JS372qlNW18UQYzgYK9ziGcerWqZOmEn9VNg== + dependencies: + is-bigint "^1.0.1" + is-boolean-object "^1.1.0" + is-number-object "^1.0.4" + is-string "^1.0.5" + is-symbol "^1.0.3" + +which-builtin-type@^1.1.3: + version "1.1.3" + resolved "https://registry.yarnpkg.com/which-builtin-type/-/which-builtin-type-1.1.3.tgz#b1b8443707cc58b6e9bf98d32110ff0c2cbd029b" + integrity sha512-YmjsSMDBYsM1CaFiayOVT06+KJeXf0o5M/CAd4o1lTadFAtacTUM49zoYxr/oroopFDfhvN6iEcBxUyc3gvKmw== + dependencies: + function.prototype.name "^1.1.5" + has-tostringtag "^1.0.0" + is-async-function "^2.0.0" + is-date-object "^1.0.5" + is-finalizationregistry "^1.0.2" + is-generator-function "^1.0.10" + is-regex "^1.1.4" + is-weakref "^1.0.2" + isarray "^2.0.5" + which-boxed-primitive "^1.0.2" + which-collection "^1.0.1" + which-typed-array "^1.1.9" + +which-collection@^1.0.1: + version "1.0.1" + resolved "https://registry.yarnpkg.com/which-collection/-/which-collection-1.0.1.tgz#70eab71ebbbd2aefaf32f917082fc62cdcb70906" + integrity sha512-W8xeTUwaln8i3K/cY1nGXzdnVZlidBcagyNFtBdD5kxnb4TvGKR7FfSIS3mYpwWS1QUCutfKz8IY8RjftB0+1A== + dependencies: + is-map "^2.0.1" + is-set "^2.0.1" + is-weakmap "^2.0.1" + is-weakset "^2.0.1" + +which-typed-array@^1.1.10, which-typed-array@^1.1.11, which-typed-array@^1.1.2, which-typed-array@^1.1.9: + version "1.1.11" + resolved "https://registry.yarnpkg.com/which-typed-array/-/which-typed-array-1.1.11.tgz#99d691f23c72aab6768680805a271b69761ed61a" + integrity sha512-qe9UWWpkeG5yzZ0tNYxDmd7vo58HDBc39mZ0xWWpolAGADdFOzkfamWLDxkOWcvHQKVmdTyQdLD4NOfjLWTKew== + dependencies: + available-typed-arrays "^1.0.5" + call-bind "^1.0.2" + for-each "^0.3.3" + gopd "^1.0.1" + has-tostringtag "^1.0.0" + +which@^2.0.1: + version "2.0.2" + resolved "https://registry.yarnpkg.com/which/-/which-2.0.2.tgz#7c6a8dd0a636a0327e10b59c9286eee93f3f51b1" + integrity sha512-BLI3Tl1TW3Pvl70l3yq3Y64i+awpwXqsGBYWkkqMtnbXgrMD+yj7rhW0kuEDxzJaYXGjEW5ogapKNMEKNMjibA== + dependencies: + isexe "^2.0.0" + +wordwrap@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/wordwrap/-/wordwrap-1.0.0.tgz#27584810891456a4171c8d0226441ade90cbcaeb" + integrity sha512-gvVzJFlPycKc5dZN4yPkP8w7Dc37BtP1yczEneOb4uq34pXZcvrtRTmWV8W+Ume+XCxKgbjM+nevkyFPMybd4Q== + +"wrap-ansi-cjs@npm:wrap-ansi@^7.0.0", wrap-ansi@^7.0.0: + name wrap-ansi-cjs + version "7.0.0" + resolved "https://registry.yarnpkg.com/wrap-ansi/-/wrap-ansi-7.0.0.tgz#67e145cff510a6a6984bdf1152911d69d2eb9e43" + integrity sha512-YVGIj2kamLSTxw6NsZjoBxfSwsn0ycdesmc4p+Q21c5zPuZ1pl+NfxVdxPtdHvmNVOQ6XSYG4AUtyt/Fi7D16Q== + dependencies: + ansi-styles "^4.0.0" + string-width "^4.1.0" + strip-ansi "^6.0.0" + +wrap-ansi@^8.1.0: + version "8.1.0" + resolved "https://registry.yarnpkg.com/wrap-ansi/-/wrap-ansi-8.1.0.tgz#56dc22368ee570face1b49819975d9b9a5ead214" + integrity sha512-si7QWI6zUMq56bESFvagtmzMdGOtoxfR+Sez11Mobfc7tm+VkUckk9bW2UeffTGVUbOksxmSw0AA2gs8g71NCQ== + dependencies: + ansi-styles "^6.1.0" + string-width "^5.0.1" + strip-ansi "^7.0.1" + +wrappy@1: + version "1.0.2" + resolved "https://registry.yarnpkg.com/wrappy/-/wrappy-1.0.2.tgz#b5243d8f3ec1aa35f1364605bc0d1036e30ab69f" + integrity sha512-l4Sp/DRseor9wL6EvV2+TuQn63dMkPjZ/sp9XkghTEbV9KlPS1xUsZ3u7/IQO4wxtcFB4bgpQPRcR3QCvezPcQ== + +write-file-atomic@^2.3.0: + version "2.4.3" + resolved "https://registry.yarnpkg.com/write-file-atomic/-/write-file-atomic-2.4.3.tgz#1fd2e9ae1df3e75b8d8c367443c692d4ca81f481" + integrity sha512-GaETH5wwsX+GcnzhPgKcKjJ6M2Cq3/iZp1WyY/X1CSqrW+jVNM9Y7D8EC2sM4ZG/V8wZlSniJnCKWPmBYAucRQ== + dependencies: + graceful-fs "^4.1.11" + imurmurhash "^0.1.4" + signal-exit "^3.0.2" + +write-file-atomic@^4.0.2: + version "4.0.2" + resolved "https://registry.yarnpkg.com/write-file-atomic/-/write-file-atomic-4.0.2.tgz#a9df01ae5b77858a027fd2e80768ee433555fcfd" + integrity sha512-7KxauUdBmSdWnmpaGFg+ppNjKF8uNLry8LyzjauQDOVONfFLNKrKvQOxZ/VuTIcS/gge/YNahf5RIIQWTSarlg== + dependencies: + imurmurhash "^0.1.4" + signal-exit "^3.0.7" + +ws@^6.1.0: + version "6.2.2" + resolved "https://registry.yarnpkg.com/ws/-/ws-6.2.2.tgz#dd5cdbd57a9979916097652d78f1cc5faea0c32e" + integrity sha512-zmhltoSR8u1cnDsD43TX59mzoMZsLKqUweyYBAIvTngR3shc0W6aOZylZmq/7hqyVxPdi+5Ud2QInblgyE72fw== + dependencies: + async-limiter "~1.0.0" + +ws@^8.2.3: + version "8.13.0" + resolved "https://registry.yarnpkg.com/ws/-/ws-8.13.0.tgz#9a9fb92f93cf41512a0735c8f4dd09b8a1211cd0" + integrity sha512-x9vcZYTrFPC7aSIbj7sRCYo7L/Xb8Iy+pW0ng0wt2vCJv7M9HOMy0UoN3rr+IFC7hb7vXoqS+P9ktyLLLhO+LA== + +xtend@^4.0.2, xtend@~4.0.1: + version "4.0.2" + resolved "https://registry.yarnpkg.com/xtend/-/xtend-4.0.2.tgz#bb72779f5fa465186b1f438f674fa347fdb5db54" + integrity sha512-LKYU1iAXJXUgAXn9URjiu+MWhyUXHsvfp7mcuYm9dSUKK0/CjtrUwFAxD82/mCWbtLsGjFIad0wIsod4zrTAEQ== + +y18n@^5.0.5: + version "5.0.8" + resolved "https://registry.yarnpkg.com/y18n/-/y18n-5.0.8.tgz#7f4934d0f7ca8c56f95314939ddcd2dd91ce1d55" + integrity sha512-0pfFzegeDWJHJIAmTLRP2DwHjdF5s7jo9tuztdQxAhINCdvS+3nGINqPd00AphqJR/0LhANUS6/+7SCb98YOfA== + +yallist@^3.0.2: + version "3.1.1" + resolved "https://registry.yarnpkg.com/yallist/-/yallist-3.1.1.tgz#dbb7daf9bfd8bac9ab45ebf602b8cbad0d5d08fd" + integrity sha512-a4UGQaWPH59mOXUYnAG2ewncQS4i4F43Tv3JoAM+s2VDAmS9NsK8GpDMLrCHPksFT7h3K6TOoUNn2pb7RoXx4g== + +yallist@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/yallist/-/yallist-4.0.0.tgz#9bb92790d9c0effec63be73519e11a35019a3a72" + integrity sha512-3wdGidZyq5PB084XLES5TpOSRA3wjXAlIWMhum2kRcv/41Sn2emQ0dycQW4uZXLejwKvg6EsvbdlVL+FYEct7A== + +yaml@^1.10.0: + version "1.10.2" + resolved "https://registry.yarnpkg.com/yaml/-/yaml-1.10.2.tgz#2301c5ffbf12b467de8da2333a459e29e7920e4b" + integrity sha512-r3vXyErRCYJ7wg28yvBY5VSoAF8ZvlcW9/BwUzEtUsjvX/DKs24dIkuwjtuprwJJHsbyUbLApepYTR1BN4uHrg== + +yaml@^2.1.1: + version "2.3.2" + resolved "https://registry.yarnpkg.com/yaml/-/yaml-2.3.2.tgz#f522db4313c671a0ca963a75670f1c12ea909144" + integrity sha512-N/lyzTPaJasoDmfV7YTrYCI0G/3ivm/9wdG0aHuheKowWQwGTsK0Eoiw6utmzAnI6pkJa0DUVygvp3spqqEKXg== + +yargs-parser@^20.2.2, yargs-parser@^20.2.9: + version "20.2.9" + resolved "https://registry.yarnpkg.com/yargs-parser/-/yargs-parser-20.2.9.tgz#2eb7dc3b0289718fc295f362753845c41a0c94ee" + integrity sha512-y11nGElTIV+CT3Zv9t7VKl+Q3hTQoT9a1Qzezhhl6Rp21gJ/IVTW7Z3y9EWXhuUBC2Shnf+DX0antecpAwSP8w== + +yargs-parser@^21.1.1: + version "21.1.1" + resolved "https://registry.yarnpkg.com/yargs-parser/-/yargs-parser-21.1.1.tgz#9096bceebf990d21bb31fa9516e0ede294a77d35" + integrity sha512-tVpsJW7DdjecAiFpbIB1e3qxIQsE6NoPc5/eTdrbbIC4h0LVsWhnoa3g+m2HclBIujHzsxZ4VJVA+GUuc2/LBw== + +yargs@^16.2.0: + version "16.2.0" + resolved "https://registry.yarnpkg.com/yargs/-/yargs-16.2.0.tgz#1c82bf0f6b6a66eafce7ef30e376f49a12477f66" + integrity sha512-D1mvvtDG0L5ft/jGWkLpG1+m0eQxOfaBvTNELraWj22wSVUMWxZUvYgJYcKh6jGGIkJFhH4IZPQhR4TKpc8mBw== + dependencies: + cliui "^7.0.2" + escalade "^3.1.1" + get-caller-file "^2.0.5" + require-directory "^2.1.1" + string-width "^4.2.0" + y18n "^5.0.5" + yargs-parser "^20.2.2" + +yargs@^17.7.2: + version "17.7.2" + resolved "https://registry.yarnpkg.com/yargs/-/yargs-17.7.2.tgz#991df39aca675a192b816e1e0363f9d75d2aa269" + integrity sha512-7dSzzRQ++CKnNI/krKnYRV7JKKPUXMEh61soaHKg9mrWEhzFWhFnxPxGl+69cD1Ou63C13NUPCnmIcrvqCuM6w== + dependencies: + cliui "^8.0.1" + escalade "^3.1.1" + get-caller-file "^2.0.5" + require-directory "^2.1.1" + string-width "^4.2.3" + y18n "^5.0.5" + yargs-parser "^21.1.1" + +yauzl@^2.10.0: + version "2.10.0" + resolved "https://registry.yarnpkg.com/yauzl/-/yauzl-2.10.0.tgz#c7eb17c93e112cb1086fa6d8e51fb0667b79a5f9" + integrity sha512-p4a9I6X6nu6IhoGmBqAcbJy1mlC4j27vEPZX9F4L4/vZT3Lyq1VkFHw/V/PUcB9Buo+DG3iHkT0x3Qya58zc3g== + dependencies: + buffer-crc32 "~0.2.3" + fd-slicer "~1.1.0" + +yocto-queue@^0.1.0: + version "0.1.0" + resolved "https://registry.yarnpkg.com/yocto-queue/-/yocto-queue-0.1.0.tgz#0294eb3dee05028d31ee1a5fa2c556a6aaf10a1b" + integrity sha512-rVksvsnNCdJ/ohGc6xgPwyN8eheCxsiLM8mxuE/t/mOVqJewPuO1miLpTHQiRgTKCLexL4MeAFVagts7HmNZ2Q== + +yocto-queue@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/yocto-queue/-/yocto-queue-1.0.0.tgz#7f816433fb2cbc511ec8bf7d263c3b58a1a3c251" + integrity sha512-9bnSc/HEW2uRy67wc+T8UwauLuPJVn28jb+GtJY16iiKWyvmYJRXVT4UamsAEGQfPohgr2q4Tq0sQbQlxTfi1g== + +zod@3.21.4: + version "3.21.4" + resolved "https://registry.yarnpkg.com/zod/-/zod-3.21.4.tgz#10882231d992519f0a10b5dd58a38c9dabbb64db" + integrity sha512-m46AKbrzKVzOzs/DZgVnG5H55N1sv1M8qZU3A8RIKbs3mrACDNeIOeilDymVb2HdmP8uwshOCF4uJ8uM9rCqJw== From e89e30a14e2d213d5df777ea0e1392e71cdf15ca Mon Sep 17 00:00:00 2001 From: Kevin K Biju <52661649+heavycrystal@users.noreply.github.com> Date: Wed, 13 Sep 2023 18:27:32 +0530 Subject: [PATCH 34/37] building ARM64 images for dev and stable Docker images (#382) --- .github/workflows/dev-docker.yml | 4 ++++ .github/workflows/stable-docker.yml | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/.github/workflows/dev-docker.yml b/.github/workflows/dev-docker.yml index 52e895a89c..8c12863ff8 100644 --- a/.github/workflows/dev-docker.yml +++ b/.github/workflows/dev-docker.yml @@ -36,6 +36,7 @@ jobs: uses: depot/build-push-action@v1 with: token: ${{ secrets.DEPOT_TOKEN }} + platforms: linux/amd64,linux/arm64 context: . file: stacks/nexus.Dockerfile push: ${{ github.ref == 'refs/heads/main' }} @@ -46,6 +47,7 @@ jobs: uses: depot/build-push-action@v1 with: token: ${{ secrets.DEPOT_TOKEN }} + platforms: linux/amd64,linux/arm64 context: . file: stacks/flow-api.Dockerfile push: ${{ github.ref == 'refs/heads/main' }} @@ -56,6 +58,7 @@ jobs: uses: depot/build-push-action@v1 with: token: ${{ secrets.DEPOT_TOKEN }} + platforms: linux/amd64,linux/arm64 context: . file: stacks/flow-worker.Dockerfile push: ${{ github.ref == 'refs/heads/main' }} @@ -66,6 +69,7 @@ jobs: uses: depot/build-push-action@v1 with: token: ${{ secrets.DEPOT_TOKEN }} + platforms: linux/amd64,linux/arm64 context: . file: stacks/flow-snapshot-worker.Dockerfile push: ${{ github.ref == 'refs/heads/main' }} diff --git a/.github/workflows/stable-docker.yml b/.github/workflows/stable-docker.yml index 78e693f442..5f5edd4ef8 100644 --- a/.github/workflows/stable-docker.yml +++ b/.github/workflows/stable-docker.yml @@ -30,6 +30,7 @@ jobs: - name: Publish PeerDB Stable Image uses: depot/build-push-action@v1 with: + platforms: linux/amd64,linux/arm64 token: ${{ secrets.DEPOT_TOKEN }} context: . file: stacks/nexus.Dockerfile @@ -40,6 +41,7 @@ jobs: - name: Publish Flow API Image uses: depot/build-push-action@v1 with: + platforms: linux/amd64,linux/arm64 token: ${{ secrets.DEPOT_TOKEN }} context: . file: stacks/flow-api.Dockerfile @@ -50,6 +52,7 @@ jobs: - name: Publish Flow Worker Stable Image uses: depot/build-push-action@v1 with: + platforms: linux/amd64,linux/arm64 token: ${{ secrets.DEPOT_TOKEN }} context: . file: stacks/flow-worker.Dockerfile @@ -60,6 +63,7 @@ jobs: - name: Publish Flow Snapshot Worker Stable Image uses: depot/build-push-action@v1 with: + platforms: linux/amd64,linux/arm64 token: ${{ secrets.DEPOT_TOKEN }} context: . file: stacks/flow-snapshot-worker.Dockerfile From f1ee5c8f626e9fa7c412b7a41184a2ef9ecdedc2 Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Wed, 13 Sep 2023 09:06:38 -0700 Subject: [PATCH 35/37] Add ubicloud runners (#383) --- .github/workflows/ci.yml | 7 +++++-- .github/workflows/dev-debian.yml | 5 ++++- .github/workflows/dev-docker.yml | 5 ++++- .github/workflows/flow.yml | 11 +++++++---- .github/workflows/golang-lint.yml | 5 ++++- .github/workflows/rust-lint.yml | 5 ++++- .github/workflows/stable-debian.yml | 5 ++++- .github/workflows/stable-docker.yml | 5 ++++- 8 files changed, 36 insertions(+), 12 deletions(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index e0ebb24681..5df761bb60 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -8,7 +8,10 @@ on: jobs: build: - runs-on: ubuntu-latest + strategy: + matrix: + runner: [ubuntu-latest, ubicloud] + runs-on: ${{ matrix.runner }} timeout-minutes: 30 services: catalog_peer: @@ -36,7 +39,7 @@ jobs: run: | sudo apt-get update sudo apt-get install -y protobuf-compiler libssl-dev pkg-config build-essential - + - name: setup gcp service account id: gcp-service-account uses: jsdaniell/create-json@v1.2.2 diff --git a/.github/workflows/dev-debian.yml b/.github/workflows/dev-debian.yml index e80c444e1f..acc2994aa3 100644 --- a/.github/workflows/dev-debian.yml +++ b/.github/workflows/dev-debian.yml @@ -9,7 +9,10 @@ on: jobs: release: name: build and release - runs-on: ubuntu-latest + strategy: + matrix: + runner: [ubuntu-latest, ubicloud] + runs-on: ${{ matrix.runner }} steps: - name: checkout sources uses: actions/checkout@v3 diff --git a/.github/workflows/dev-docker.yml b/.github/workflows/dev-docker.yml index 8c12863ff8..0bc3f33538 100644 --- a/.github/workflows/dev-docker.yml +++ b/.github/workflows/dev-docker.yml @@ -8,7 +8,10 @@ on: jobs: docker-build: - runs-on: ubuntu-latest + strategy: + matrix: + runner: [ubuntu-latest, ubicloud] + runs-on: ${{ matrix.runner }} permissions: contents: read packages: write diff --git a/.github/workflows/flow.yml b/.github/workflows/flow.yml index 5f1623eba3..ce58be8e4d 100644 --- a/.github/workflows/flow.yml +++ b/.github/workflows/flow.yml @@ -8,7 +8,10 @@ on: jobs: flow_test: - runs-on: ubuntu-latest + strategy: + matrix: + runner: [ubuntu-latest, ubicloud] + runs-on: ${{ matrix.runner }} timeout-minutes: 30 services: pg_cdc: @@ -59,13 +62,13 @@ jobs: - name: create hstore extension and increase logical replication limits run: | docker exec pg_cdc psql -h localhost -p 5432 -U postgres -c "CREATE EXTENSION hstore;" - docker exec pg_cdc psql -h localhost -p 5432 -U postgres -c "ALTER SYSTEM SET wal_level=logical;" + docker exec pg_cdc psql -h localhost -p 5432 -U postgres -c "ALTER SYSTEM SET wal_level=logical;" docker exec pg_cdc psql -h localhost -p 5432 -U postgres -c "ALTER SYSTEM SET max_replication_slots=100;" docker exec pg_cdc psql -h localhost -p 5432 -U postgres -c "ALTER SYSTEM SET max_wal_senders=100;" - docker restart pg_cdc + docker restart pg_cdc working-directory: ./flow env: - PG_CDC: + PG_CDC: empty PGPASSWORD: postgres - name: run tests diff --git a/.github/workflows/golang-lint.yml b/.github/workflows/golang-lint.yml index 22a329c7c4..4b1821b7fe 100644 --- a/.github/workflows/golang-lint.yml +++ b/.github/workflows/golang-lint.yml @@ -8,7 +8,10 @@ jobs: checks: write contents: read pull-requests: write - runs-on: ubuntu-latest + strategy: + matrix: + runner: [ubuntu-latest, ubicloud] + runs-on: ${{ matrix.runner }} steps: - name: checkout uses: actions/checkout@v3 diff --git a/.github/workflows/rust-lint.yml b/.github/workflows/rust-lint.yml index 6047673ea6..029160274e 100644 --- a/.github/workflows/rust-lint.yml +++ b/.github/workflows/rust-lint.yml @@ -6,7 +6,10 @@ jobs: checks: write contents: read pull-requests: write - runs-on: ubuntu-latest + strategy: + matrix: + runner: [ubuntu-latest, ubicloud] + runs-on: ${{ matrix.runner }} steps: - name: checkout uses: actions/checkout@v3 diff --git a/.github/workflows/stable-debian.yml b/.github/workflows/stable-debian.yml index 6f337c1dc9..b58afce10e 100644 --- a/.github/workflows/stable-debian.yml +++ b/.github/workflows/stable-debian.yml @@ -8,7 +8,10 @@ on: jobs: release: name: build and release - runs-on: ubuntu-latest + strategy: + matrix: + runner: [ubuntu-latest, ubicloud] + runs-on: ${{ matrix.runner }} permissions: contents: write steps: diff --git a/.github/workflows/stable-docker.yml b/.github/workflows/stable-docker.yml index 5f5edd4ef8..a4b68f37c3 100644 --- a/.github/workflows/stable-docker.yml +++ b/.github/workflows/stable-docker.yml @@ -7,7 +7,10 @@ on: jobs: docker-build: - runs-on: ubuntu-latest + strategy: + matrix: + runner: [ubuntu-latest, ubicloud] + runs-on: ${{ matrix.runner }} permissions: contents: read packages: write From 2da793b1d835f8218246b17b5ca716e928e09b59 Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Thu, 14 Sep 2023 13:57:59 -0400 Subject: [PATCH 36/37] Pass QValue by pointer to significantly reduce gc pressure (#381) - also use pyroscope for better info --- docker-compose.yml | 10 +- flow/cmd/main.go | 14 +- flow/cmd/worker.go | 61 +++++-- flow/connectors/postgres/cdc.go | 38 +++-- flow/connectors/postgres/postgres_cdc_test.go | 161 ++++++++++++------ flow/go.mod | 2 + flow/go.sum | 4 + flow/model/model.go | 113 +++++++++--- flow/workflows/peer_flow.go | 16 +- 9 files changed, 302 insertions(+), 117 deletions(-) diff --git a/docker-compose.yml b/docker-compose.yml index 9472443616..a5c527b056 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -18,6 +18,7 @@ x-flow-worker-env: &flow-worker-env ENABLE_METRICS: "true" # enables exporting of mirror metrics to Catalog in the PEERDB_STATS schema. ENABLE_STATS: "true" + PYROSCOPE_SERVER_ADDRESS: http://pyroscope:4040 services: catalog: @@ -60,6 +61,12 @@ services: labels: kompose.volume.type: configMap + pyroscope: + container_name: pyroscope + image: grafana/pyroscope:latest + ports: + - 4040:4040 + temporal-admin-tools: container_name: temporal-admin-tools depends_on: @@ -120,7 +127,6 @@ services: dockerfile: stacks/flow-worker.Dockerfile environment: <<: [*catalog-config, *flow-worker-env] - PROFILING_SERVER: 0.0.0.0:6060 METRICS_SERVER: 0.0.0.0:6061 ports: - 6060:6060 @@ -136,7 +142,6 @@ services: dockerfile: stacks/flow-worker.Dockerfile environment: <<: [*catalog-config, *flow-worker-env] - PROFILING_SERVER: 0.0.0.0:6062 METRICS_SERVER: 0.0.0.0:6063 ports: - 6062:6062 @@ -155,7 +160,6 @@ services: dockerfile: stacks/flow-worker.Dockerfile environment: <<: [*catalog-config, *flow-worker-env] - PROFILING_SERVER: 0.0.0.0:6064 METRICS_SERVER: 0.0.0.0:6065 ports: - 6064:6064 diff --git a/flow/cmd/main.go b/flow/cmd/main.go index 9b42fc6a55..e2c0a6a411 100644 --- a/flow/cmd/main.go +++ b/flow/cmd/main.go @@ -51,11 +51,11 @@ func main() { EnvVars: []string{"ENABLE_STATS"}, } - profilingServerFlag := &cli.StringFlag{ - Name: "profiling-server", - Value: "localhost:6060", // Default is localhost:6060 - Usage: "HTTP server address for profiling", - EnvVars: []string{"PROFILING_SERVER"}, + pyroscopeServerFlag := &cli.StringFlag{ + Name: "pyroscope-server-address", + Value: "http://pyroscope:4040", + Usage: "HTTP server address for pyroscope", + EnvVars: []string{"PYROSCOPE_SERVER_ADDRESS"}, } metricsServerFlag := &cli.StringFlag{ @@ -77,7 +77,7 @@ func main() { EnableProfiling: ctx.Bool("enable-profiling"), EnableMetrics: ctx.Bool("enable-metrics"), EnableMonitoring: ctx.Bool("enable-monitoring"), - ProfilingServer: ctx.String("profiling-server"), + PyroscopeServer: ctx.String("pyroscope-server-address"), MetricsServer: ctx.String("metrics-server"), }) }, @@ -86,7 +86,7 @@ func main() { profilingFlag, metricsFlag, monitoringFlag, - profilingServerFlag, + pyroscopeServerFlag, metricsServerFlag, }, }, diff --git a/flow/cmd/worker.go b/flow/cmd/worker.go index 6abdd7e05f..a6528fbd32 100644 --- a/flow/cmd/worker.go +++ b/flow/cmd/worker.go @@ -2,7 +2,6 @@ package main import ( "fmt" - "net/http" "os" "os/signal" "runtime" @@ -20,6 +19,7 @@ import ( "github.com/uber-go/tally/v4" "github.com/uber-go/tally/v4/prometheus" + "github.com/grafana/pyroscope-go" prom "github.com/prometheus/client_golang/prometheus" log "github.com/sirupsen/logrus" "go.temporal.io/sdk/client" @@ -32,26 +32,55 @@ type WorkerOptions struct { EnableProfiling bool EnableMetrics bool EnableMonitoring bool - ProfilingServer string + PyroscopeServer string MetricsServer string } +func setupPyroscope(opts *WorkerOptions) { + if opts.PyroscopeServer == "" { + log.Fatal("pyroscope server address is not set but profiling is enabled") + } + + // measure contention + runtime.SetMutexProfileFraction(5) + runtime.SetBlockProfileRate(5) + + _, err := pyroscope.Start(pyroscope.Config{ + ApplicationName: "io.peerdb.flow_worker", + + ServerAddress: opts.PyroscopeServer, + + // you can disable logging by setting this to nil + Logger: log.StandardLogger(), + + // you can provide static tags via a map: + Tags: map[string]string{"hostname": os.Getenv("HOSTNAME")}, + + ProfileTypes: []pyroscope.ProfileType{ + // these profile types are enabled by default: + pyroscope.ProfileCPU, + pyroscope.ProfileAllocObjects, + pyroscope.ProfileAllocSpace, + pyroscope.ProfileInuseObjects, + pyroscope.ProfileInuseSpace, + + // these profile types are optional: + pyroscope.ProfileGoroutines, + pyroscope.ProfileMutexCount, + pyroscope.ProfileMutexDuration, + pyroscope.ProfileBlockCount, + pyroscope.ProfileBlockDuration, + }, + }) + + if err != nil { + log.Fatal(err) + } +} + func WorkerMain(opts *WorkerOptions) error { if opts.EnableProfiling { - // Start HTTP profiling server with timeouts - go func() { - server := http.Server{ - Addr: opts.ProfilingServer, - ReadTimeout: 5 * time.Minute, - WriteTimeout: 15 * time.Minute, - } - - log.Infof("starting profiling server on %s", opts.ProfilingServer) - - if err := server.ListenAndServe(); err != nil { - log.Errorf("unable to start profiling server: %v", err) - } - }() + setupPyroscope(opts) } go func() { diff --git a/flow/connectors/postgres/cdc.go b/flow/connectors/postgres/cdc.go index 34d91dea45..22ca6e516e 100644 --- a/flow/connectors/postgres/cdc.go +++ b/flow/connectors/postgres/cdc.go @@ -221,10 +221,14 @@ func (p *PostgresCDCSource) consumeStream( // should be ideally sourceTableName as we are in pullRecrods. // will change in future pkeyCol := req.TableNameSchemaMapping[tableName].PrimaryKeyColumn - pkeyColVal := rec.GetItems()[pkeyCol] + pkeyColVal, err := rec.GetItems().GetValueByColName(pkeyCol) + if err != nil { + return nil, fmt.Errorf("error getting pkey column value: %w", err) + } + tablePkeyVal := model.TableWithPkey{ TableName: tableName, - PkeyColVal: pkeyColVal, + PkeyColVal: *pkeyColVal, } _, ok := records.TablePKeyLastSeen[tablePkeyVal] if !ok { @@ -232,22 +236,23 @@ func (p *PostgresCDCSource) consumeStream( records.TablePKeyLastSeen[tablePkeyVal] = len(records.Records) - 1 } else { oldRec := records.Records[records.TablePKeyLastSeen[tablePkeyVal]] - // iterate through unchanged toast cols and set them - for col, val := range oldRec.GetItems() { - if _, ok := r.NewItems[col]; !ok { - r.NewItems[col] = val - delete(r.UnchangedToastColumns, col) - } + // iterate through unchanged toast cols and set them in new record + updatedCols := r.NewItems.UpdateIfNotExists(oldRec.GetItems()) + for _, col := range updatedCols { + delete(r.UnchangedToastColumns, col) } records.Records = append(records.Records, rec) records.TablePKeyLastSeen[tablePkeyVal] = len(records.Records) - 1 } case *model.InsertRecord: pkeyCol := req.TableNameSchemaMapping[tableName].PrimaryKeyColumn - pkeyColVal := rec.GetItems()[pkeyCol] + pkeyColVal, err := rec.GetItems().GetValueByColName(pkeyCol) + if err != nil { + return nil, fmt.Errorf("error getting pkey column value: %w", err) + } tablePkeyVal := model.TableWithPkey{ TableName: tableName, - PkeyColVal: pkeyColVal, + PkeyColVal: *pkeyColVal, } records.Records = append(records.Records, rec) // all columns will be set in insert record, so add it to the map @@ -430,34 +435,35 @@ 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]bool, error) { // if the tuple is nil, return an empty map if tuple == nil { - return make(model.RecordItems), make(map[string]bool), nil + return model.NewRecordItems(), make(map[string]bool), nil } // create empty map of string to interface{} - items := make(model.RecordItems) + items := model.NewRecordItems() unchangedToastColumns := make(map[string]bool) for idx, col := range tuple.Columns { colName := rel.Columns[idx].Name switch col.DataType { case 'n': // null - items[colName] = qvalue.QValue{Kind: qvalue.QValueKindInvalid, Value: nil} + val := &qvalue.QValue{Kind: qvalue.QValueKindInvalid, Value: nil} + items.AddColumn(colName, val) case 't': // text /* bytea also appears here as a hex */ data, err := p.decodeColumnData(col.Data, rel.Columns[idx].DataType, pgtype.TextFormatCode) if err != nil { return nil, nil, fmt.Errorf("error decoding text column data: %w", err) } - items[colName] = *data + items.AddColumn(colName, data) case 'b': // binary data, err := p.decodeColumnData(col.Data, rel.Columns[idx].DataType, pgtype.BinaryFormatCode) if err != nil { return nil, nil, fmt.Errorf("error decoding binary column data: %w", err) } - items[colName] = *data + items.AddColumn(colName, data) case 'u': // unchanged toast unchangedToastColumns[colName] = true default: diff --git a/flow/connectors/postgres/postgres_cdc_test.go b/flow/connectors/postgres/postgres_cdc_test.go index 727ffa133a..11659b43d5 100644 --- a/flow/connectors/postgres/postgres_cdc_test.go +++ b/flow/connectors/postgres/postgres_cdc_test.go @@ -11,6 +11,7 @@ import ( "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/jackc/pgx/v5" + "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" ) @@ -39,13 +40,19 @@ func (suite *PostgresCDCTestSuite) insertSimpleRecords(srcTableName string) { func (suite *PostgresCDCTestSuite) validateInsertedSimpleRecords(records []model.Record, srcTableName string, dstTableName string) { suite.Equal(3, len(records)) - matchData := []model.RecordItems{ - {"id": qvalue.QValue{Kind: qvalue.QValueKindInt32, Value: int32(2)}, - "name": qvalue.QValue{Kind: qvalue.QValueKindString, Value: "quick"}}, - {"id": qvalue.QValue{Kind: qvalue.QValueKindInt32, Value: int32(4)}, - "name": qvalue.QValue{Kind: qvalue.QValueKindString, Value: "brown"}}, - {"id": qvalue.QValue{Kind: qvalue.QValueKindInt32, Value: int32(8)}, - "name": qvalue.QValue{Kind: qvalue.QValueKindString, Value: "fox"}}, + matchData := []*model.RecordItems{ + model.NewRecordItemWithData(map[string]*qvalue.QValue{ + "id": {Kind: qvalue.QValueKindInt32, Value: int32(2)}, + "name": {Kind: qvalue.QValueKindString, Value: "quick"}, + }), + model.NewRecordItemWithData(map[string]*qvalue.QValue{ + "id": {Kind: qvalue.QValueKindInt32, Value: int32(4)}, + "name": {Kind: qvalue.QValueKindString, Value: "brown"}, + }), + model.NewRecordItemWithData(map[string]*qvalue.QValue{ + "id": {Kind: qvalue.QValueKindInt32, Value: int32(8)}, + "name": {Kind: qvalue.QValueKindString, Value: "fox"}, + }), } for idx, record := range records { suite.IsType(&model.InsertRecord{}, record) @@ -84,15 +91,22 @@ func (suite *PostgresCDCTestSuite) validateSimpleMutatedRecords(records []model. suite.Equal(srcTableName, updateRecord.SourceTableName) suite.Equal(dstTableName, updateRecord.DestinationTableName) suite.Equal(model.RecordItems{}, updateRecord.OldItems) - suite.Equal(model.RecordItems{"id": qvalue.QValue{Kind: qvalue.QValueKindInt32, Value: int32(2)}, - "name": qvalue.QValue{Kind: qvalue.QValueKindString, Value: "slow"}}, updateRecord.NewItems) + + items := model.NewRecordItemWithData(map[string]*qvalue.QValue{ + "id": {Kind: qvalue.QValueKindInt32, Value: int32(2)}, + "name": {Kind: qvalue.QValueKindString, Value: "slow"}, + }) + suite.Equal(items, updateRecord.NewItems) suite.IsType(&model.DeleteRecord{}, records[1]) deleteRecord := records[1].(*model.DeleteRecord) suite.Equal(srcTableName, deleteRecord.SourceTableName) suite.Equal(dstTableName, deleteRecord.DestinationTableName) - suite.Equal(model.RecordItems{"id": qvalue.QValue{Kind: qvalue.QValueKindInt32, Value: int32(8)}, - "name": qvalue.QValue{Kind: qvalue.QValueKindInvalid, Value: nil}}, deleteRecord.Items) + items = model.NewRecordItemWithData(map[string]*qvalue.QValue{ + "id": {Kind: qvalue.QValueKindInt32, Value: int32(8)}, + "name": {Kind: qvalue.QValueKindInvalid, Value: nil}, + }) + suite.Equal(items, deleteRecord.Items) } func (suite *PostgresCDCTestSuite) randBytea(n int) []byte { @@ -143,13 +157,29 @@ func (suite *PostgresCDCTestSuite) validateInsertedToastRecords(records []model. insertRecord := record.(*model.InsertRecord) suite.Equal(srcTableName, insertRecord.SourceTableName) suite.Equal(dstTableName, insertRecord.DestinationTableName) - suite.Equal(5, len(insertRecord.Items)) + suite.Equal(5, insertRecord.Items.Len()) + + idVal, err := insertRecord.Items.GetValueByColName("id") + suite.NoError(err, "Error fetching id") + + n_tVal, err := insertRecord.Items.GetValueByColName("n_t") + suite.NoError(err, "Error fetching n_t") + + lz4_tVal, err := insertRecord.Items.GetValueByColName("lz4_t") + suite.NoError(err, "Error fetching lz4_t") + + n_bVal, err := insertRecord.Items.GetValueByColName("n_b") + suite.NoError(err, "Error fetching n_b") + + lz4_bVal, err := insertRecord.Items.GetValueByColName("lz4_b") + suite.NoError(err, "Error fetching lz4_b") - suite.Equal(int32(idx+1), insertRecord.Items["id"].Value.(int32)) - suite.Equal(32768, len(insertRecord.Items["n_t"].Value.(string))) - suite.Equal(32768, len(insertRecord.Items["lz4_t"].Value.(string))) - suite.Equal(32768, len(insertRecord.Items["n_b"].Value.([]byte))) - suite.Equal(32768, len(insertRecord.Items["lz4_b"].Value.([]byte))) + // Perform the actual value checks + suite.Equal(int32(idx+1), idVal.Value.(int32)) + suite.Equal(32768, len(n_tVal.Value.(string))) + suite.Equal(32768, len(lz4_tVal.Value.(string))) + suite.Equal(32768, len(n_bVal.Value.([]byte))) + suite.Equal(32768, len(lz4_bVal.Value.([]byte))) } } @@ -191,61 +221,76 @@ func (suite *PostgresCDCTestSuite) validateMutatedToastRecords(records []model.R updateRecord := records[0].(*model.UpdateRecord) suite.Equal(srcTableName, updateRecord.SourceTableName) suite.Equal(dstTableName, updateRecord.DestinationTableName) - suite.Equal(2, len(updateRecord.NewItems)) - suite.Equal(int32(1), updateRecord.NewItems["id"].Value.(int32)) - suite.Equal(qvalue.QValueKindString, updateRecord.NewItems["n_t"].Kind) - suite.Equal(65536, len(updateRecord.NewItems["n_t"].Value.(string))) + items := updateRecord.NewItems + suite.Equal(2, items.Len()) + v, err := items.GetValueByColName("id") + suite.NoError(err, "Error fetching id") + suite.Equal(int32(1), v.Value.(int32)) + v, err = items.GetValueByColName("n_t") + suite.NoError(err, "Error fetching n_t") + 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.IsType(&model.UpdateRecord{}, records[1]) updateRecord = records[1].(*model.UpdateRecord) suite.Equal(srcTableName, updateRecord.SourceTableName) suite.Equal(dstTableName, updateRecord.DestinationTableName) - suite.Equal(4, len(updateRecord.NewItems)) - suite.Equal(qvalue.QValueKindInt32, updateRecord.NewItems["id"].Kind) - suite.Equal(int32(3), updateRecord.NewItems["id"].Value.(int32)) - suite.Equal(qvalue.QValueKindString, updateRecord.NewItems["lz4_t"].Kind) - suite.Equal(65536, len(updateRecord.NewItems["lz4_t"].Value.(string))) - suite.Equal(qvalue.QValueKindBytes, updateRecord.NewItems["n_b"].Kind) - suite.Equal(65536, len(updateRecord.NewItems["n_b"].Value.([]byte))) - suite.Equal(qvalue.QValueKindBytes, updateRecord.NewItems["lz4_b"].Kind) - suite.Equal(65536, len(updateRecord.NewItems["lz4_b"].Value.([]byte))) + + items = updateRecord.NewItems + suite.Equal(4, items.Len()) + v = items.GetColumnValue("id") + suite.Equal(qvalue.QValueKindInt32, v.Kind) + suite.Equal(int32(3), v.Value.(int32)) + v = items.GetColumnValue("lz4_t") + suite.Equal(qvalue.QValueKindString, v.Kind) + suite.Equal(65536, len(v.Value.(string))) + v = items.GetColumnValue("n_b") + suite.Equal(qvalue.QValueKindBytes, v.Kind) + suite.Equal(65536, len(v.Value.([]byte))) + v = items.GetColumnValue("lz4_b") + 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"]) - + // Test case for records[2] suite.IsType(&model.UpdateRecord{}, records[2]) updateRecord = records[2].(*model.UpdateRecord) suite.Equal(srcTableName, updateRecord.SourceTableName) suite.Equal(dstTableName, updateRecord.DestinationTableName) - suite.Equal(5, len(updateRecord.NewItems)) - suite.Equal(int32(4), updateRecord.NewItems["id"].Value.(int32)) - suite.Equal(qvalue.QValueKindString, updateRecord.NewItems["n_t"].Kind) - suite.Equal(65536, len(updateRecord.NewItems["n_t"].Value.(string))) - suite.Equal(qvalue.QValueKindString, updateRecord.NewItems["lz4_t"].Kind) - suite.Equal(65536, len(updateRecord.NewItems["lz4_t"].Value.(string))) - suite.Equal(qvalue.QValueKindBytes, updateRecord.NewItems["n_b"].Kind) - suite.Equal(65536, len(updateRecord.NewItems["n_b"].Value.([]byte))) - suite.Equal(qvalue.QValueKindBytes, updateRecord.NewItems["lz4_b"].Kind) - suite.Equal(65536, len(updateRecord.NewItems["lz4_b"].Value.([]byte))) + + items = updateRecord.NewItems + suite.Equal(5, items.Len()) + v = items.GetColumnValue("id") + suite.Equal(int32(4), v.Value.(int32)) + suite.Equal(qvalue.QValueKindString, items.GetColumnValue("n_t").Kind) + suite.Equal(65536, len(items.GetColumnValue("n_t").Value.(string))) + suite.Equal(qvalue.QValueKindString, items.GetColumnValue("lz4_t").Kind) + suite.Equal(65536, len(items.GetColumnValue("lz4_t").Value.(string))) + suite.Equal(qvalue.QValueKindBytes, items.GetColumnValue("n_b").Kind) + suite.Equal(65536, len(items.GetColumnValue("n_b").Value.([]byte))) + suite.Equal(qvalue.QValueKindBytes, items.GetColumnValue("lz4_b").Kind) + suite.Equal(65536, len(items.GetColumnValue("lz4_b").Value.([]byte))) suite.Equal(0, len(updateRecord.UnchangedToastColumns)) + // Test case for records[3] suite.IsType(&model.DeleteRecord{}, records[3]) deleteRecord := records[3].(*model.DeleteRecord) suite.Equal(srcTableName, deleteRecord.SourceTableName) suite.Equal(dstTableName, deleteRecord.DestinationTableName) - suite.Equal(5, len(deleteRecord.Items)) - suite.Equal(int32(3), deleteRecord.Items["id"].Value.(int32)) - suite.Equal(qvalue.QValueKindInvalid, deleteRecord.Items["n_t"].Kind) - suite.Nil(deleteRecord.Items["n_t"].Value) - suite.Equal(qvalue.QValueKindInvalid, deleteRecord.Items["lz4_t"].Kind) - suite.Nil(deleteRecord.Items["lz4_t"].Value) - suite.Equal(qvalue.QValueKindInvalid, deleteRecord.Items["n_b"].Kind) - suite.Nil(deleteRecord.Items["n_b"].Value) - suite.Equal(qvalue.QValueKindInvalid, deleteRecord.Items["lz4_b"].Kind) - suite.Nil(deleteRecord.Items["lz4_b"].Value) + items = deleteRecord.Items + suite.Equal(5, items.Len()) + suite.Equal(int32(3), items.GetColumnValue("id").Value.(int32)) + suite.Equal(qvalue.QValueKindInvalid, items.GetColumnValue("n_t").Kind) + suite.Nil(items.GetColumnValue("n_t").Value) + suite.Equal(qvalue.QValueKindInvalid, items.GetColumnValue("lz4_t").Kind) + suite.Nil(items.GetColumnValue("lz4_t").Value) + suite.Equal(qvalue.QValueKindInvalid, items.GetColumnValue("n_b").Kind) + suite.Nil(items.GetColumnValue("n_b").Value) + suite.Equal(qvalue.QValueKindInvalid, items.GetColumnValue("lz4_b").Kind) + suite.Nil(items.GetColumnValue("lz4_b").Value) } func (suite *PostgresCDCTestSuite) SetupSuite() { @@ -648,8 +693,16 @@ func (suite *PostgresCDCTestSuite) TestAllTypesHappyFlow() { RelationMessageMapping: relationMessageMapping, }) suite.failTestError(err) - suite.Equal(1, len(records.RecordBatch.Records)) - suite.Equal(35, len(records.RecordBatch.Records[0].GetItems())) + require.Equal(suite.T(), 1, len(records.RecordBatch.Records)) + + items := records.RecordBatch.Records[0].GetItems() + numCols := items.Len() + if numCols != 35 { + jsonStr, err := items.ToJSON() + suite.failTestError(err) + fmt.Printf("record batch json: %s\n", jsonStr) + suite.FailNow("expected 35 columns, got %d", numCols) + } err = suite.connector.PullFlowCleanup(allTypesHappyFlowName) suite.failTestError(err) diff --git a/flow/go.mod b/flow/go.mod index df3e3ed06f..e11e83f977 100644 --- a/flow/go.mod +++ b/flow/go.mod @@ -36,6 +36,7 @@ require ( require ( github.com/golang-jwt/jwt/v5 v5.0.0 // indirect + github.com/grafana/pyroscope-go/godeltaprof v0.1.3 // indirect github.com/grpc-ecosystem/grpc-gateway v1.16.0 // indirect github.com/pkg/errors v0.9.1 // indirect github.com/twmb/murmur3 v1.1.8 // indirect @@ -106,6 +107,7 @@ require ( github.com/google/s2a-go v0.1.5 // indirect github.com/googleapis/enterprise-certificate-proxy v0.2.5 // indirect github.com/googleapis/gax-go/v2 v2.12.0 // indirect + github.com/grafana/pyroscope-go v1.0.2 github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c // indirect github.com/hashicorp/errwrap v1.1.0 // indirect diff --git a/flow/go.sum b/flow/go.sum index ee6182af9b..c0c5591487 100644 --- a/flow/go.sum +++ b/flow/go.sum @@ -1193,6 +1193,10 @@ github.com/googleapis/go-type-adapters v1.0.0/go.mod h1:zHW75FOG2aur7gAO2B+MLby+ github.com/googleapis/google-cloud-go-testing v0.0.0-20200911160855-bcd43fbb19e8/go.mod h1:dvDLG8qkwmyD9a/MJJN3XJcT3xFxOKAvTZGvuZmac9g= github.com/gorilla/securecookie v1.1.1/go.mod h1:ra0sb63/xPlUeL+yeDciTfxMRAA+MP+HVt/4epWDjd4= github.com/gorilla/sessions v1.2.1/go.mod h1:dk2InVEVJ0sfLlnXv9EAgkf6ecYs/i80K/zI+bUmuGM= +github.com/grafana/pyroscope-go v1.0.2 h1:dEFgO9VbhYTwuwpCC5coTpuW0JjISEWDZtvRAW9v5Tw= +github.com/grafana/pyroscope-go v1.0.2/go.mod h1:bShDKsVZdzxq+Ol6no0JKigU9y5FTWUcFditMXaH09o= +github.com/grafana/pyroscope-go/godeltaprof v0.1.3 h1:eunWpv1B3Z7ZK9o4499EmQGlY+CsDmSZ4FbxjRx37uk= +github.com/grafana/pyroscope-go/godeltaprof v0.1.3/go.mod h1:1HSPtjU8vLG0jE9JrTdzjgFqdJ/VgN7fvxBNq3luJko= github.com/grpc-ecosystem/go-grpc-middleware v1.3.0/go.mod h1:z0ButlSOZa5vEBq9m2m2hlwIgKw+rp3sdCBRoJY+30Y= github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= github.com/grpc-ecosystem/go-grpc-middleware v1.4.0/go.mod h1:g5qyo/la0ALbONm6Vbp88Yd8NsDy6rZz+RcrMPxvld8= diff --git a/flow/model/model.go b/flow/model/model.go index 9bc5877cba..3984ad2848 100644 --- a/flow/model/model.go +++ b/flow/model/model.go @@ -2,6 +2,8 @@ package model import ( "encoding/json" + "errors" + "fmt" "math/big" "time" @@ -38,39 +40,112 @@ type Record interface { // get table name GetTableName() string // get columns and values for the record - GetItems() RecordItems + GetItems() *RecordItems } -type RecordItems map[string]qvalue.QValue +type RecordItems struct { + colToValIdx map[string]int + values []*qvalue.QValue +} + +func NewRecordItems() *RecordItems { + return &RecordItems{ + colToValIdx: make(map[string]int), + // create a slice of 64 qvalues so that we don't have to allocate memory + // for each record to reduce GC pressure + values: make([]*qvalue.QValue, 0, 32), + } +} + +func NewRecordItemWithData(data map[string]*qvalue.QValue) *RecordItems { + recordItem := NewRecordItems() + for col, val := range data { + recordItem.colToValIdx[col] = len(recordItem.values) + recordItem.values = append(recordItem.values, val) + } + return recordItem +} + +func (r *RecordItems) AddColumn(col string, val *qvalue.QValue) { + if idx, ok := r.colToValIdx[col]; ok { + r.values[idx] = val + } else { + r.colToValIdx[col] = len(r.values) + r.values = append(r.values, val) + } +} + +func (r *RecordItems) GetColumnValue(col string) *qvalue.QValue { + if idx, ok := r.colToValIdx[col]; ok { + return r.values[idx] + } + return nil +} -func (r RecordItems) ToJSON() (string, error) { +// UpdateIfNotExists takes in a RecordItems as input and updates the values of the +// current RecordItems with the values from the input RecordItems for the columns +// that are present in the input RecordItems but not in the current RecordItems. +// We return the slice of col names that were updated. +func (r *RecordItems) UpdateIfNotExists(input *RecordItems) []string { + updatedCols := make([]string, 0) + for col, idx := range input.colToValIdx { + if _, ok := r.colToValIdx[col]; !ok { + r.colToValIdx[col] = len(r.values) + r.values = append(r.values, input.values[idx]) + updatedCols = append(updatedCols, col) + } + } + return updatedCols +} + +func (r *RecordItems) GetValueByColName(colName string) (*qvalue.QValue, error) { + idx, ok := r.colToValIdx[colName] + if !ok { + return nil, fmt.Errorf("column name %s not found", colName) + } + return r.values[idx], nil +} + +func (r *RecordItems) Len() int { + return len(r.values) +} + +func (r *RecordItems) ToJSON() (string, error) { + if r.colToValIdx == nil { + return "", errors.New("colToValIdx is nil") + } jsonStruct := make(map[string]interface{}) - for k, v := range r { + for col, idx := range r.colToValIdx { + v := r.values[idx] var err error switch v.Kind { case qvalue.QValueKindString, qvalue.QValueKindJSON: if len(v.Value.(string)) > 15*1024*1024 { - jsonStruct[k] = "" + jsonStruct[col] = "" } else { - jsonStruct[k] = v.Value + jsonStruct[col] = v.Value } case qvalue.QValueKindTimestamp, qvalue.QValueKindTimestampTZ, qvalue.QValueKindDate, qvalue.QValueKindTime, qvalue.QValueKindTimeTZ: - jsonStruct[k], err = v.GoTimeConvert() + jsonStruct[col], err = v.GoTimeConvert() if err != nil { return "", err } case qvalue.QValueKindNumeric: - bigRat := v.Value.(*big.Rat) - jsonStruct[k] = bigRat.FloatString(9) + bigRat, ok := v.Value.(*big.Rat) + if !ok { + return "", errors.New("expected *big.Rat value") + } + jsonStruct[col] = bigRat.FloatString(9) default: - jsonStruct[k] = v.Value + jsonStruct[col] = v.Value } } jsonBytes, err := json.Marshal(jsonStruct) if err != nil { return "", err } + return string(jsonBytes), nil } @@ -84,7 +159,7 @@ type InsertRecord struct { // CommitID is the ID of the commit corresponding to this record. CommitID int64 // Items is a map of column name to value. - Items RecordItems + Items *RecordItems // unchanged toast columns UnchangedToastColumns map[string]bool } @@ -98,7 +173,7 @@ func (r *InsertRecord) GetTableName() string { return r.DestinationTableName } -func (r *InsertRecord) GetItems() RecordItems { +func (r *InsertRecord) GetItems() *RecordItems { return r.Items } @@ -110,9 +185,9 @@ type UpdateRecord struct { // Name of the destination table DestinationTableName string // OldItems is a map of column name to value. - OldItems RecordItems + OldItems *RecordItems // NewItems is a map of column name to value. - NewItems RecordItems + NewItems *RecordItems // unchanged toast columns UnchangedToastColumns map[string]bool } @@ -127,7 +202,7 @@ func (r *UpdateRecord) GetTableName() string { return r.DestinationTableName } -func (r *UpdateRecord) GetItems() RecordItems { +func (r *UpdateRecord) GetItems() *RecordItems { return r.NewItems } @@ -139,7 +214,7 @@ type DeleteRecord struct { // CheckPointID is the ID of the record. CheckPointID int64 // Items is a map of column name to value. - Items RecordItems + Items *RecordItems // unchanged toast columns UnchangedToastColumns map[string]bool } @@ -153,13 +228,13 @@ func (r *DeleteRecord) GetTableName() string { return r.SourceTableName } -func (r *DeleteRecord) GetItems() RecordItems { +func (r *DeleteRecord) GetItems() *RecordItems { return r.Items } type TableWithPkey struct { TableName string - PkeyColVal interface{} + PkeyColVal qvalue.QValue } type RecordBatch struct { @@ -238,7 +313,7 @@ func (r *RelationRecord) GetTableName() string { return r.TableSchemaDelta.SrcTableName } -func (r *RelationRecord) GetItems() RecordItems { +func (r *RelationRecord) GetItems() *RecordItems { return nil } diff --git a/flow/workflows/peer_flow.go b/flow/workflows/peer_flow.go index e509e59e52..9c33d5464d 100644 --- a/flow/workflows/peer_flow.go +++ b/flow/workflows/peer_flow.go @@ -93,7 +93,7 @@ func NewStartedPeerFlowState() *PeerFlowState { } // truncate the progress and other arrays to a max of 10 elements -func (s *PeerFlowState) TruncateProgress() { +func (s *PeerFlowState) TruncateProgress(log log.Logger) { if len(s.Progress) > 10 { s.Progress = s.Progress[len(s.Progress)-10:] } @@ -103,6 +103,18 @@ func (s *PeerFlowState) TruncateProgress() { if len(s.NormalizeFlowStatuses) > 10 { s.NormalizeFlowStatuses = s.NormalizeFlowStatuses[len(s.NormalizeFlowStatuses)-10:] } + + if s.SyncFlowErrors != nil { + // log and clear the error + log.Error("sync flow error: ", s.SyncFlowErrors) + s.SyncFlowErrors = nil + } + + if s.NormalizeFlowErrors != nil { + // log and clear the error + log.Error("normalize flow error: ", s.NormalizeFlowErrors) + s.NormalizeFlowErrors = nil + } } // PeerFlowWorkflowExecution represents the state for execution of a peer flow. @@ -413,6 +425,6 @@ func PeerFlowWorkflowWithConfig( } } - state.TruncateProgress() + state.TruncateProgress(w.logger) return nil, workflow.NewContinueAsNewError(ctx, PeerFlowWorkflowWithConfig, cfg, limits, state) } From 21e3dd8f18465a0e5c38fc971a769fd3be91af93 Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Fri, 15 Sep 2023 09:43:26 -0400 Subject: [PATCH 37/37] update rust deps (#388) --- nexus/Cargo.lock | 1291 +++++++++++++++++++++------------------------- nexus/Cargo.toml | 2 + 2 files changed, 597 insertions(+), 696 deletions(-) diff --git a/nexus/Cargo.lock b/nexus/Cargo.lock index d7ef91cbaa..9900e59f31 100644 --- a/nexus/Cargo.lock +++ b/nexus/Cargo.lock @@ -4,9 +4,9 @@ version = 3 [[package]] name = "addr2line" -version = "0.19.0" +version = "0.21.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a76fd60b23679b7d19bd066031410fb7e458ccc5e958eb5c325888ce4baedc97" +checksum = "8a30b2e23b9e17a9f90641c7ab1549cd9b44f296d3ccbf309d2863cfe398a0cb" dependencies = [ "gimli", ] @@ -28,11 +28,22 @@ dependencies = [ "version_check", ] +[[package]] +name = "ahash" +version = "0.8.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2c99f64d1e06488f620f932677e24bc6e2897582980441ae90a671415bd7ec2f" +dependencies = [ + "cfg-if", + "once_cell", + "version_check", +] + [[package]] name = "aho-corasick" -version = "0.7.20" +version = "1.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cc936419f96fa211c1b9166887b38e5e40b19958e5b895be7c1f93adec7071ac" +checksum = "0c378d78423fdad8089616f827526ee33c19f2fddbd5de1629152c9593ba4783" dependencies = [ "memchr", ] @@ -69,30 +80,29 @@ dependencies = [ [[package]] name = "anstream" -version = "0.3.0" +version = "0.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9e579a7752471abc2a8268df8b20005e3eadd975f585398f17efcfd8d4927371" +checksum = "b1f58811cfac344940f1a400b6e6231ce35171f614f26439e80f8c1465c5cc0c" dependencies = [ "anstyle", "anstyle-parse", "anstyle-query", "anstyle-wincon", "colorchoice", - "is-terminal", "utf8parse", ] [[package]] name = "anstyle" -version = "1.0.0" +version = "1.0.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "41ed9a86bf92ae6580e0a31281f65a1b1d867c0cc68d5346e2ae128dddfa6a7d" +checksum = "b84bf0a05bbb2a83e5eb6fa36bb6e87baa08193c35ff52bbf6b38d8af2890e46" [[package]] name = "anstyle-parse" -version = "0.2.0" +version = "0.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e765fd216e48e067936442276d1d57399e37bce53c264d6fefbe298080cb57ee" +checksum = "938874ff5980b03a87c5524b3ae5b59cf99b1d6bc836848df7bc5ada9643c333" dependencies = [ "utf8parse", ] @@ -108,9 +118,9 @@ dependencies = [ [[package]] name = "anstyle-wincon" -version = "1.0.0" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4bcd8291a340dd8ac70e18878bc4501dd7b4ff970cfa21c207d36ece51ea88fd" +checksum = "58f54d10c6dfa51283a066ceab3ec1ab78d13fae00aa49243a45e4571fb79dfd" dependencies = [ "anstyle", "windows-sys 0.48.0", @@ -118,9 +128,9 @@ dependencies = [ [[package]] name = "anyhow" -version = "1.0.70" +version = "1.0.75" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7de8ce5e0f9f8d88245311066a578d72b7af3e7088f32783804676302df237e4" +checksum = "a4668cab20f66d8d020e1fbc0ebe47217433c1b6c8f2040faf858554e394ace6" [[package]] name = "ar" @@ -142,9 +152,9 @@ checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" [[package]] name = "async-compression" -version = "0.3.15" +version = "0.4.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "942c7cd7ae39e91bde4820d74132e9862e62c2f386c3aa90ccf55949f5bad63a" +checksum = "bb42b2197bf15ccb092b62c74515dbd8b86d0effd934795f6687c93b6e679a2c" dependencies = [ "flate2", "futures-core", @@ -155,13 +165,13 @@ dependencies = [ [[package]] name = "async-recursion" -version = "1.0.4" +version = "1.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0e97ce7de6cf12de5d7226c73f5ba9811622f4db3a5b91b55c53e987e5f91cba" +checksum = "5fd55a5ba1179988837d24ab4c7cc8ed6efdeff578ede0416b4225a5fca35bd0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.15", + "syn 2.0.33", ] [[package]] @@ -183,29 +193,18 @@ checksum = "16e62a023e7c117e27523144c5d2459f4397fcc3cab0085af8e2224f643a0193" dependencies = [ "proc-macro2", "quote", - "syn 2.0.15", + "syn 2.0.33", ] [[package]] name = "async-trait" -version = "0.1.68" +version = "0.1.73" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b9ccdd8f2a161be9bd5c023df56f1b2a0bd1d83872ae53b71a84a12c9bf6e842" +checksum = "bc00ceb34980c03614e35a3a4e218276a0a824e911d07651cd0d858a51e8c0f0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.15", -] - -[[package]] -name = "atty" -version = "0.2.14" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d9b39be18770d11421cdb1b9947a45dd3f37e93092cbf377614828a319d5fee8" -dependencies = [ - "hermit-abi 0.1.19", - "libc", - "winapi", + "syn 2.0.33", ] [[package]] @@ -216,13 +215,13 @@ checksum = "d468802bab17cbc0cc575e9b053f41e72aa36bfa6b7f55e3529ffa43161b97fa" [[package]] name = "axum" -version = "0.6.17" +version = "0.6.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b70caf9f1b0c045f7da350636435b775a9733adf2df56e8aa2a29210fbc335d4" +checksum = "3b829e4e32b91e643de6eafe82b1d90675f5874230191a4ffbc1b336dec4d6bf" dependencies = [ "async-trait", "axum-core", - "bitflags", + "bitflags 1.3.2", "bytes", "futures-util", "http", @@ -261,9 +260,9 @@ dependencies = [ [[package]] name = "backtrace" -version = "0.3.67" +version = "0.3.69" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "233d376d6d185f2a3093e58f283f60f880315b6c60075b01f36b3b85154564ca" +checksum = "2089b7e3f35b9dd2d0ed921ead4f6d318c27680d4a5bd167b3ee120edb105837" dependencies = [ "addr2line", "cc", @@ -282,9 +281,9 @@ checksum = "9e1b586273c5702936fe7b7d6896644d8be71e6314cfe09d3167c95f712589e8" [[package]] name = "base64" -version = "0.21.0" +version = "0.21.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4a4ddaa51a5bc52a6948f74c06d20aaaddb71924eab79b8c97a8c556e942d6a" +checksum = "9ba43ea6f343b788c8764558649e08df62f86c6ef251fdaeb1ffd010a9ae50a2" [[package]] name = "base64ct" @@ -294,9 +293,9 @@ checksum = "8c3c1a368f70d6cf7302d78f8f7093da241fb8e8807c05cc9e51a125895a6d5b" [[package]] name = "bcder" -version = "0.7.2" +version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab26f019795af36086f2ca879aeeaae7566bdfd2fe0821a0328d3fdd9d1da2d9" +checksum = "bf16bec990f8ea25cab661199904ef452fcf11f565c404ce6cffbdf3f8cbbc47" dependencies = [ "bytes", "smallvec", @@ -304,9 +303,9 @@ dependencies = [ [[package]] name = "bigdecimal" -version = "0.3.0" +version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6aaf33151a6429fe9211d1b276eafdf70cdff28b071e76c0b0e1503221ea3744" +checksum = "a6773ddc0eafc0e509fb60e48dff7f450f8e674a0686ae8605e8d9901bd5eefa" dependencies = [ "num-bigint", "num-integer", @@ -320,6 +319,12 @@ version = "1.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" +[[package]] +name = "bitflags" +version = "2.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b4682ae6287fcf752ecaabbfcc7b6f9b72aa33933dc23a554d853aea8eea8635" + [[package]] name = "bitvec" version = "1.0.1" @@ -348,7 +353,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4114279215a005bc675e386011e594e1d9b800918cea18fcadadcce864a2046b" dependencies = [ "borsh-derive", - "hashbrown", + "hashbrown 0.13.2", ] [[package]] @@ -388,9 +393,9 @@ dependencies = [ [[package]] name = "bumpalo" -version = "3.12.0" +version = "3.14.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0d261e256854913907f67ed06efbc3338dfe6179796deefc1ff763fc1aee5535" +checksum = "7f30e7476521f6f8af1a1c4c0b8cc94f0bee37d91763d0ca2665f299b6cd8aec" [[package]] name = "bytecheck" @@ -422,15 +427,15 @@ checksum = "14c189c53d098945499cdfa7ecc63567cf3886b3332b312a5b4585d8d3a6a610" [[package]] name = "bytes" -version = "1.4.0" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "89b2fd2a0dcf38d7971e2194b6b6eebab45ae01067456a7fd93d5547a61b70be" +checksum = "a2bd12c1caf447e69cd4528f47f94d203fd2582878ecb9e9465484c4148a8223" [[package]] name = "cargo-deb" -version = "1.43.1" +version = "1.44.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "35d7a62f626b537fb110b4a2588a18be8f9588b1a1c5c0e16c33f56a18cb9ecd" +checksum = "ca5e0b34a6289c8a680045e8e719302bd98d5caa0c0e38519ca1643f2c5147c6" dependencies = [ "ar", "cargo_toml", @@ -450,18 +455,18 @@ dependencies = [ "serde_json", "tar", "tempfile", - "toml 0.7.4", + "toml 0.7.8", "xz2", ] [[package]] name = "cargo_toml" -version = "0.15.2" +version = "0.15.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7f83bc2e401ed041b7057345ebc488c005efa0341d5541ce7004d30458d0090b" +checksum = "599aa35200ffff8f04c1925aa1acc92fa2e08874379ef42e210a80e527e60838" dependencies = [ "serde", - "toml 0.7.4", + "toml 0.7.8", ] [[package]] @@ -484,9 +489,12 @@ dependencies = [ [[package]] name = "cc" -version = "1.0.79" +version = "1.0.83" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "50d30906286121d95be3d479533b458f87493b30a4b5f79a607db8f5d11aa91f" +checksum = "f1174fb0b6ec23863f8b971027804a42614e347eafb0a95bf0b12cdae21fc4d0" +dependencies = [ + "libc", +] [[package]] name = "cfg-if" @@ -496,71 +504,58 @@ checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" [[package]] name = "chrono" -version = "0.4.26" +version = "0.4.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ec837a71355b28f6556dbd569b37b3f363091c0bd4b2e735674521b4c5fd9bc5" +checksum = "defd4e7873dbddba6c7c91e199c7fcb946abc4a6a4ac3195400bcfb01b5de877" dependencies = [ "android-tzdata", "iana-time-zone", "js-sys", "num-traits", "serde", - "time 0.1.45", "wasm-bindgen", - "winapi", + "windows-targets", ] [[package]] name = "clap" -version = "4.2.2" +version = "4.4.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9b802d85aaf3a1cdb02b224ba472ebdea62014fccfcb269b95a4d76443b5ee5a" +checksum = "84ed82781cea27b43c9b106a979fe450a13a31aab0500595fb3fc06616de08e6" dependencies = [ "clap_builder", "clap_derive", - "once_cell", ] [[package]] name = "clap_builder" -version = "4.2.2" +version = "4.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "14a1a858f532119338887a4b8e1af9c60de8249cd7bafd68036a489e261e37b6" +checksum = "2bb9faaa7c2ef94b2743a21f5a29e6f0010dff4caa69ac8e9d6cf8b6fa74da08" dependencies = [ "anstream", "anstyle", - "bitflags", "clap_lex", "strsim", ] [[package]] name = "clap_derive" -version = "4.2.0" +version = "4.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3f9644cd56d6b87dbe899ef8b053e331c0637664e9e21a33dfcdc36093f5c5c4" +checksum = "0862016ff20d69b84ef8247369fabf5c008a7417002411897d40ee1f4532b873" dependencies = [ "heck", "proc-macro2", "quote", - "syn 2.0.15", + "syn 2.0.33", ] [[package]] name = "clap_lex" -version = "0.4.1" +version = "0.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8a2dd5a6fe8c6e3502f568a6353e5273bbb15193ad9a89e457b9970798efbea1" - -[[package]] -name = "codespan-reporting" -version = "0.11.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3538270d33cc669650c4b093848450d380def10c331d38c768e34cac80576e6e" -dependencies = [ - "termcolor", - "unicode-width", -] +checksum = "cd7cc57abe963c6d3b9d8be5b06ba7c8957a930305ca90304f24ef040aa6f961" [[package]] name = "colorchoice" @@ -570,32 +565,32 @@ checksum = "acbf1af155f9b9ef647e42cdc158db4b64a1b61f743629225fde6f3e0be2a7c7" [[package]] name = "colored" -version = "2.0.0" +version = "2.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b3616f750b84d8f0de8a58bda93e08e2a81ad3f523089b05f1dffecab48c6cbd" +checksum = "2674ec482fbc38012cf31e6c42ba0177b431a0cb6f15fe40efa5aab1bda516f6" dependencies = [ - "atty", + "is-terminal", "lazy_static", - "winapi", + "windows-sys 0.48.0", ] [[package]] name = "console-api" -version = "0.4.0" +version = "0.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e57ff02e8ad8e06ab9731d5dc72dc23bef9200778eae1a89d555d8c42e5d4a86" +checksum = "c2895653b4d9f1538a83970077cb01dfc77a4810524e51a110944688e916b18e" dependencies = [ "prost", "prost-types", - "tonic 0.8.3", + "tonic", "tracing-core", ] [[package]] name = "console-subscriber" -version = "0.1.8" +version = "0.1.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "22a3a81dfaf6b66bce5d159eddae701e3a002f194d378cbf7be5f053c281d9be" +checksum = "d4cf42660ac07fcebed809cfe561dd8730bcd35b075215e6479c516bcd0d11cb" dependencies = [ "console-api", "crossbeam-channel", @@ -609,7 +604,7 @@ dependencies = [ "thread_local", "tokio", "tokio-stream", - "tonic 0.8.3", + "tonic", "tracing", "tracing-core", "tracing-subscriber", @@ -623,9 +618,9 @@ checksum = "e4c78c047431fee22c1a7bb92e00ad095a02a983affe4d8a72e2a2c62c1b94f3" [[package]] name = "const-oid" -version = "0.9.2" +version = "0.9.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "520fbf3c07483f94e3e3ca9d0cfd913d7718ef2483d2cfd91c0d9e91474ab913" +checksum = "28c122c3980598d243d63d9a704629a2d748d101f278052ff068be5a4423ab6f" [[package]] name = "core-foundation" @@ -645,9 +640,9 @@ checksum = "e496a50fda8aacccc86d7529e2c1e0892dbd0f898a6b5645b5561b89c3210efa" [[package]] name = "cpufeatures" -version = "0.2.6" +version = "0.2.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "280a9f2d8b3a38871a3c8a46fb80db65e5e5ed97da80c4d08bf27fb63e35e181" +checksum = "a17b76ff3a4162b0b27f354a0c87015ddad39d35f9c0c36607a3bdd175dde1f1" dependencies = [ "libc", ] @@ -684,9 +679,9 @@ dependencies = [ [[package]] name = "crossbeam-epoch" -version = "0.9.14" +version = "0.9.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "46bd5f3f85273295a9d14aedfb86f6aadbff6d8f5295c4a9edb08e819dcf5695" +checksum = "ae211234986c545741a7dc064309f67ee1e5ad243d0e48335adc0484d960bcc7" dependencies = [ "autocfg", "cfg-if", @@ -697,9 +692,9 @@ dependencies = [ [[package]] name = "crossbeam-utils" -version = "0.8.15" +version = "0.8.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3c063cd8cc95f5c377ed0d4b49a4b21f632396ff690e8470c29b3359b346984b" +checksum = "5a22b2d63d4d1dc0b7f1b6b2747dd0088008a9be28b6ddf0b1e7d335e3037294" dependencies = [ "cfg-if", ] @@ -724,68 +719,14 @@ dependencies = [ "typenum", ] -[[package]] -name = "ctor" -version = "0.1.26" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6d2301688392eb071b0bf1a37be05c469d3cc4dbbd95df672fe28ab021e6a096" -dependencies = [ - "quote", - "syn 1.0.109", -] - -[[package]] -name = "cxx" -version = "1.0.94" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f61f1b6389c3fe1c316bf8a4dccc90a38208354b330925bce1f74a6c4756eb93" -dependencies = [ - "cc", - "cxxbridge-flags", - "cxxbridge-macro", - "link-cplusplus", -] - -[[package]] -name = "cxx-build" -version = "1.0.94" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "12cee708e8962df2aeb38f594aae5d827c022b6460ac71a7a3e2c3c2aae5a07b" -dependencies = [ - "cc", - "codespan-reporting", - "once_cell", - "proc-macro2", - "quote", - "scratch", - "syn 2.0.15", -] - -[[package]] -name = "cxxbridge-flags" -version = "1.0.94" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7944172ae7e4068c533afbb984114a56c46e9ccddda550499caa222902c7f7bb" - -[[package]] -name = "cxxbridge-macro" -version = "1.0.94" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2345488264226bf682893e25de0769f3360aac9957980ec49361b083ddaa5bc5" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.15", -] - [[package]] name = "dashmap" -version = "5.4.0" +version = "5.5.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "907076dfda823b0b36d2a1bb5f90c96660a5bbcd7729e10727f07858f22c4edc" +checksum = "978747c1d849a7d2ee5e8adc0159961c48fb7e5db2f06af6723b80123bb53856" dependencies = [ "cfg-if", - "hashbrown", + "hashbrown 0.14.0", "lock_api", "once_cell", "parking_lot_core", @@ -838,14 +779,23 @@ dependencies = [ [[package]] name = "der" -version = "0.7.7" +version = "0.7.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c7ed52955ce76b1554f509074bb357d3fb8ac9b51288a65a3fd480d1dfba946" +checksum = "fffa369a668c8af7dbf8b5e56c9f744fbd399949ed171606040001947de40b1c" dependencies = [ - "const-oid 0.9.2", + "const-oid 0.9.5", "zeroize", ] +[[package]] +name = "deranged" +version = "0.3.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f2696e8a945f658fd14dc3b87242e6b80cd0f36ff04ea560fa39082368847946" +dependencies = [ + "serde", +] + [[package]] name = "derive-new" version = "0.5.9" @@ -865,9 +815,9 @@ checksum = "56254986775e3233ffa9c4d7d3faaf6d36a2c09d30b20687e9f88bc8bafc16c8" [[package]] name = "digest" -version = "0.10.6" +version = "0.10.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8168378f4e5023e7218c89c891c0fd8ecdb5e5e4f18cb78f38cf245dd021e76f" +checksum = "9ed9a281f7bc9b7576e61468ba615a66a5c8cfdff42420a70aa82701a3b1e292" dependencies = [ "block-buffer", "crypto-common", @@ -888,15 +838,15 @@ checksum = "1aaf95b3e5c8f23aa320147307562d361db0ae0d51242340f558153b4eb2439b" [[package]] name = "either" -version = "1.8.1" +version = "1.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7fcaabb2fef8c910e7f4c7ce9f67a1283a1715879a7c230ca9d6d1ae31f16d91" +checksum = "a26ae43d7bcc3b814de94796a5e736d4029efb0ee900c12e2d54c993ad1a1e07" [[package]] name = "encoding_rs" -version = "0.8.32" +version = "0.8.33" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "071a31f4ee85403370b58aca746f01041ede6f0da2730960ad001edc2b71b394" +checksum = "7268b386296a025e474d5140678f75d6de9493ae55a5d709eeb9dd08149945e1" dependencies = [ "cfg-if", ] @@ -914,11 +864,17 @@ dependencies = [ "termcolor", ] +[[package]] +name = "equivalent" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5443807d6dff69373d433ab9ef5378ad8df50ca6298caf15de6e52e24aaf54d5" + [[package]] name = "errno" -version = "0.3.1" +version = "0.3.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4bcfec3a70f97c962c307b2d2c56e358cf1d00b558d74262b5f929ee8cc7e73a" +checksum = "136526188508e25c6fef639d7927dfb3e0e3084488bf202267829cf7fc23dbdd" dependencies = [ "errno-dragonfly", "libc", @@ -943,25 +899,28 @@ checksum = "4443176a9f2c162692bd3d352d745ef9413eec5782a80d8fd6f8a1ac692a07f7" [[package]] name = "fastrand" -version = "1.9.0" +version = "2.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e51093e27b0797c359783294ca4f0a911c270184cb10f85783b118614a1501be" -dependencies = [ - "instant", -] +checksum = "6999dc1837253364c2ebb0704ba97994bd874e8f195d665c50b7548f6ea92764" [[package]] name = "filetime" -version = "0.2.21" +version = "0.2.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5cbc844cecaee9d4443931972e1289c8ff485cb4cc2767cb03ca139ed6885153" +checksum = "d4029edd3e734da6fe05b6cd7bd2960760a616bd2ddd0d59a0124746d6272af0" dependencies = [ "cfg-if", "libc", - "redox_syscall 0.2.16", + "redox_syscall", "windows-sys 0.48.0", ] +[[package]] +name = "finl_unicode" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8fcfdc7a0362c9f4444381a9e697c79d435fe65b52a37466fc2c1184cee9edc6" + [[package]] name = "fixedbitset" version = "0.4.2" @@ -970,9 +929,9 @@ checksum = "0ce7134b9999ecaf8bcd65542e436736ef32ddca1b3e06094cb6ec5755203b80" [[package]] name = "flate2" -version = "1.0.25" +version = "1.0.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a8a2db397cb1c8772f31494cb8917e48cd1e64f0fa7efac59fbd741a0a8ce841" +checksum = "c6c98ee8095e9d1dcbf2fcc6d95acccb90d1c81db1e44725c6a984b1dbdfb010" dependencies = [ "crc32fast", "miniz_oxide", @@ -990,7 +949,7 @@ dependencies = [ "serde_json", "serde_yaml", "tokio", - "tonic 0.9.2", + "tonic", "tonic-health", "tracing", ] @@ -1018,9 +977,9 @@ checksum = "00b0228411908ca8685dba7fc2cdd70ec9990a6e753e89b6ac91a84c40fbaf4b" [[package]] name = "form_urlencoded" -version = "1.1.0" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a9c384f161156f5260c24a097c56119f9be8c798586aecc13afbcbe7b7e26bf8" +checksum = "a62bc1cf6f830c2ec14a513a9fb124d0a213a629668a4186f329db21fe045652" dependencies = [ "percent-encoding", ] @@ -1087,7 +1046,7 @@ checksum = "89ca545a94061b6365f2c7355b4b32bd20df3ff95f02da9329b34ccc3bd6ee72" dependencies = [ "proc-macro2", "quote", - "syn 2.0.15", + "syn 2.0.33", ] [[package]] @@ -1128,13 +1087,13 @@ checksum = "70a65dce87dddc032499606b2b3aea836be6fcc7c7cd0b5a2466bce74b90fa52" dependencies = [ "async-stream", "hyper", - "hyper-rustls", + "hyper-rustls 0.23.2", "log", "reqwest", "serde", "serde_json", "thiserror", - "time 0.3.20", + "time", "tokio", "tokio-stream", "url", @@ -1162,13 +1121,13 @@ dependencies = [ [[package]] name = "getrandom" -version = "0.2.9" +version = "0.2.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c85e1d9ab2eadba7e5040d4e09cbd6d072b76a557ad64e797c2cb9d4da21d7e4" +checksum = "be4136b2a15dd319360be1c07d9933517ccf0be8f16bf62a3bee4f0d618df427" dependencies = [ "cfg-if", "libc", - "wasi 0.11.0+wasi-snapshot-preview1", + "wasi", ] [[package]] @@ -1185,9 +1144,9 @@ dependencies = [ [[package]] name = "gimli" -version = "0.27.2" +version = "0.28.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ad0a93d233ebf96623465aad4046a8d3aa4da22d4f4beba5388838c8a434bbb4" +checksum = "6fb8d784f27acf97159b40fc4db5ecd8aa23b9ad5ef69cdd136d3bc80665f0c0" [[package]] name = "glob" @@ -1197,9 +1156,9 @@ checksum = "d2fabcfbdc87f4758337ca535fb41a6d701b65693ce38287d856d1674551ec9b" [[package]] name = "h2" -version = "0.3.18" +version = "0.3.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "17f8a914c2987b688368b5138aa05321db91f4090cf26118185672ad588bce21" +checksum = "91fc23aa11be92976ef4729127f1a74adf36d8436f7816b185d18df956790833" dependencies = [ "bytes", "fnv", @@ -1207,7 +1166,7 @@ dependencies = [ "futures-sink", "futures-util", "http", - "indexmap", + "indexmap 1.9.3", "slab", "tokio", "tokio-util", @@ -1220,9 +1179,24 @@ version = "0.12.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" dependencies = [ - "ahash", + "ahash 0.7.6", +] + +[[package]] +name = "hashbrown" +version = "0.13.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "43a3c133739dddd0d2990f9a4bdf8eb4b21ef50e4851ca85ab661199821d510e" +dependencies = [ + "ahash 0.8.3", ] +[[package]] +name = "hashbrown" +version = "0.14.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2c6201b9ff9fd90a5a3bac2e56a830d0caa509576f0e503818ee82c181b3437a" + [[package]] name = "hdrhistogram" version = "7.5.2" @@ -1244,27 +1218,9 @@ checksum = "95505c38b4572b2d910cecb0281560f54b440a19336cbbcb27bf6ce6adc6f5a8" [[package]] name = "hermit-abi" -version = "0.1.19" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62b467343b94ba476dcb2500d242dadbb39557df889310ac77c5d99100aaac33" -dependencies = [ - "libc", -] - -[[package]] -name = "hermit-abi" -version = "0.2.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ee512640fe35acbfb4bb779db6f0d80704c2cacfa2e39b601ef3e3f47d1ae4c7" -dependencies = [ - "libc", -] - -[[package]] -name = "hermit-abi" -version = "0.3.1" +version = "0.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fed44880c466736ef9a5c5b5facefb5ed0785676d0c02d612db14e54f0d84286" +checksum = "443144c8cdadd93ebf52ddb4056d257f5b52c04d3c804e657d19eb73fc33668b" [[package]] name = "hex" @@ -1281,6 +1237,15 @@ dependencies = [ "digest", ] +[[package]] +name = "home" +version = "0.5.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5444c27eef6923071f7ebcc33e3444508466a76f7a2b93da00ed6e19f30c1ddb" +dependencies = [ + "windows-sys 0.48.0", +] + [[package]] name = "http" version = "0.2.9" @@ -1311,9 +1276,9 @@ checksum = "d897f394bad6a705d5f4104762e116a75639e470d80901eed05a860a95cb1904" [[package]] name = "httpdate" -version = "1.0.2" +version = "1.0.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c4a1e36c821dbe04574f602848a19f742f4fb3c98d40449f11bcad18d6b17421" +checksum = "df3b46402a9d5adb4c86a0cf463f42e19994e3ee891101b1841f30a545cb49a9" [[package]] name = "humantime" @@ -1323,9 +1288,9 @@ checksum = "9a3a5bfb195931eeb336b2a7b4d761daec841b97f947d34394601737a7bba5e4" [[package]] name = "hyper" -version = "0.14.26" +version = "0.14.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab302d72a6f11a3b910431ff93aae7e773078c769f0a3ef15fb9ec692ed147d4" +checksum = "ffb1cfd654a8219eaef89881fdb3bb3b1cdc5fa75ded05d6933b2b382e395468" dependencies = [ "bytes", "futures-channel", @@ -1354,12 +1319,26 @@ dependencies = [ "http", "hyper", "log", - "rustls 0.20.8", + "rustls 0.20.9", "rustls-native-certs", "tokio", "tokio-rustls 0.23.4", ] +[[package]] +name = "hyper-rustls" +version = "0.24.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8d78e1e73ec14cf7375674f74d7dde185c8206fd9dea6fb6295e8a98098aaa97" +dependencies = [ + "futures-util", + "http", + "hyper", + "rustls 0.21.7", + "tokio", + "tokio-rustls 0.24.1", +] + [[package]] name = "hyper-timeout" version = "0.4.1" @@ -1387,9 +1366,9 @@ dependencies = [ [[package]] name = "iana-time-zone" -version = "0.1.56" +version = "0.1.57" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0722cd7114b7de04316e7ea5456a0bbb20e4adb46fd27a3697adb812cff0f37c" +checksum = "2fad5b825842d2b38bd206f3e81d6957625fd7f0a361e345c30e01a0ae2dd613" dependencies = [ "android_system_properties", "core-foundation-sys", @@ -1401,19 +1380,18 @@ dependencies = [ [[package]] name = "iana-time-zone-haiku" -version = "0.1.1" +version = "0.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0703ae284fc167426161c2e3f1da3ea71d94b21bedbcc9494e92b28e334e3dca" +checksum = "f31827a206f56af32e590ba56d5d2d085f558508192593743f16b2306495269f" dependencies = [ - "cxx", - "cxx-build", + "cc", ] [[package]] name = "idna" -version = "0.3.0" +version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e14ddfc70884202db2244c223200c204c2bda1bc6e0998d11b5e024d657209e6" +checksum = "7d20d6b07bfbc108882d88ed8e37d39636dcc260e15e30c45e6ba089610b917c" dependencies = [ "unicode-bidi", "unicode-normalization", @@ -1426,43 +1404,32 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bd070e393353796e801d209ad339e89596eb4c8d430d18ede6a1cced8fafbd99" dependencies = [ "autocfg", - "hashbrown", + "hashbrown 0.12.3", ] [[package]] -name = "instant" -version = "0.1.12" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7a5bbe824c507c5da5956355e86a746d82e0e1464f65d862cc5e71da70e94b2c" -dependencies = [ - "cfg-if", -] - -[[package]] -name = "io-lifetimes" -version = "1.0.10" +name = "indexmap" +version = "2.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c66c74d2ae7e79a5a8f7ac924adbe38ee42a859c6539ad869eb51f0b52dc220" +checksum = "d5477fe2230a79769d8dc68e0eabf5437907c0457a5614a9e8dddb67f65eb65d" dependencies = [ - "hermit-abi 0.3.1", - "libc", - "windows-sys 0.48.0", + "equivalent", + "hashbrown 0.14.0", ] [[package]] name = "ipnet" -version = "2.7.2" +version = "2.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "12b6ee2129af8d4fb011108c73d99a1b83a85977f23b82460c0ae2e25bb4b57f" +checksum = "28b29a3cd74f0f4598934efe3aeba42bae0eb4680554128851ebbecb02af14e6" [[package]] name = "is-terminal" -version = "0.4.7" +version = "0.4.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "adcf93614601c8129ddf72e2d5633df827ba6551541c6d8c59520a371475be1f" +checksum = "cb0889898416213fab133e1d33a0e5858a48177452750691bde3666d0fdbaf8b" dependencies = [ - "hermit-abi 0.3.1", - "io-lifetimes", + "hermit-abi", "rustix", "windows-sys 0.48.0", ] @@ -1478,15 +1445,15 @@ dependencies = [ [[package]] name = "itoa" -version = "1.0.6" +version = "1.0.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "453ad9f582a441959e5f0d088b02ce04cfe8d51a8eaf077f12ac6d3e94164ca6" +checksum = "af150ab688ff2122fcef229be89cb50dd66af9e01a4ff320cc137eecc9bacc38" [[package]] name = "js-sys" -version = "0.3.61" +version = "0.3.64" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "445dde2150c55e483f3d8416706b97ec8e8237c307e5b7b4b8dd15e6af2a0730" +checksum = "c5f195fe497f702db0f318b07fdd68edb16955aed830df8363d837542f8f935a" dependencies = [ "wasm-bindgen", ] @@ -1497,7 +1464,7 @@ version = "8.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6971da4d9c3aa03c3d8f3ff0f4155b534aad021292003895a469716b2a230378" dependencies = [ - "base64 0.21.0", + "base64 0.21.4", "pem 1.1.1", "ring", "serde", @@ -1516,9 +1483,9 @@ dependencies = [ [[package]] name = "libc" -version = "0.2.141" +version = "0.2.148" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3304a64d199bb964be99741b7a14d26972741915b3649639149b2479bb46f4b5" +checksum = "9cdc71e17332e86d2e1d38c1f99edcb6288ee11b815fb1a4b049eaa2114d369b" [[package]] name = "libm" @@ -1526,26 +1493,17 @@ version = "0.2.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f7012b1bbb0719e1097c47611d3898568c546d597c2e74d66f6087edd5233ff4" -[[package]] -name = "link-cplusplus" -version = "1.0.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ecd207c9c713c34f95a097a5b029ac2ce6010530c7b49d7fea24d977dede04f5" -dependencies = [ - "cc", -] - [[package]] name = "linux-raw-sys" -version = "0.3.1" +version = "0.4.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d59d8c75012853d2e872fb56bc8a2e53718e2cafe1a4c823143141c6d90c322f" +checksum = "1a9bad9f94746442c783ca431b22403b519cd7fbeed0533fdd6328b2f2212128" [[package]] name = "lock_api" -version = "0.4.9" +version = "0.4.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "435011366fe56583b16cf956f9df0095b405b82d76425bc8981c0e22e60ec4df" +checksum = "c1cc9717a20b1bb222f333e6a92fd32f7d8a18ddc5a3191a11af45dcbf4dcd16" dependencies = [ "autocfg", "scopeguard", @@ -1553,12 +1511,9 @@ dependencies = [ [[package]] name = "log" -version = "0.4.17" +version = "0.4.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "abb12e687cfb44aa40f41fc3978ef76448f9b6038cad6aef4259d3c095a2382e" -dependencies = [ - "cfg-if", -] +checksum = "b5e6163cb8c49088c2c36f57875e58ccd8c87c7427f7fbd50ea6710b2f3f2e8f" [[package]] name = "lzma-sys" @@ -1577,7 +1532,7 @@ version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8263075bb86c5a1b1427b5ae862e8889656f126e9f77c484496e8b47cf5c5558" dependencies = [ - "regex-automata", + "regex-automata 0.1.10", ] [[package]] @@ -1588,9 +1543,9 @@ checksum = "2532096657941c2fea9c289d370a250971c689d4f143798ff67113ec042024a5" [[package]] name = "matchit" -version = "0.7.0" +version = "0.7.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b87248edafb776e59e6ee64a79086f65890d3510f2c656c000bf2a7e8a0aea40" +checksum = "ed1202b2a6f884ae56f04cff409ab315c5ce26b5e58d7412e484f01fd52f52ef" [[package]] name = "md-5" @@ -1609,15 +1564,15 @@ checksum = "490cc448043f947bae3cbee9c203358d62dbee0db12107a74be5c30ccfd09771" [[package]] name = "memchr" -version = "2.5.0" +version = "2.6.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2dffe52ecf27772e601905b7522cb4ef790d2cc203488bbd0e2fe85fcb74566d" +checksum = "8f232d6ef707e1956a43342693d2a31e72989554d58299d7a88738cc95b0d35c" [[package]] name = "memoffset" -version = "0.8.0" +version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d61c719bcfbcf5d62b3a09efa6088de8c54bc0bfcd3ea7ae39fcc186108b8de1" +checksum = "5a634b1c61a95585bd15607c6ab0c4e5b226e695ff2800ba0cdccddf208c406c" dependencies = [ "autocfg", ] @@ -1636,23 +1591,22 @@ checksum = "68354c5c6bd36d73ff3feceb05efa59b6acb7626617f4962be322a825e61f79a" [[package]] name = "miniz_oxide" -version = "0.6.2" +version = "0.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b275950c28b37e794e8c55d88aeb5e139d0ce23fdbbeda68f8d7174abdf9e8fa" +checksum = "e7810e0be55b428ada41041c41f32c9f1a42817901b4ccf45fa3d4b6561e74c7" dependencies = [ "adler", ] [[package]] name = "mio" -version = "0.8.6" +version = "0.8.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b9d9a46eff5b4ff64b45a9e316a6d1e0bc719ef429cbec4dc630684212bfdf9" +checksum = "927a765cd3fc26206e66b296465fa9d3e5ab003e651c1b3c060e7956d96b19d2" dependencies = [ "libc", - "log", - "wasi 0.11.0+wasi-snapshot-preview1", - "windows-sys 0.45.0", + "wasi", + "windows-sys 0.48.0", ] [[package]] @@ -1701,9 +1655,9 @@ dependencies = [ [[package]] name = "num-bigint" -version = "0.4.3" +version = "0.4.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f93ab6289c7b344a8a9f60f88d80aa20032336fe78da341afc91c8a2341fc75f" +checksum = "608e7659b5c3d7cba262d894801b9ec9d00de989e8a82bd4bef91d08da45cdc0" dependencies = [ "autocfg", "num-integer", @@ -1712,9 +1666,9 @@ dependencies = [ [[package]] name = "num-bigint-dig" -version = "0.8.2" +version = "0.8.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2399c9463abc5f909349d8aa9ba080e0b88b3ce2885389b60b993f39b1a56905" +checksum = "dc84195820f291c7697304f3cbdadd1cb7199c0efc917ff5eafd71225c136151" dependencies = [ "byteorder", "lazy_static", @@ -1750,9 +1704,9 @@ dependencies = [ [[package]] name = "num-traits" -version = "0.2.15" +version = "0.2.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "578ede34cf02f8924ab9447f50c28075b4d3e5b269972345e7e0372b38c6cdcd" +checksum = "f30b0abd723be7e2ffca1272140fac1a2f084c77ec3e123c192b66af1ee9e6c2" dependencies = [ "autocfg", "libm", @@ -1760,11 +1714,11 @@ dependencies = [ [[package]] name = "num_cpus" -version = "1.15.0" +version = "1.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0fac9e2da13b5eb447a6ce3d392f23a29d8694bff781bf03a16cd9ac8697593b" +checksum = "4161fcb6d602d4d2081af7c3a45852d875a03dd337a6bfdd6e06407b61342a43" dependencies = [ - "hermit-abi 0.2.6", + "hermit-abi", "libc", ] @@ -1779,26 +1733,26 @@ dependencies = [ [[package]] name = "object" -version = "0.30.3" +version = "0.32.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ea86265d3d3dcb6a27fc51bd29a4bf387fae9d2986b823079d4986af253eb439" +checksum = "9cf5f9dd3933bd50a9e1f149ec995f39ae2c496d31fd772c1fd45ebc27e902b0" dependencies = [ "memchr", ] [[package]] name = "once_cell" -version = "1.17.1" +version = "1.18.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b7e5500299e16ebb147ae15a00a942af264cf3688f47923b8fc2cd5858f23ad3" +checksum = "dd8b5dd2ae5ed71462c540258bedcb51965123ad7e7ccf4b9a8cafaa4a63576d" [[package]] name = "openssl" -version = "0.10.55" +version = "0.10.57" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "345df152bc43501c5eb9e4654ff05f794effb78d4efe3d53abc158baddc0703d" +checksum = "bac25ee399abb46215765b1cb35bc0212377e58a061560d8b29b024fd0430e7c" dependencies = [ - "bitflags", + "bitflags 2.4.0", "cfg-if", "foreign-types", "libc", @@ -1815,7 +1769,7 @@ checksum = "a948666b637a0f465e8564c73e89d4dde00d72d4d473cc972f390fc3dcee7d9c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.15", + "syn 2.0.33", ] [[package]] @@ -1826,18 +1780,18 @@ checksum = "ff011a302c396a5197692431fc1948019154afc178baf7d8e37367442a4601cf" [[package]] name = "openssl-src" -version = "111.25.3+1.1.1t" +version = "300.1.3+3.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "924757a6a226bf60da5f7dd0311a34d2b52283dd82ddeb103208ddc66362f80c" +checksum = "cd2c101a165fff9935e34def4669595ab1c7847943c42be86e21503e482be107" dependencies = [ "cc", ] [[package]] name = "openssl-sys" -version = "0.9.90" +version = "0.9.93" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "374533b0e45f3a7ced10fcaeccca020e66656bc03dac384f852e4e5a7a8104a6" +checksum = "db4d56a4c0478783083cfafcc42493dd4a981d41669da64b4572a2a089b51b1d" dependencies = [ "cc", "libc", @@ -1846,15 +1800,6 @@ dependencies = [ "vcpkg", ] -[[package]] -name = "output_vt100" -version = "0.1.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "628223faebab4e3e40667ee0b2336d34a5b960ff60ea743ddfdbcf7770bcfb66" -dependencies = [ - "winapi", -] - [[package]] name = "overload" version = "0.1.1" @@ -1873,15 +1818,15 @@ dependencies = [ [[package]] name = "parking_lot_core" -version = "0.9.7" +version = "0.9.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9069cbb9f99e3a5083476ccb29ceb1de18b9118cafa53e90c9551235de2b9521" +checksum = "93f00c865fe7cabf650081affecd3871070f26767e7b2070a3ffae14c654b447" dependencies = [ "cfg-if", "libc", - "redox_syscall 0.2.16", + "redox_syscall", "smallvec", - "windows-sys 0.45.0", + "windows-targets", ] [[package]] @@ -2084,7 +2029,7 @@ dependencies = [ "serde_json", "sha256", "sqlparser", - "time 0.3.20", + "time", "tokio", "tracing", "tracing-appender", @@ -2107,7 +2052,7 @@ version = "2.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6b13fe415cdf3c8e44518e18a7c95a13431d9bdf6d15367d82b23c377fdd441a" dependencies = [ - "base64 0.21.0", + "base64 0.21.4", "serde", ] @@ -2122,18 +2067,18 @@ dependencies = [ [[package]] name = "percent-encoding" -version = "2.2.0" +version = "2.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "478c572c3d73181ff3c2539045f6eb99e5491218eae919370993b890cdbdd98e" +checksum = "9b2a4787296e9989611394c33f193f676704af1686e70b8f8033ab5ba9a35a94" [[package]] name = "petgraph" -version = "0.6.3" +version = "0.6.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4dd7d28ee937e54fe3080c91faa1c3a46c06de6252988a7f4592ba2310ef22a4" +checksum = "e1d3afd2628e69da2be385eb6f2fd57c8ac7977ceeff6dc166ff1657b0e386a9" dependencies = [ "fixedbitset", - "indexmap", + "indexmap 2.0.0", ] [[package]] @@ -2150,7 +2095,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e2de42ee35f9694def25c37c15f564555411d9904b48e33680618ee7359080dc" dependencies = [ "async-trait", - "base64 0.21.0", + "base64 0.21.4", "bytes", "chrono", "derive-new", @@ -2164,56 +2109,56 @@ dependencies = [ "ring", "stringprep", "thiserror", - "time 0.3.20", + "time", "tokio", - "tokio-rustls 0.24.0", + "tokio-rustls 0.24.1", "tokio-util", "x509-certificate", ] [[package]] name = "phf" -version = "0.11.1" +version = "0.11.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "928c6535de93548188ef63bb7c4036bd415cd8f36ad25af44b9789b2ee72a48c" +checksum = "ade2d8b8f33c7333b51bcf0428d37e217e9f32192ae4772156f65063b8ce03dc" dependencies = [ "phf_shared", ] [[package]] name = "phf_shared" -version = "0.11.1" +version = "0.11.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e1fb5f6f826b772a8d4c0394209441e7d37cbbb967ae9c7e0e8134365c9ee676" +checksum = "90fcb95eef784c2ac79119d1dd819e162b5da872ce6f3c3abe1e8ca1c082f72b" dependencies = [ - "siphasher", + "siphasher 0.3.11", ] [[package]] name = "pin-project" -version = "1.0.12" +version = "1.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ad29a609b6bcd67fee905812e544992d216af9d755757c05ed2d0e15a74c6ecc" +checksum = "fda4ed1c6c173e3fc7a83629421152e01d7b1f9b7f65fb301e490e8cfc656422" dependencies = [ "pin-project-internal", ] [[package]] name = "pin-project-internal" -version = "1.0.12" +version = "1.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "069bdb1e05adc7a8990dce9cc75370895fbe4e3d58b9b73bf1aee56359344a55" +checksum = "4359fd9c9171ec6e8c62926d6faaf553a8dc3f64e1507e76da7911b4f6a04405" dependencies = [ "proc-macro2", "quote", - "syn 1.0.109", + "syn 2.0.33", ] [[package]] name = "pin-project-lite" -version = "0.2.9" +version = "0.2.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e0a7ae3ac2f1173085d398531c705756c94a4c56843785df85a60c1a0afac116" +checksum = "8afb450f006bf6385ca15ef45d71d2288452bc3683ce2e2cacc0d18e4be60b58" [[package]] name = "pin-utils" @@ -2251,9 +2196,9 @@ checksum = "26072860ba924cbfa98ea39c8c19b4dd6a4a25423dbdf219c1eca91aa0cf6964" [[package]] name = "postgres" -version = "0.19.5" +version = "0.19.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0bed5017bc2ff49649c0075d0d7a9d676933c1292480c1d137776fb205b5cd18" +checksum = "7915b33ed60abc46040cbcaa25ffa1c7ec240668e0477c4f3070786f5916d451" dependencies = [ "bytes", "fallible-iterator", @@ -2302,11 +2247,11 @@ dependencies = [ [[package]] name = "postgres-protocol" -version = "0.6.5" +version = "0.6.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "78b7fa9f396f51dffd61546fd8573ee20592287996568e6175ceb0f8699ad75d" +checksum = "49b6c5ef183cd3ab4ba005f1ca64c21e8bd97ce4699cfea9e8d9a2c4958ca520" dependencies = [ - "base64 0.21.0", + "base64 0.21.4", "byteorder", "bytes", "fallible-iterator", @@ -2320,9 +2265,9 @@ dependencies = [ [[package]] name = "postgres-types" -version = "0.2.5" +version = "0.2.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f028f05971fe20f512bcc679e2c10227e57809a3af86a7606304435bc8896cd6" +checksum = "8d2234cdee9408b523530a9b6d2d6b373d1db34f6a8e51dc03ded1828d7fb67c" dependencies = [ "array-init", "bytes", @@ -2342,13 +2287,11 @@ checksum = "5b40af805b3121feab8a3c29f04d8ad262fa8e0561883e7653e024ae4479e6de" [[package]] name = "pretty_assertions" -version = "1.3.0" +version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a25e9bcb20aa780fd0bb16b72403a9064d6b3f22f026946029acb941a50af755" +checksum = "af7cee1a6c8a5b9208b3cb1061f10c0cb689087b3d8ce85fb9d2dd7a29b6ba66" dependencies = [ - "ctor", "diff", - "output_vt100", "yansi", ] @@ -2397,9 +2340,9 @@ dependencies = [ [[package]] name = "proc-macro2" -version = "1.0.56" +version = "1.0.67" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2b63bdb0cd06f1f4dedf69b254734f9b45af66e4a031e42a7480257d9898b435" +checksum = "3d433d9f1a3e8c1263d9456598b16fec66f4acc9a74dacffd35c7bb09b3a1328" dependencies = [ "unicode-ident", ] @@ -2470,7 +2413,7 @@ dependencies = [ "serde", "serde_json", "sqlparser", - "tonic 0.9.2", + "tonic", "tonic-reflection", ] @@ -2502,9 +2445,9 @@ checksum = "a993555f31e5a609f617c12db6250dedcac1b0a85076912c436e6fc9b2c8e6a3" [[package]] name = "quote" -version = "1.0.26" +version = "1.0.33" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4424af4bf778aae2051a77b60283332f386554255d722233d09fbfc7e30da2fc" +checksum = "5267fca4496028628a95160fc423a33e8b2e6af8a5302579e322e4b520293cae" dependencies = [ "proc-macro2", ] @@ -2567,29 +2510,20 @@ dependencies = [ "num_cpus", ] -[[package]] -name = "redox_syscall" -version = "0.2.16" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fb5a58c1855b4b6819d59012155603f0b22ad30cad752600aadfcb695265519a" -dependencies = [ - "bitflags", -] - [[package]] name = "redox_syscall" version = "0.3.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "567664f262709473930a4bf9e51bf2ebf3348f2e748ccc50dea20646858f8f29" dependencies = [ - "bitflags", + "bitflags 1.3.2", ] [[package]] name = "refinery" -version = "0.8.7" +version = "0.8.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a65acc0e1f0e7c80480aacb5087e3509b85806985f54dd7dfeafee4c3128943a" +checksum = "529664dbccc0a296947615c997a857912d72d1c44be1fafb7bae54ecfa7a8c24" dependencies = [ "refinery-core", "refinery-macros", @@ -2597,9 +2531,9 @@ dependencies = [ [[package]] name = "refinery-core" -version = "0.8.7" +version = "0.8.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6d3af3c282b6dae2e9952b28a637ee8148ee34ef6edfd0415df6de2799d4fc3b" +checksum = "e895cb870cf06e92318cbbeb701f274d022d5ca87a16fa8244e291cd035ef954" dependencies = [ "async-trait", "cfg-if", @@ -2607,38 +2541,39 @@ dependencies = [ "log", "regex", "serde", - "siphasher", + "siphasher 1.0.0", "thiserror", - "time 0.3.20", + "time", "tokio", "tokio-postgres", - "toml 0.5.11", + "toml 0.7.8", "url", "walkdir", ] [[package]] name = "refinery-macros" -version = "0.8.7" +version = "0.8.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3d6feea59ecc7e3ebf27244271063f3dfb59226629e2661f535c771fcb358ae9" +checksum = "123e8b80f8010c3ae38330c81e76938fc7adf6cdbfbaad20295bb8c22718b4f1" dependencies = [ "proc-macro2", "quote", "refinery-core", "regex", - "syn 1.0.109", + "syn 2.0.33", ] [[package]] name = "regex" -version = "1.7.3" +version = "1.9.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8b1f693b24f6ac912f4893ef08244d70b6067480d2f1a46e950c9691e6749d1d" +checksum = "697061221ea1b4a94a624f67d0ae2bfe4e22b8a17b6a192afb11046542cc8c47" dependencies = [ "aho-corasick", "memchr", - "regex-syntax", + "regex-automata 0.3.8", + "regex-syntax 0.7.5", ] [[package]] @@ -2647,7 +2582,18 @@ version = "0.1.10" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6c230d73fb8d8c1b9c0b3135c5142a8acee3a0558fb8db5cf1cb65f8d7862132" dependencies = [ - "regex-syntax", + "regex-syntax 0.6.29", +] + +[[package]] +name = "regex-automata" +version = "0.3.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c2f401f4955220693b56f8ec66ee9c78abffd8d1c4f23dc41a23839eb88f0795" +dependencies = [ + "aho-corasick", + "memchr", + "regex-syntax 0.7.5", ] [[package]] @@ -2656,6 +2602,12 @@ version = "0.6.29" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f162c6dd7b008981e4d40210aca20b4bd0f9b60ca9271061b07f78537722f2e1" +[[package]] +name = "regex-syntax" +version = "0.7.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dbb5fb1acd8a1a18b3dd5be62d25485eb770e05afb408a9627d14d451bae12da" + [[package]] name = "rend" version = "0.4.0" @@ -2667,12 +2619,12 @@ dependencies = [ [[package]] name = "reqwest" -version = "0.11.16" +version = "0.11.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "27b71749df584b7f4cac2c426c127a7c785a5106cc98f7a8feb044115f0fa254" +checksum = "3e9ad3fe7488d7e34558a2033d45a0c90b72d97b4f80705666fea71472e2e6a1" dependencies = [ "async-compression", - "base64 0.21.0", + "base64 0.21.4", "bytes", "encoding_rs", "futures-core", @@ -2681,7 +2633,7 @@ dependencies = [ "http", "http-body", "hyper", - "hyper-rustls", + "hyper-rustls 0.24.1", "hyper-tls", "ipnet", "js-sys", @@ -2691,21 +2643,21 @@ dependencies = [ "once_cell", "percent-encoding", "pin-project-lite", - "rustls 0.20.8", - "rustls-pemfile 1.0.2", + "rustls 0.21.7", + "rustls-pemfile 1.0.3", "serde", "serde_json", "serde_urlencoded", "tokio", "tokio-native-tls", - "tokio-rustls 0.23.4", + "tokio-rustls 0.24.1", "tokio-util", "tower-service", "url", "wasm-bindgen", "wasm-bindgen-futures", "web-sys", - "webpki-roots", + "webpki-roots 0.25.2", "winreg", ] @@ -2738,13 +2690,13 @@ checksum = "0200c8230b013893c0b2d6213d6ec64ed2b9be2e0e016682b7224ff82cff5c58" dependencies = [ "bitvec", "bytecheck", - "hashbrown", + "hashbrown 0.12.3", "ptr_meta", "rend", "rkyv_derive", "seahash", "tinyvec", - "uuid 1.4.0", + "uuid 1.4.1", ] [[package]] @@ -2780,14 +2732,12 @@ dependencies = [ [[package]] name = "rust_decimal" -version = "1.30.0" +version = "1.32.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d0446843641c69436765a35a5a77088e28c2e6a12da93e84aa3ab1cd4aa5a042" +checksum = "a4c4216490d5a413bc6d10fa4742bd7d4955941d062c0ef873141d6b0e7b30fd" dependencies = [ "arrayvec", "borsh", - "bytecheck", - "byteorder", "bytes", "num-traits", "postgres", @@ -2806,13 +2756,12 @@ checksum = "d626bb9dae77e28219937af045c257c28bfd3f69333c512553507f5f9798cb76" [[package]] name = "rustix" -version = "0.37.11" +version = "0.38.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "85597d61f83914ddeba6a47b3b8ffe7365107221c2e557ed94426489fefb5f77" +checksum = "d7db8590df6dfcd144d22afd1b83b36c21a18d7cbc1dc4bb5295a8712e9eb662" dependencies = [ - "bitflags", + "bitflags 2.4.0", "errno", - "io-lifetimes", "libc", "linux-raw-sys", "windows-sys 0.48.0", @@ -2820,9 +2769,9 @@ dependencies = [ [[package]] name = "rustls" -version = "0.20.8" +version = "0.20.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fff78fc74d175294f4e83b28343315ffcfb114b156f0185e9741cb5570f50e2f" +checksum = "1b80e3dec595989ea8510028f30c408a4630db12c9cbb8de34203b89d6577e99" dependencies = [ "log", "ring", @@ -2832,24 +2781,24 @@ dependencies = [ [[package]] name = "rustls" -version = "0.21.0" +version = "0.21.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07180898a28ed6a7f7ba2311594308f595e3dd2e3c3812fa0a80a47b45f17e5d" +checksum = "cd8d6c9f025a446bc4d18ad9632e69aec8f287aa84499ee335599fabd20c3fd8" dependencies = [ "log", "ring", - "rustls-webpki", + "rustls-webpki 0.101.5", "sct", ] [[package]] name = "rustls-native-certs" -version = "0.6.2" +version = "0.6.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0167bac7a9f490495f3c33013e7722b53cb087ecbe082fb0c6387c96f634ea50" +checksum = "a9aace74cb666635c918e9c12bc0d348266037aa8eb599b5cba565709a8dff00" dependencies = [ "openssl-probe", - "rustls-pemfile 1.0.2", + "rustls-pemfile 1.0.3", "schannel", "security-framework", ] @@ -2865,18 +2814,28 @@ dependencies = [ [[package]] name = "rustls-pemfile" -version = "1.0.2" +version = "1.0.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d194b56d58803a43635bdc398cd17e383d6f71f9182b9a192c127ca42494a59b" +checksum = "2d3987094b1d07b653b7dfdc3f70ce9a1da9c51ac18c1b06b662e4f9a0e9f4b2" dependencies = [ - "base64 0.21.0", + "base64 0.21.4", ] [[package]] name = "rustls-webpki" -version = "0.100.2" +version = "0.100.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e98ff011474fa39949b7e5c0428f9b4937eda7da7848bbb947786b7be0b27dab" +checksum = "5f6a5fc258f1c1276dfe3016516945546e2d5383911efc0fc4f1cdc5df3a4ae3" +dependencies = [ + "ring", + "untrusted", +] + +[[package]] +name = "rustls-webpki" +version = "0.101.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "45a27e3b59326c16e23d30aeb7a36a24cc0d29e71d68ff611cdfb4a01d013bed" dependencies = [ "ring", "untrusted", @@ -2884,15 +2843,15 @@ dependencies = [ [[package]] name = "rustversion" -version = "1.0.12" +version = "1.0.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4f3208ce4d8448b3f3e7d168a73f5e0c43a61e32930de3bceeccedb388b6bf06" +checksum = "7ffc183a10b4478d04cbbbfc96d0873219d962dd5accaff2ffbd4ceb7df837f4" [[package]] name = "ryu" -version = "1.0.13" +version = "1.0.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f91339c0467de62360649f8d3e185ca8de4224ff281f66000de5eb2a77a79041" +checksum = "1ad4cc8da4ef723ed60bced201181d83791ad433213d8c24efffda1eec85d741" [[package]] name = "same-file" @@ -2905,24 +2864,18 @@ dependencies = [ [[package]] name = "schannel" -version = "0.1.21" +version = "0.1.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "713cfb06c7059f3588fb8044c0fad1d09e3c01d225e25b9220dbfdcf16dbb1b3" +checksum = "0c3733bf4cf7ea0880754e19cb5a462007c4a8c1914bff372ccc95b464f1df88" dependencies = [ - "windows-sys 0.42.0", + "windows-sys 0.48.0", ] [[package]] name = "scopeguard" -version = "1.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d29ab0c6d3fc0ee92fe66e2d99f700eab17a8d57d1c1d3b748380fb20baa78cd" - -[[package]] -name = "scratch" -version = "1.0.5" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1792db035ce95be60c3f8853017b3999209281c24e2ba5bc8e59bf97a0c590c1" +checksum = "94143f37725109f92c262ed2cf5e59bce7498c01bcc1502d7b9afe439a4e9f49" [[package]] name = "sct" @@ -2951,11 +2904,11 @@ dependencies = [ [[package]] name = "security-framework" -version = "2.8.2" +version = "2.9.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a332be01508d814fed64bf28f798a146d73792121129962fdf335bb3c49a4254" +checksum = "05b64fb303737d99b81884b2c63433e9ae28abebe5eb5045dcdd175dc2ecf4de" dependencies = [ - "bitflags", + "bitflags 1.3.2", "core-foundation", "core-foundation-sys", "libc", @@ -2964,9 +2917,9 @@ dependencies = [ [[package]] name = "security-framework-sys" -version = "2.8.0" +version = "2.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "31c9bb296072e961fcbd8853511dd39c2d8be2deb1e17c6860b1d30732b323b4" +checksum = "e932934257d3b408ed8f30db49d85ea163bfe74961f017f405b025af298f0c7a" dependencies = [ "core-foundation-sys", "libc", @@ -2974,38 +2927,38 @@ dependencies = [ [[package]] name = "serde" -version = "1.0.160" +version = "1.0.188" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bb2f3770c8bce3bcda7e149193a069a0f4365bda1fa5cd88e03bca26afc1216c" +checksum = "cf9e0fcba69a370eed61bcf2b728575f726b50b55cba78064753d708ddc7549e" dependencies = [ "serde_derive", ] [[package]] name = "serde_bytes" -version = "0.11.9" +version = "0.11.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "416bda436f9aab92e02c8e10d49a15ddd339cea90b6e340fe51ed97abb548294" +checksum = "ab33ec92f677585af6d88c65593ae2375adde54efdbf16d597f2cbc7a6d368ff" dependencies = [ "serde", ] [[package]] name = "serde_derive" -version = "1.0.160" +version = "1.0.188" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "291a097c63d8497e00160b166a967a4a79c64f3facdd01cbd7502231688d77df" +checksum = "4eca7ac642d82aa35b60049a6eccb4be6be75e599bd2e9adb5f875a737654af2" dependencies = [ "proc-macro2", "quote", - "syn 2.0.15", + "syn 2.0.33", ] [[package]] name = "serde_json" -version = "1.0.97" +version = "1.0.107" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bdf3bf93142acad5821c99197022e170842cdbc1c30482b98750c688c640842a" +checksum = "6b420ce6e3d8bd882e9b243c6eed35dbc9a6110c9769e74b584e0d68d1f20c65" dependencies = [ "itoa", "ryu", @@ -3014,9 +2967,9 @@ dependencies = [ [[package]] name = "serde_spanned" -version = "0.6.2" +version = "0.6.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "93107647184f6027e3b7dcb2e11034cf95ffa1e3a682c67951963ac69c1c007d" +checksum = "96426c9936fd7a0124915f9185ea1d20aa9445cc9821142f0a73bc9207a2e186" dependencies = [ "serde", ] @@ -3035,11 +2988,11 @@ dependencies = [ [[package]] name = "serde_yaml" -version = "0.9.21" +version = "0.9.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d9d684e3ec7de3bf5466b32bd75303ac16f0736426e5a4e0d6e489559ce1249c" +checksum = "1a49e178e4452f45cb61d0cd8cebc1b0fafd3e41929e996cef79aa3aca91f574" dependencies = [ - "indexmap", + "indexmap 2.0.0", "itoa", "ryu", "serde", @@ -3048,9 +3001,9 @@ dependencies = [ [[package]] name = "sha2" -version = "0.10.6" +version = "0.10.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "82e6b795fe2e3b1e845bafcb27aa35405c4d47cdfc92af5fc8d3002f76cebdc0" +checksum = "479fb9d862239e610720565ca91403019f2f00410f1864c5aa7479b950a76ed8" dependencies = [ "cfg-if", "cpufeatures", @@ -3059,12 +3012,15 @@ dependencies = [ [[package]] name = "sha256" -version = "1.1.3" +version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f9f8b5de2bac3a4ae28e9b611072a8e326d9b26c8189c0972d4c321fa684f1f" +checksum = "7895c8ae88588ccead14ff438b939b0c569cd619116f14b4d13fdff7b8333386" dependencies = [ + "async-trait", + "bytes", "hex", "sha2", + "tokio", ] [[package]] @@ -3106,48 +3062,53 @@ dependencies = [ "num-bigint", "num-traits", "thiserror", - "time 0.3.20", + "time", ] [[package]] name = "simple_logger" -version = "4.1.0" +version = "4.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e78beb34673091ccf96a8816fce8bfd30d1292c7621ca2bcb5f2ba0fae4f558d" +checksum = "2230cd5c29b815c9b699fb610b49a5ed65588f3509d9f0108be3a885da629333" dependencies = [ - "atty", "colored", "log", - "time 0.3.20", + "time", "windows-sys 0.42.0", ] [[package]] name = "siphasher" -version = "0.3.10" +version = "0.3.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7bd3e3206899af3f8b12af284fafc038cc1dc2b41d1b89dd17297221c5d225de" +checksum = "38b58827f4464d87d377d175e90bf58eb00fd8716ff0a62f80356b5e61555d0d" + +[[package]] +name = "siphasher" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "54ac45299ccbd390721be55b412d41931911f654fa99e2cb8bfb57184b2061fe" [[package]] name = "slab" -version = "0.4.8" +version = "0.4.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6528351c9bc8ab22353f9d776db39a20288e8d6c37ef8cfe3317cf875eecfc2d" +checksum = "8f92a496fb766b417c996b9c5e57daf2f7ad3b0bebe1ccfca4856390e3d3bb67" dependencies = [ "autocfg", ] [[package]] name = "smallvec" -version = "1.10.0" +version = "1.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a507befe795404456341dfab10cef66ead4c041f62b8b11bbb92bffe5d0953e0" +checksum = "62bb4feee49fdd9f707ef802e22365a35de4b7b299de4763d44bfea899442ff9" [[package]] name = "snafu" -version = "0.7.4" +version = "0.7.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cb0656e7e3ffb70f6c39b3c2a86332bb74aa3c679da781642590f3c1118c5045" +checksum = "e4de37ad025c587a29e8f3f5605c00f70b98715ef90b9061a815b9e59e9042d6" dependencies = [ "backtrace", "doc-comment", @@ -3156,9 +3117,9 @@ dependencies = [ [[package]] name = "snafu-derive" -version = "0.7.4" +version = "0.7.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "475b3bbe5245c26f2d8a6f62d67c1f30eb9fffeccee721c45d162c3ebbdf81b2" +checksum = "990079665f075b699031e9c08fd3ab99be5029b96f3b78dc0709e8f77e4efebf" dependencies = [ "heck", "proc-macro2", @@ -3178,9 +3139,9 @@ dependencies = [ [[package]] name = "socket2" -version = "0.5.2" +version = "0.5.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6d283f86695ae989d1e18440a943880967156325ba025f05049946bff47bcc2b" +checksum = "4031e820eb552adee9295814c0ced9e5cf38ddf1e8b7d566d6de8e2538ea989e" dependencies = [ "libc", "windows-sys 0.48.0", @@ -3209,7 +3170,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9d1e996ef02c474957d681f1b05213dfb0abab947b446a62d37770b23500184a" dependencies = [ "base64ct", - "der 0.7.7", + "der 0.7.8", ] [[package]] @@ -3237,10 +3198,11 @@ dependencies = [ [[package]] name = "stringprep" -version = "0.1.2" +version = "0.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8ee348cb74b87454fff4b551cbf727025810a004f88aeacae7f85b87f4e9a1c1" +checksum = "bb41d74e231a107a1b4ee36bd1214b11285b77768d2e3824aedafa988fd36ee6" dependencies = [ + "finl_unicode", "unicode-bidi", "unicode-normalization", ] @@ -3253,9 +3215,9 @@ checksum = "73473c0e59e6d5812c5dfe2a064a6444949f089e20eec9a2e5506596494e4623" [[package]] name = "subtle" -version = "2.4.1" +version = "2.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6bdef32e8150c2a081110b42772ffe7d7c9032b606bc226c8260fd97e0976601" +checksum = "81cdd64d312baedb58e21336b31bc043b77e01cc99033ce76ef539f78e965ebc" [[package]] name = "syn" @@ -3270,9 +3232,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.15" +version = "2.0.33" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a34fcf3e8b60f57e6a14301a2e916d323af98b0ea63c599441eec8558660c822" +checksum = "9caece70c63bfba29ec2fed841a09851b14a235c60010fa4de58089b6c025668" dependencies = [ "proc-macro2", "quote", @@ -3293,9 +3255,9 @@ checksum = "55937e1799185b12863d447f42597ed69d9928686b8d88a1df17376a097d8369" [[package]] name = "tar" -version = "0.4.38" +version = "0.4.40" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4b55807c0344e1e6c04d7c965f5289c39a8d94ae23ed5c0b57aabac549f871c6" +checksum = "b16afcea1f22891c49a00c751c7b63b2233284064f11a200fc624137c51e2ddb" dependencies = [ "filetime", "libc", @@ -3304,15 +3266,15 @@ dependencies = [ [[package]] name = "tempfile" -version = "3.5.0" +version = "3.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b9fbec84f381d5795b08656e4912bec604d162bff9291d6189a78f4c8ab87998" +checksum = "cb94d2f3cc536af71caac6b6fcebf65860b347e7ce0cc9ebe8f70d3e521054ef" dependencies = [ "cfg-if", "fastrand", - "redox_syscall 0.3.5", + "redox_syscall", "rustix", - "windows-sys 0.45.0", + "windows-sys 0.48.0", ] [[package]] @@ -3326,22 +3288,22 @@ dependencies = [ [[package]] name = "thiserror" -version = "1.0.40" +version = "1.0.48" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "978c9a314bd8dc99be594bc3c175faaa9794be04a5a5e153caba6915336cebac" +checksum = "9d6d7a740b8a666a7e828dd00da9c0dc290dff53154ea77ac109281de90589b7" dependencies = [ "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "1.0.40" +version = "1.0.48" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f9456a42c5b0d803c8cd86e73dd7cc9edd429499f37a3550d286d5e86720569f" +checksum = "49922ecae66cc8a249b77e68d1d0623c1b2c514f0060c27cdc68bd62a1219d35" dependencies = [ "proc-macro2", "quote", - "syn 2.0.15", + "syn 2.0.33", ] [[package]] @@ -3356,21 +3318,11 @@ dependencies = [ [[package]] name = "time" -version = "0.1.45" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1b797afad3f312d1c66a56d11d0316f916356d11bd158fbc6ca6389ff6bf805a" -dependencies = [ - "libc", - "wasi 0.10.0+wasi-snapshot-preview1", - "winapi", -] - -[[package]] -name = "time" -version = "0.3.20" +version = "0.3.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd0cbfecb4d19b5ea75bb31ad904eb5b9fa13f21079c3b92017ebdf4999a5890" +checksum = "17f6bb557fd245c28e6411aa56b6403c689ad95061f50e4be16c274e70a17e48" dependencies = [ + "deranged", "itoa", "libc", "num_threads", @@ -3381,15 +3333,15 @@ dependencies = [ [[package]] name = "time-core" -version = "0.1.0" +version = "0.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2e153e1f1acaef8acc537e68b44906d2db6436e2b35ac2c6b42640fff91f00fd" +checksum = "7300fbefb4dadc1af235a9cef3737cea692a9d97e1b9cbcd4ebdae6f8868e6fb" [[package]] name = "time-macros" -version = "0.2.8" +version = "0.2.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fd80a657e71da814b8e5d60d3374fc6d35045062245d80224748ae522dd76f36" +checksum = "1a942f44339478ef67935ab2bbaec2fb0322496cf3cbe84b261e06ac3814c572" dependencies = [ "time-core", ] @@ -3411,11 +3363,11 @@ checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.27.0" +version = "1.32.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d0de47a4eecbe11f498978a9b29d792f0d2692d1dd003650c24c76510e3bc001" +checksum = "17ed6077ed6cd6c74735e21f37eb16dc3935f96878b1fe961074089cc80893f9" dependencies = [ - "autocfg", + "backtrace", "bytes", "libc", "mio", @@ -3423,10 +3375,10 @@ dependencies = [ "parking_lot", "pin-project-lite", "signal-hook-registry", - "socket2 0.4.9", + "socket2 0.5.4", "tokio-macros", "tracing", - "windows-sys 0.45.0", + "windows-sys 0.48.0", ] [[package]] @@ -3441,13 +3393,13 @@ dependencies = [ [[package]] name = "tokio-macros" -version = "2.0.0" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "61a573bdc87985e9d6ddeed1b3d864e8a302c847e40d647746df2f1de209d1ce" +checksum = "630bdcf245f78637c13ec01ffae6187cca34625e8c63150d424b59e55af2675e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.15", + "syn 2.0.33", ] [[package]] @@ -3474,9 +3426,9 @@ dependencies = [ [[package]] name = "tokio-postgres" -version = "0.7.8" +version = "0.7.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6e89f6234aa8fd43779746012fcf53603cdb91fdd8399aa0de868c2d56b6dde1" +checksum = "d340244b32d920260ae7448cb72b6e238bddc3d4f7603394e7dd46ed8e48f5b8" dependencies = [ "async-trait", "byteorder", @@ -3491,9 +3443,11 @@ dependencies = [ "pin-project-lite", "postgres-protocol", "postgres-types", - "socket2 0.5.2", + "rand", + "socket2 0.5.4", "tokio", "tokio-util", + "whoami", ] [[package]] @@ -3502,26 +3456,26 @@ version = "0.23.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c43ee83903113e03984cb9e5cebe6c04a5116269e900e3ddba8f068a62adda59" dependencies = [ - "rustls 0.20.8", + "rustls 0.20.9", "tokio", "webpki", ] [[package]] name = "tokio-rustls" -version = "0.24.0" +version = "0.24.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e0d409377ff5b1e3ca6437aa86c1eb7d40c134bfec254e44c830defa92669db5" +checksum = "c28327cf380ac148141087fbfb9de9d7bd4e84ab5d2c28fbc911d753de8a7081" dependencies = [ - "rustls 0.21.0", + "rustls 0.21.7", "tokio", ] [[package]] name = "tokio-stream" -version = "0.1.12" +version = "0.1.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8fb52b74f05dbf495a8fba459fdc331812b96aa086d9eb78101fa0d4569c3313" +checksum = "397c988d37662c7dda6d2208364a706264bf3d6138b11d436cbac0ad38832842" dependencies = [ "futures-core", "pin-project-lite", @@ -3530,9 +3484,9 @@ dependencies = [ [[package]] name = "tokio-util" -version = "0.7.7" +version = "0.7.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5427d89453009325de0d8f342c9490009f76e999cb7672d77e46267448f7e6b2" +checksum = "806fe8c2c87eccc8b3267cbae29ed3ab2d0bd37fca70ab622e46aaa9375ddb7d" dependencies = [ "bytes", "futures-core", @@ -3553,9 +3507,9 @@ dependencies = [ [[package]] name = "toml" -version = "0.7.4" +version = "0.7.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d6135d499e69981f9ff0ef2167955a5333c35e36f6937d382974566b3d5b94ec" +checksum = "dd79e69d3b627db300ff956027cc6c3798cef26d22526befdfcd12feeb6d2257" dependencies = [ "serde", "serde_spanned", @@ -3565,58 +3519,26 @@ dependencies = [ [[package]] name = "toml_datetime" -version = "0.6.2" +version = "0.6.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5a76a9312f5ba4c2dec6b9161fdf25d87ad8a09256ccea5a556fef03c706a10f" +checksum = "7cda73e2f1397b1262d6dfdcef8aafae14d1de7748d66822d3bfeeb6d03e5e4b" dependencies = [ "serde", ] [[package]] name = "toml_edit" -version = "0.19.10" +version = "0.19.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2380d56e8670370eee6566b0bfd4265f65b3f432e8c6d85623f728d4fa31f739" +checksum = "1b5bb770da30e5cbfde35a2d7b9b8a2c4b8ef89548a7a6aeab5c9a576e3e7421" dependencies = [ - "indexmap", + "indexmap 2.0.0", "serde", "serde_spanned", "toml_datetime", "winnow", ] -[[package]] -name = "tonic" -version = "0.8.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8f219fad3b929bef19b1f86fbc0358d35daed8f2cac972037ac0dc10bbb8d5fb" -dependencies = [ - "async-stream", - "async-trait", - "axum", - "base64 0.13.1", - "bytes", - "futures-core", - "futures-util", - "h2", - "http", - "http-body", - "hyper", - "hyper-timeout", - "percent-encoding", - "pin-project", - "prost", - "prost-derive", - "tokio", - "tokio-stream", - "tokio-util", - "tower", - "tower-layer", - "tower-service", - "tracing", - "tracing-futures", -] - [[package]] name = "tonic" version = "0.9.2" @@ -3625,7 +3547,7 @@ checksum = "3082666a3a6433f7f511c7192923fa1fe07c69332d3c6a2e6bb040b569199d5a" dependencies = [ "async-trait", "axum", - "base64 0.21.0", + "base64 0.21.4", "bytes", "futures-core", "futures-util", @@ -3655,7 +3577,7 @@ dependencies = [ "prost", "tokio", "tokio-stream", - "tonic 0.9.2", + "tonic", ] [[package]] @@ -3668,7 +3590,7 @@ dependencies = [ "prost-types", "tokio", "tokio-stream", - "tonic 0.9.2", + "tonic", ] [[package]] @@ -3679,7 +3601,7 @@ checksum = "b8fa9be0de6cf49e536ce1851f987bd21a43b771b09473c3549a6c853db37c1c" dependencies = [ "futures-core", "futures-util", - "indexmap", + "indexmap 1.9.3", "pin-project", "pin-project-lite", "rand", @@ -3722,41 +3644,31 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "09d48f71a791638519505cefafe162606f706c25592e4bde4d97600c0195312e" dependencies = [ "crossbeam-channel", - "time 0.3.20", + "time", "tracing-subscriber", ] [[package]] name = "tracing-attributes" -version = "0.1.23" +version = "0.1.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4017f8f45139870ca7e672686113917c71c7a6e02d4924eda67186083c03081a" +checksum = "5f4f31f56159e98206da9efd823404b79b6ef3143b4a7ab76e67b1751b25a4ab" dependencies = [ "proc-macro2", "quote", - "syn 1.0.109", + "syn 2.0.33", ] [[package]] name = "tracing-core" -version = "0.1.30" +version = "0.1.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "24eb03ba0eab1fd845050058ce5e616558e8f8d8fca633e6b163fe25c797213a" +checksum = "0955b8137a1df6f1a2e9a37d8a6656291ff0297c1a97c24e0d8425fe2312f79a" dependencies = [ "once_cell", "valuable", ] -[[package]] -name = "tracing-futures" -version = "0.2.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "97d095ae15e245a057c8e8451bab9b3ee1e1f68e9ba2b4fbc18d0ac5237835f2" -dependencies = [ - "pin-project", - "tracing", -] - [[package]] name = "tracing-log" version = "0.1.3" @@ -3806,9 +3718,9 @@ checksum = "92888ba5573ff080736b3648696b70cafad7d250551175acbaa4e0385b3e1460" [[package]] name = "unicode-ident" -version = "1.0.8" +version = "1.0.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e5464a87b239f13a63a501f2701565754bae92d243d4bb7eb12f6d57d2269bf4" +checksum = "3354b9ac3fae1ff6755cb6db53683adb661634f67557942dea4facebec0fee4b" [[package]] name = "unicode-normalization" @@ -3827,9 +3739,9 @@ checksum = "c0edd1e5b14653f783770bce4a4dabb4a5108a5370a5f5d8cfe8710c361f6c8b" [[package]] name = "unsafe-libyaml" -version = "0.2.8" +version = "0.2.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1865806a559042e51ab5414598446a5871b561d21b6764f2eabb0dd481d880a6" +checksum = "f28467d3e1d3c6586d8f25fa243f544f5800fec42d97032474e17222c2b75cfa" [[package]] name = "untrusted" @@ -3839,28 +3751,28 @@ checksum = "a156c684c91ea7d62626509bce3cb4e1d9ed5c4d978f7b4352658f96a4c26b4a" [[package]] name = "ureq" -version = "2.6.2" +version = "2.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "338b31dd1314f68f3aabf3ed57ab922df95ffcd902476ca7ba3c4ce7b908c46d" +checksum = "0b11c96ac7ee530603dcdf68ed1557050f374ce55a5a07193ebf8cbc9f8927e9" dependencies = [ - "base64 0.13.1", + "base64 0.21.4", "encoding_rs", "flate2", "log", "once_cell", - "rustls 0.20.8", + "rustls 0.21.7", + "rustls-webpki 0.100.3", "serde", "serde_json", "url", - "webpki", - "webpki-roots", + "webpki-roots 0.23.1", ] [[package]] name = "url" -version = "2.3.1" +version = "2.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0d68c799ae75762b8c3fe375feb6600ef5602c883c5d21eb51c09f22b83c4643" +checksum = "143b538f18257fac9cad154828a57c6bf5157e1aa604d4816b5995bf6de87ae5" dependencies = [ "form_urlencoded", "idna", @@ -3869,9 +3781,9 @@ dependencies = [ [[package]] name = "urlencoding" -version = "2.1.2" +version = "2.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e8db7427f936968176eaa7cdf81b7f98b980b18495ec28f1b5791ac3bfe3eea9" +checksum = "daf8dba3b7eb870caf1ddeed7bc9d2a049f3cfdfae7cb521b087cc33ae4c49da" [[package]] name = "utf8parse" @@ -3891,9 +3803,9 @@ dependencies = [ [[package]] name = "uuid" -version = "1.4.0" +version = "1.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d023da39d1fde5a8a3fe1f3e01ca9632ada0a63e9797de55a879d6e2236277be" +checksum = "79daa5ed5740825c40b389c5e50312b9c86df53fccd33f281df655642b43869d" [[package]] name = "valuable" @@ -3905,7 +3817,7 @@ checksum = "830b7e5d4d90034032940e4ace0d9a9a057e7a45cd94e6c007832e39edb82f6d" name = "value" version = "0.1.0" dependencies = [ - "base64 0.21.0", + "base64 0.21.4", "bytes", "chrono", "hex", @@ -3933,9 +3845,9 @@ checksum = "49874b5167b65d7193b8aba1567f5c7d93d001cafc34600cee003eda787e483f" [[package]] name = "walkdir" -version = "2.3.3" +version = "2.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "36df944cda56c7d8d8b7496af378e6b16de9284591917d307c9b4d313c44e698" +checksum = "d71d857dc86794ca4c280d616f7da00d2dbfd8cd788846559a6813e6aa4b54ee" dependencies = [ "same-file", "winapi-util", @@ -3943,20 +3855,13 @@ dependencies = [ [[package]] name = "want" -version = "0.3.0" +version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1ce8a968cb1cd110d136ff8b819a556d6fb6d919363c61534f6860c7eb172ba0" +checksum = "bfa7760aed19e106de2c7c0b581b509f2f25d3dacaf737cb82ac61bc6d760b0e" dependencies = [ - "log", "try-lock", ] -[[package]] -name = "wasi" -version = "0.10.0+wasi-snapshot-preview1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1a143597ca7c7793eff794def352d41792a93c481eb1042423ff7ff72ba2c31f" - [[package]] name = "wasi" version = "0.11.0+wasi-snapshot-preview1" @@ -3965,9 +3870,9 @@ checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" [[package]] name = "wasm-bindgen" -version = "0.2.84" +version = "0.2.87" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "31f8dcbc21f30d9b8f2ea926ecb58f6b91192c17e9d33594b3df58b2007ca53b" +checksum = "7706a72ab36d8cb1f80ffbf0e071533974a60d0a308d01a5d0375bf60499a342" dependencies = [ "cfg-if", "wasm-bindgen-macro", @@ -3975,24 +3880,24 @@ dependencies = [ [[package]] name = "wasm-bindgen-backend" -version = "0.2.84" +version = "0.2.87" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "95ce90fd5bcc06af55a641a86428ee4229e44e07033963a2290a8e241607ccb9" +checksum = "5ef2b6d3c510e9625e5fe6f509ab07d66a760f0885d858736483c32ed7809abd" dependencies = [ "bumpalo", "log", "once_cell", "proc-macro2", "quote", - "syn 1.0.109", + "syn 2.0.33", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-futures" -version = "0.4.34" +version = "0.4.37" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f219e0d211ba40266969f6dbdd90636da12f75bee4fc9d6c23d1260dadb51454" +checksum = "c02dbc21516f9f1f04f187958890d7e6026df8d16540b7ad9492bc34a67cea03" dependencies = [ "cfg-if", "js-sys", @@ -4002,9 +3907,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro" -version = "0.2.84" +version = "0.2.87" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4c21f77c0bedc37fd5dc21f897894a5ca01e7bb159884559461862ae90c0b4c5" +checksum = "dee495e55982a3bd48105a7b947fd2a9b4a8ae3010041b9e0faab3f9cd028f1d" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -4012,28 +3917,28 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.84" +version = "0.2.87" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2aff81306fcac3c7515ad4e177f521b5c9a15f2b08f4e32d823066102f35a5f6" +checksum = "54681b18a46765f095758388f2d0cf16eb8d4169b639ab575a8f5693af210c7b" dependencies = [ "proc-macro2", "quote", - "syn 1.0.109", + "syn 2.0.33", "wasm-bindgen-backend", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-shared" -version = "0.2.84" +version = "0.2.87" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0046fef7e28c3804e5e38bfa31ea2a0f73905319b677e57ebe37e49358989b5d" +checksum = "ca6ad05a4870b2bf5fe995117d3728437bd27d7cd5f06f13c17443ef369775a1" [[package]] name = "web-sys" -version = "0.3.61" +version = "0.3.64" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e33b99f4b23ba3eec1a53ac264e35a755f00e966e0065077d6027c0f575b0b97" +checksum = "9b85cbef8c220a6abc02aefd892dfc0fc23afb1c6a426316ec33253a3877249b" dependencies = [ "js-sys", "wasm-bindgen", @@ -4041,9 +3946,9 @@ dependencies = [ [[package]] name = "webpki" -version = "0.22.0" +version = "0.22.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f095d78192e208183081cc07bc5515ef55216397af48b873e5edcd72637fa1bd" +checksum = "f0e74f82d49d545ad128049b7e88f6576df2da6b02e9ce565c6f533be576957e" dependencies = [ "ring", "untrusted", @@ -4051,22 +3956,39 @@ dependencies = [ [[package]] name = "webpki-roots" -version = "0.22.6" +version = "0.23.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b6c71e40d7d2c34a5106301fb632274ca37242cd0c9d3e64dbece371a40a2d87" +checksum = "b03058f88386e5ff5310d9111d53f48b17d732b401aeb83a8d5190f2ac459338" dependencies = [ - "webpki", + "rustls-webpki 0.100.3", ] +[[package]] +name = "webpki-roots" +version = "0.25.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "14247bb57be4f377dfb94c72830b8ce8fc6beac03cf4bf7b9732eadd414123fc" + [[package]] name = "which" -version = "4.4.0" +version = "4.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2441c784c52b289a054b7201fc93253e288f094e2f4be9058343127c4226a269" +checksum = "87ba24419a2078cd2b0f2ede2691b6c66d8e47836da3b6db8265ebad47afbfc7" dependencies = [ "either", - "libc", + "home", "once_cell", + "rustix", +] + +[[package]] +name = "whoami" +version = "1.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "22fc3756b8a9133049b26c7f61ab35416c130e8c09b660f5b3958b446f52cc50" +dependencies = [ + "wasm-bindgen", + "web-sys", ] [[package]] @@ -4106,7 +4028,7 @@ version = "0.48.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e686886bc078bc1b0b600cac0147aadb815089b6e4da64016cbd754b6342700f" dependencies = [ - "windows-targets 0.48.0", + "windows-targets", ] [[package]] @@ -4124,52 +4046,28 @@ dependencies = [ "windows_x86_64_msvc 0.42.2", ] -[[package]] -name = "windows-sys" -version = "0.45.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "75283be5efb2831d37ea142365f009c02ec203cd29a3ebecbc093d52315b66d0" -dependencies = [ - "windows-targets 0.42.2", -] - [[package]] name = "windows-sys" version = "0.48.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "677d2418bec65e3338edb076e806bc1ec15693c5d0104683f2efe857f61056a9" dependencies = [ - "windows-targets 0.48.0", + "windows-targets", ] [[package]] name = "windows-targets" -version = "0.42.2" +version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8e5180c00cd44c9b1c88adb3693291f1cd93605ded80c250a75d472756b4d071" +checksum = "9a2fa6e2155d7247be68c096456083145c183cbbbc2764150dda45a87197940c" dependencies = [ - "windows_aarch64_gnullvm 0.42.2", - "windows_aarch64_msvc 0.42.2", - "windows_i686_gnu 0.42.2", - "windows_i686_msvc 0.42.2", - "windows_x86_64_gnu 0.42.2", - "windows_x86_64_gnullvm 0.42.2", - "windows_x86_64_msvc 0.42.2", -] - -[[package]] -name = "windows-targets" -version = "0.48.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b1eb6f0cd7c80c79759c929114ef071b87354ce476d9d94271031c0497adfd5" -dependencies = [ - "windows_aarch64_gnullvm 0.48.0", - "windows_aarch64_msvc 0.48.0", - "windows_i686_gnu 0.48.0", - "windows_i686_msvc 0.48.0", - "windows_x86_64_gnu 0.48.0", - "windows_x86_64_gnullvm 0.48.0", - "windows_x86_64_msvc 0.48.0", + "windows_aarch64_gnullvm 0.48.5", + "windows_aarch64_msvc 0.48.5", + "windows_i686_gnu 0.48.5", + "windows_i686_msvc 0.48.5", + "windows_x86_64_gnu 0.48.5", + "windows_x86_64_gnullvm 0.48.5", + "windows_x86_64_msvc 0.48.5", ] [[package]] @@ -4180,9 +4078,9 @@ checksum = "597a5118570b68bc08d8d59125332c54f1ba9d9adeedeef5b99b02ba2b0698f8" [[package]] name = "windows_aarch64_gnullvm" -version = "0.48.0" +version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "91ae572e1b79dba883e0d315474df7305d12f569b400fcf90581b06062f7e1bc" +checksum = "2b38e32f0abccf9987a4e3079dfb67dcd799fb61361e53e2882c3cbaf0d905d8" [[package]] name = "windows_aarch64_msvc" @@ -4192,9 +4090,9 @@ checksum = "e08e8864a60f06ef0d0ff4ba04124db8b0fb3be5776a5cd47641e942e58c4d43" [[package]] name = "windows_aarch64_msvc" -version = "0.48.0" +version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b2ef27e0d7bdfcfc7b868b317c1d32c641a6fe4629c171b8928c7b08d98d7cf3" +checksum = "dc35310971f3b2dbbf3f0690a219f40e2d9afcf64f9ab7cc1be722937c26b4bc" [[package]] name = "windows_i686_gnu" @@ -4204,9 +4102,9 @@ checksum = "c61d927d8da41da96a81f029489353e68739737d3beca43145c8afec9a31a84f" [[package]] name = "windows_i686_gnu" -version = "0.48.0" +version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "622a1962a7db830d6fd0a69683c80a18fda201879f0f447f065a3b7467daa241" +checksum = "a75915e7def60c94dcef72200b9a8e58e5091744960da64ec734a6c6e9b3743e" [[package]] name = "windows_i686_msvc" @@ -4216,9 +4114,9 @@ checksum = "44d840b6ec649f480a41c8d80f9c65108b92d89345dd94027bfe06ac444d1060" [[package]] name = "windows_i686_msvc" -version = "0.48.0" +version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4542c6e364ce21bf45d69fdd2a8e455fa38d316158cfd43b3ac1c5b1b19f8e00" +checksum = "8f55c233f70c4b27f66c523580f78f1004e8b5a8b659e05a4eb49d4166cca406" [[package]] name = "windows_x86_64_gnu" @@ -4228,9 +4126,9 @@ checksum = "8de912b8b8feb55c064867cf047dda097f92d51efad5b491dfb98f6bbb70cb36" [[package]] name = "windows_x86_64_gnu" -version = "0.48.0" +version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ca2b8a661f7628cbd23440e50b05d705db3686f894fc9580820623656af974b1" +checksum = "53d40abd2583d23e4718fddf1ebec84dbff8381c07cae67ff7768bbf19c6718e" [[package]] name = "windows_x86_64_gnullvm" @@ -4240,9 +4138,9 @@ checksum = "26d41b46a36d453748aedef1486d5c7a85db22e56aff34643984ea85514e94a3" [[package]] name = "windows_x86_64_gnullvm" -version = "0.48.0" +version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7896dbc1f41e08872e9d5e8f8baa8fdd2677f29468c4e156210174edc7f7b953" +checksum = "0b7b52767868a23d5bab768e390dc5f5c55825b6d30b86c844ff2dc7414044cc" [[package]] name = "windows_x86_64_msvc" @@ -4252,26 +4150,27 @@ checksum = "9aec5da331524158c6d1a4ac0ab1541149c0b9505fde06423b02f5ef0106b9f0" [[package]] name = "windows_x86_64_msvc" -version = "0.48.0" +version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1a515f5799fe4961cb532f983ce2b23082366b898e52ffbce459c86f67c8378a" +checksum = "ed94fce61571a4006852b7389a063ab983c02eb1bb37b47f8272ce92d06d9538" [[package]] name = "winnow" -version = "0.4.6" +version = "0.5.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "61de7bac303dc551fe038e2b3cef0f571087a47571ea6e79a87692ac99b99699" +checksum = "7c2e3184b9c4e92ad5167ca73039d0c42476302ab603e2fec4487511f38ccefc" dependencies = [ "memchr", ] [[package]] name = "winreg" -version = "0.10.1" +version = "0.50.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "80d0f4e272c85def139476380b12f9ac60926689dd2e01d4923222f40580869d" +checksum = "524e57b2c537c0f9b1e69f1965311ec12182b4122e45035b1508cd24d2adadb1" dependencies = [ - "winapi", + "cfg-if", + "windows-sys 0.48.0", ] [[package]] @@ -4292,7 +4191,7 @@ dependencies = [ "bcder", "bytes", "chrono", - "der 0.7.7", + "der 0.7.8", "hex", "pem 2.0.1", "ring", @@ -4303,9 +4202,9 @@ dependencies = [ [[package]] name = "xattr" -version = "0.2.3" +version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6d1526bbe5aaeb5eb06885f4d987bcdfa5e23187055de9b83fe00156a821fabc" +checksum = "f4686009f71ff3e5c4dbcf1a282d0a44db3f021ba69350cd42086b3e5f1c6985" dependencies = [ "libc", ] @@ -4337,16 +4236,16 @@ dependencies = [ "futures", "http", "hyper", - "hyper-rustls", + "hyper-rustls 0.23.2", "itertools", "log", "percent-encoding", - "rustls 0.20.8", + "rustls 0.20.9", "rustls-pemfile 0.3.0", "seahash", "serde", "serde_json", - "time 0.3.20", + "time", "tokio", "tower-service", "url", diff --git a/nexus/Cargo.toml b/nexus/Cargo.toml index 7eeaad272c..a07c40ea46 100644 --- a/nexus/Cargo.toml +++ b/nexus/Cargo.toml @@ -15,3 +15,5 @@ members = [ "server", "value", ] + +resolver = "2"