diff --git a/.github/workflows/flow.yml b/.github/workflows/flow.yml index 2adf8c619d..3ffcf15137 100644 --- a/.github/workflows/flow.yml +++ b/.github/workflows/flow.yml @@ -65,3 +65,7 @@ jobs: AWS_REGION: ${{ secrets.AWS_REGION }} TEST_BQ_CREDS: ${{ github.workspace }}/bq_service_account.json TEST_SF_CREDS: ${{ github.workspace }}/snowflake_creds.json + AZURE_TENANT_ID: ${{ secrets.AZURE_TENANT_ID }} + AZURE_CLIENT_ID: ${{ secrets.AZURE_CLIENT_ID }} + AZURE_CLIENT_SECRET: ${{ secrets.AZURE_CLIENT_SECRET }} + AZURE_SUBSCRIPTION_ID: ${{ secrets.AZURE_SUBSCRIPTION_ID }} diff --git a/docker-compose.yml b/docker-compose.yml index 2230925bf1..5106d65dae 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -73,7 +73,7 @@ services: - 8112:8112 environment: TEMPORAL_HOST_PORT: temporalite:7233 - PEERDB_CATALOG_DB: postgres + PEERDB_CATALOG_DATABASE: postgres PEERDB_CATALOG_HOST: catalog PEERDB_CATALOG_PASSWORD: postgres PEERDB_CATALOG_PORT: 5432 @@ -121,7 +121,7 @@ services: PEERDB_CATALOG_PORT: 5432 PEERDB_CATALOG_USER: postgres PEERDB_CATALOG_PASSWORD: postgres - PEERDB_CATALOG_DB: postgres + PEERDB_CATALOG_DATABASE: postgres PEERDB_PASSWORD: peerdb PEERDB_FLOW_SERVER_ADDRESS: http://flow_api:8112 ports: diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index f78d93e996..7445531d86 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -79,8 +79,10 @@ func (a *FlowableActivity) CheckConnection( return nil, fmt.Errorf("failed to get connector: %w", err) } + needsSetup := conn.NeedsSetupMetadataTables() + return &CheckConnectionResult{ - NeedsSetupMetadataTables: conn.NeedsSetupMetadataTables(), + NeedsSetupMetadataTables: needsSetup, }, nil } @@ -213,13 +215,13 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, input *protos.StartFlo return nil, fmt.Errorf("failed to get destination connector: %w", err) } - log.Println("initializing table schema...") + log.Info("initializing table schema...") err = dest.InitializeTableSchema(input.FlowConnectionConfigs.TableNameSchemaMapping) if err != nil { return nil, fmt.Errorf("failed to initialize table schema: %w", err) } - log.Println("pulling records...") + log.Info("pulling records...") records, err := src.PullRecords(&model.PullRecordsRequest{ FlowJobName: input.FlowConnectionConfigs.FlowJobName, @@ -235,14 +237,20 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, input *protos.StartFlo } // log the number of records - log.Printf("pulled %d records", len(records.Records)) + numRecords := len(records.Records) + log.Printf("pulled %d records", numRecords) + + if numRecords == 0 { + log.Info("no records to push") + return nil, nil + } res, err := dest.SyncRecords(&model.SyncRecordsRequest{ Records: records, FlowJobName: input.FlowConnectionConfigs.FlowJobName, }) - log.Println("pushed records") + log.Info("pushed records") if err != nil { return nil, fmt.Errorf("failed to push records: %w", err) @@ -266,7 +274,7 @@ func (a *FlowableActivity) StartNormalize(ctx context.Context, input *protos.Sta return nil, fmt.Errorf("failed to get destination connector: %w", err) } - log.Println("initializing table schema...") + log.Info("initializing table schema...") err = dest.InitializeTableSchema(input.FlowConnectionConfigs.TableNameSchemaMapping) if err != nil { return nil, fmt.Errorf("failed to initialize table schema: %w", err) @@ -280,7 +288,9 @@ func (a *FlowableActivity) StartNormalize(ctx context.Context, input *protos.Sta } // log the number of batches normalized - log.Printf("normalized records from batch %d to batch %d\n", res.StartBatchID, res.EndBatchID) + if res != nil { + log.Printf("normalized records from batch %d to batch %d\n", res.StartBatchID, res.EndBatchID) + } return res, nil } diff --git a/flow/cmd/api.go b/flow/cmd/api.go index dab6c39cac..560d9ecbb8 100644 --- a/flow/cmd/api.go +++ b/flow/cmd/api.go @@ -90,6 +90,43 @@ func (a *APIServer) StartPeerFlow(reqCtx context.Context, input *peerflow.PeerFl return workflowID, nil } +// StartPeerFlowWithConfig starts a peer flow workflow with the given config +func (a *APIServer) StartPeerFlowWithConfig( + reqCtx context.Context, + input *protos.FlowConnectionConfigs) (string, error) { + workflowID := fmt.Sprintf("%s-peerflow-%s", input.FlowJobName, uuid.New()) + workflowOptions := client.StartWorkflowOptions{ + ID: workflowID, + TaskQueue: shared.PeerFlowTaskQueue, + } + + maxBatchSize := int(input.MaxBatchSize) + if maxBatchSize == 0 { + maxBatchSize = 100000 + } + + limits := &peerflow.PeerFlowLimits{ + TotalSyncFlows: 0, + TotalNormalizeFlows: 0, + MaxBatchSize: maxBatchSize, + } + + state := peerflow.NewStartedPeerFlowState() + _, err := a.temporalClient.ExecuteWorkflow( + reqCtx, // context + workflowOptions, // workflow start options + peerflow.PeerFlowWorkflowWithConfig, // workflow function + input, // workflow input + limits, // workflow limits + state, // workflow state + ) + if err != nil { + return "", fmt.Errorf("unable to start PeerFlow workflow: %w", err) + } + + return workflowID, nil +} + func genConfigForQRepFlow(config *protos.QRepConfig, flowOptions map[string]interface{}, queryString string, destinationTableIdentifier string) error { config.InitialCopyOnly = false @@ -335,6 +372,30 @@ func APIMain(args *APIServerParams) error { } }) + r.POST("/flows/start_with_config", func(c *gin.Context) { + var reqJSON protos.FlowConnectionConfigs + data, _ := c.GetRawData() + + if err := protojson.Unmarshal(data, &reqJSON); err != nil { + c.JSON(http.StatusBadRequest, gin.H{ + "error": err.Error(), + }) + return + } + + ctx := c.Request.Context() + if id, err := apiServer.StartPeerFlowWithConfig(ctx, &reqJSON); err != nil { + c.JSON(http.StatusInternalServerError, gin.H{ + "error": err.Error(), + }) + } else { + c.JSON(http.StatusOK, gin.H{ + "status": "ok", + "workflow_id": id, + }) + } + }) + r.POST("/qrep/start", func(c *gin.Context) { var reqJSON protos.QRepConfig data, _ := c.GetRawData() diff --git a/flow/cmd/main.go b/flow/cmd/main.go index e5d61e603a..dd6c32a2c9 100644 --- a/flow/cmd/main.go +++ b/flow/cmd/main.go @@ -95,7 +95,7 @@ func main() { &cli.StringFlag{ Name: "catalog-db", Value: "postgres", - EnvVars: []string{"PEERDB_CATALOG_DB"}, + EnvVars: []string{"PEERDB_CATALOG_DATABASE"}, }, temporalHostPortFlag, }, diff --git a/flow/cmd/worker.go b/flow/cmd/worker.go index a9fd18842d..c55bbe7281 100644 --- a/flow/cmd/worker.go +++ b/flow/cmd/worker.go @@ -51,6 +51,7 @@ func WorkerMain(opts *WorkerOptions) error { w := worker.New(c, shared.PeerFlowTaskQueue, worker.Options{}) w.RegisterWorkflow(peerflow.PeerFlowWorkflow) + w.RegisterWorkflow(peerflow.PeerFlowWorkflowWithConfig) w.RegisterWorkflow(peerflow.SyncFlowWorkflow) w.RegisterWorkflow(peerflow.SetupFlowWorkflow) w.RegisterWorkflow(peerflow.NormalizeFlowWorkflow) diff --git a/flow/connectors/bigquery/bigquery.go b/flow/connectors/bigquery/bigquery.go index d4c9f9df1c..57da83dffd 100644 --- a/flow/connectors/bigquery/bigquery.go +++ b/flow/connectors/bigquery/bigquery.go @@ -978,7 +978,7 @@ func (m *MergeStmtGenerator) generateFlattenedCTE() string { castStmt = fmt.Sprintf("CAST(JSON_EXTRACT(_peerdb_data, '$.%s') AS %s) AS %s", colName, bqType, colName) // expecting data in BASE64 format - case qvalue.QValueKindBytes: + case qvalue.QValueKindBytes, qvalue.QValueKindBit: castStmt = fmt.Sprintf("FROM_BASE64(JSON_EXTRACT_SCALAR(_peerdb_data, '$.%s')) AS %s", colName, colName) // MAKE_INTERVAL(years INT64, months INT64, days INT64, hours INT64, minutes INT64, seconds INT64) @@ -991,11 +991,6 @@ func (m *MergeStmtGenerator) generateFlattenedCTE() string { // "CAST(JSON_EXTRACT_SCALAR(_peerdb_data, '$.%s.Days') AS INT64),0,0,"+ // "CAST(CAST(JSON_EXTRACT_SCALAR(_peerdb_data, '$.%s.Microseconds') AS INT64)/1000000 AS INT64)) AS %s", // colName, colName, colName, colName) - case qvalue.QValueKindBit: - // sample raw data for BIT {"a":{"Bytes":"oA==","Len":3,"Valid":true},"id":1} - // need to check correctness TODO - castStmt = fmt.Sprintf("FROM_BASE64(JSON_EXTRACT_SCALAR(_peerdb_data, '$.%s.Bytes')) AS %s", - colName, colName) // TODO add proper granularity for time types, then restore this // case model.ColumnTypeTime: // castStmt = fmt.Sprintf("time(timestamp_micros(CAST(JSON_EXTRACT(_peerdb_data, '$.%s.Microseconds')"+ diff --git a/flow/connectors/bigquery/qrecord_value_saver.go b/flow/connectors/bigquery/qrecord_value_saver.go index 672f7ee0b3..b8f1eb15aa 100644 --- a/flow/connectors/bigquery/qrecord_value_saver.go +++ b/flow/connectors/bigquery/qrecord_value_saver.go @@ -48,6 +48,11 @@ func (q QRecordValueSaver) Save() (map[string]bigquery.Value, string, error) { for i, v := range q.Record.Entries { k := q.ColumnNames[i] + if v.Value == nil { + bqValues[k] = nil + continue + } + switch v.Kind { case qvalue.QValueKindFloat32: val, ok := v.Value.(float32) @@ -113,7 +118,7 @@ func (q QRecordValueSaver) Save() (map[string]bigquery.Value, string, error) { bqValues[k] = RatToBigQueryNumeric(val) - case qvalue.QValueKindBytes: + case qvalue.QValueKindBytes, qvalue.QValueKindBit: val, ok := v.Value.([]byte) if !ok { return nil, "", fmt.Errorf("failed to convert %v to []byte", v.Value) diff --git a/flow/connectors/core.go b/flow/connectors/core.go index 7c74e33999..fba78d107f 100644 --- a/flow/connectors/core.go +++ b/flow/connectors/core.go @@ -5,6 +5,7 @@ import ( "fmt" connbigquery "github.com/PeerDB-io/peer-flow/connectors/bigquery" + conneventhub "github.com/PeerDB-io/peer-flow/connectors/eventhub" connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" connsnowflake "github.com/PeerDB-io/peer-flow/connectors/snowflake" "github.com/PeerDB-io/peer-flow/generated/protos" @@ -17,8 +18,6 @@ type Connector interface { NeedsSetupMetadataTables() bool SetupMetadataTables() error GetLastOffset(jobName string) (*protos.LastSyncState, error) - GetLastSyncBatchID(jobName string) (int64, error) - GetLastNormalizeBatchID(jobName string) (int64, error) // GetTableSchema returns the schema of a table. GetTableSchema(req *protos.GetTableSchemaInput) (*protos.TableSchema, error) @@ -86,6 +85,8 @@ func GetConnector(ctx context.Context, config *protos.Peer) (Connector, error) { return connbigquery.NewBigQueryConnector(ctx, config.GetBigqueryConfig()) case *protos.Peer_SnowflakeConfig: return connsnowflake.NewSnowflakeConnector(ctx, config.GetSnowflakeConfig()) + case *protos.Peer_EventhubConfig: + return conneventhub.NewEventHubConnector(ctx, config.GetEventhubConfig()) default: return nil, fmt.Errorf("requested connector is not yet implemented") } diff --git a/flow/connectors/eventhub/eventhub.go b/flow/connectors/eventhub/eventhub.go new file mode 100644 index 0000000000..cfcf4d508a --- /dev/null +++ b/flow/connectors/eventhub/eventhub.go @@ -0,0 +1,310 @@ +package conneventhub + +import ( + "context" + "errors" + "fmt" + "time" + + "github.com/Azure/azure-amqp-common-go/v4/aad" + "github.com/Azure/azure-amqp-common-go/v4/auth" + eventhub "github.com/Azure/azure-event-hubs-go/v3" + "github.com/Azure/azure-sdk-for-go/sdk/azidentity" + "github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/eventhub/armeventhub" + "github.com/PeerDB-io/peer-flow/connectors/utils" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/model" + log "github.com/sirupsen/logrus" + "go.temporal.io/sdk/activity" +) + +type EventHubConnector struct { + ctx context.Context + config *protos.EventHubConfig + pgMetadata *PostgresMetadataStore + tableSchemas map[string]*protos.TableSchema + creds *azidentity.DefaultAzureCredential + tokenProvider auth.TokenProvider + hubs map[string]*eventhub.Hub +} + +// NewEventHubConnector creates a new EventHubConnector. +func NewEventHubConnector( + ctx context.Context, + config *protos.EventHubConfig, +) (*EventHubConnector, error) { + defaultAzureCreds, err := azidentity.NewDefaultAzureCredential(nil) + if err != nil { + log.Errorf("failed to get default azure credentials: %v", err) + return nil, err + } + + jwtTokenProvider, err := aad.NewJWTProvider(aad.JWTProviderWithEnvironmentVars()) + if err != nil { + log.Errorf("failed to get jwt token provider: %v", err) + return nil, err + } + + pgMetadata, err := NewPostgresMetadataStore(ctx, config.GetMetadataDb()) + if err != nil { + log.Errorf("failed to create postgres metadata store: %v", err) + return nil, err + } + + return &EventHubConnector{ + ctx: ctx, + config: config, + pgMetadata: pgMetadata, + creds: defaultAzureCreds, + tokenProvider: jwtTokenProvider, + hubs: make(map[string]*eventhub.Hub), + }, nil +} + +func (c *EventHubConnector) Close() error { + var allErrors error + + // close all the event hub connections. + for _, hub := range c.hubs { + err := hub.Close(c.ctx) + if err != nil { + log.Errorf("failed to close event hub connection: %v", err) + allErrors = errors.Join(allErrors, err) + } + } + + // close the postgres metadata store. + err := c.pgMetadata.Close() + if err != nil { + log.Errorf("failed to close postgres metadata store: %v", err) + allErrors = errors.Join(allErrors, err) + } + + return allErrors +} + +func (c *EventHubConnector) ConnectionActive() bool { + return true +} + +func (c *EventHubConnector) EnsurePullability( + req *protos.EnsurePullabilityInput) (*protos.EnsurePullabilityOutput, error) { + panic("ensure pullability not implemented for event hub") +} + +func (c *EventHubConnector) SetupReplication(req *protos.SetupReplicationInput) error { + panic("setup replication not implemented for event hub") +} + +func (c *EventHubConnector) InitializeTableSchema(req map[string]*protos.TableSchema) error { + c.tableSchemas = req + return nil +} + +func (c *EventHubConnector) PullRecords(req *model.PullRecordsRequest) (*model.RecordBatch, error) { + panic("pull records not implemented for event hub") +} + +func (c *EventHubConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.SyncResponse, error) { + batch := req.Records + + eventsPerHeartBeat := 1000 + eventsPerBatch := 100000 + + batchPerTopic := make(map[string][]*eventhub.Event) + for i, record := range batch.Records { + json, err := record.GetItems().ToJSON() + if err != nil { + log.Errorf("failed to convert record to json: %v", err) + return nil, err + } + + // TODO (kaushik): this is a hack to get the table name. + topicName := record.GetTableName() + + if _, ok := batchPerTopic[topicName]; !ok { + batchPerTopic[topicName] = make([]*eventhub.Event, 0) + } + + batchPerTopic[topicName] = append(batchPerTopic[topicName], eventhub.NewEventFromString(json)) + + if i%eventsPerHeartBeat == 0 { + activity.RecordHeartbeat(c.ctx, fmt.Sprintf("sent %d records to hub: %s", i, topicName)) + } + + if i%eventsPerBatch == 0 { + err := c.sendEventBatch(batchPerTopic) + if err != nil { + return nil, err + } + + batchPerTopic = make(map[string][]*eventhub.Event) + } + } + + // send the remaining events. + if len(batchPerTopic) > 0 { + err := c.sendEventBatch(batchPerTopic) + if err != nil { + return nil, err + } + } + + log.Infof("successfully sent %d records to event hub", len(batch.Records)) + + err := c.UpdateLastOffset(req.FlowJobName, batch.LastCheckPointID) + if err != nil { + log.Errorf("failed to update last offset: %v", err) + return nil, err + } + + return &model.SyncResponse{ + FirstSyncedCheckPointID: batch.FirstCheckPointID, + LastSyncedCheckPointID: batch.LastCheckPointID, + NumRecordsSynced: int64(len(batch.Records)), + }, nil +} + +// send the batch to the event hub. +func (c *EventHubConnector) sendEventBatch(events map[string][]*eventhub.Event) error { + if len(events) == 0 { + log.Info("no events to send") + return nil + } + + // make a context with 10 second timeout as hub retries for as + // long as the context allows it to. + subCtx, cancel := context.WithTimeout(c.ctx, 5*time.Minute) + defer cancel() + + numEventsPushed := 0 + for tblName, eventBatch := range events { + hub, err := c.getOrCreateHubConnection(tblName) + if err != nil { + log.Errorf("failed to get event hub connection: %v", err) + return err + } + + err = hub.SendBatch(subCtx, eventhub.NewEventBatchIterator(eventBatch...)) + if err != nil { + log.Errorf("failed to send event batch: %v", err) + return err + } + + numEventsPushed += len(eventBatch) + } + + log.Infof("successfully sent a batch of %d events to event hub", numEventsPushed) + + return nil +} + +func (c *EventHubConnector) getOrCreateHubConnection(name string) (*eventhub.Hub, error) { + hub, ok := c.hubs[name] + if !ok { + hub, err := eventhub.NewHub(c.config.GetNamespace(), name, c.tokenProvider) + if err != nil { + log.Errorf("failed to create event hub connection: %v", err) + return nil, err + } + c.hubs[name] = hub + return hub, nil + } + + return hub, nil +} + +func (c *EventHubConnector) CreateRawTable(req *protos.CreateRawTableInput) (*protos.CreateRawTableOutput, error) { + // create topics for each table + // key is the source table and value is the destination topic name. + tableMap := req.GetTableNameMapping() + + for _, table := range tableMap { + err := c.ensureEventHub(c.ctx, table) + if err != nil { + log.Errorf("failed to get event hub properties: %v", err) + return nil, err + } + } + + return nil, nil +} + +func (c *EventHubConnector) GetTableSchema(req *protos.GetTableSchemaInput) (*protos.TableSchema, error) { + panic("get table schema not implemented for event hub") +} + +func (c *EventHubConnector) ensureEventHub(ctx context.Context, name string) error { + hubClient, err := c.getEventHubMgmtClient() + if err != nil { + return err + } + + namespace := c.config.GetNamespace() + resourceGroup := c.config.GetResourceGroup() + _, err = hubClient.Get(ctx, resourceGroup, namespace, name, nil) + + // TODO (kaushik): make these configurable. + partitionCount := int64(3) + retention := int64(1) + if err != nil { + opts := armeventhub.Eventhub{ + Properties: &armeventhub.Properties{ + PartitionCount: &partitionCount, + MessageRetentionInDays: &retention, + }, + } + + _, err := hubClient.CreateOrUpdate(ctx, resourceGroup, namespace, name, opts, nil) + if err != nil { + log.Errorf("failed to create event hub: %v", err) + return err + } + + log.Infof("event hub %s created", name) + } else { + log.Infof("event hub %s already exists", name) + } + + return nil +} + +func (c *EventHubConnector) getEventHubMgmtClient() (*armeventhub.EventHubsClient, error) { + subID, err := utils.GetAzureSubscriptionID() + if err != nil { + log.Errorf("failed to get azure subscription id: %v", err) + return nil, err + } + + hubClient, err := armeventhub.NewEventHubsClient(subID, c.creds, nil) + if err != nil { + log.Errorf("failed to get event hub client: %v", err) + return nil, err + } + + return hubClient, nil +} + +// Normalization + +func (c *EventHubConnector) SetupNormalizedTable( + req *protos.SetupNormalizedTableInput) (*protos.SetupNormalizedTableOutput, error) { + log.Infof("normalization for event hub is a no-op") + return nil, nil +} + +func (c *EventHubConnector) NormalizeRecords(req *model.NormalizeRecordsRequest) (*model.NormalizeResponse, error) { + log.Infof("normalization for event hub is a no-op") + return nil, nil +} + +// cleanup + +func (c *EventHubConnector) PullFlowCleanup(jobName string) error { + panic("pull flow cleanup not implemented for event hub") +} + +func (c *EventHubConnector) SyncFlowCleanup(jobName string) error { + // TODO (kaushik): this has to be implemented for DROP PEER support. + panic("sync flow cleanup not implemented for event hub") +} diff --git a/flow/connectors/eventhub/metadata.go b/flow/connectors/eventhub/metadata.go new file mode 100644 index 0000000000..514fe1894c --- /dev/null +++ b/flow/connectors/eventhub/metadata.go @@ -0,0 +1,171 @@ +package conneventhub + +import ( + "context" + + "github.com/PeerDB-io/peer-flow/connectors/utils" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/jackc/pgx/v5/pgxpool" + log "github.com/sirupsen/logrus" +) + +const ( + // schema for the peerdb metadata + metadataSchema = "peerdb_eventhub_metadata" + // The name of the table that stores the last sync state. + lastSyncStateTableName = "last_sync_state" +) + +type PostgresMetadataStore struct { + config *protos.PostgresConfig + pool *pgxpool.Pool +} + +func NewPostgresMetadataStore(ctx context.Context, pgConfig *protos.PostgresConfig) (*PostgresMetadataStore, error) { + connectionString := utils.GetPGConnectionString(pgConfig) + + pool, err := pgxpool.New(ctx, connectionString) + if err != nil { + log.Errorf("failed to create connection pool: %v", err) + return nil, err + } + log.Info("created connection pool for eventhub metadata store") + + return &PostgresMetadataStore{ + config: pgConfig, + pool: pool, + }, nil +} + +func (p *PostgresMetadataStore) Close() error { + if p.pool != nil { + p.pool.Close() + } + + return nil +} + +func (c *EventHubConnector) NeedsSetupMetadataTables() bool { + ms := c.pgMetadata + + // check if schema exists + rows := ms.pool.QueryRow(c.ctx, "SELECT count(*) FROM pg_catalog.pg_namespace WHERE nspname = $1", metadataSchema) + + var exists int64 + err := rows.Scan(&exists) + if err != nil { + log.Errorf("failed to check if schema exists: %v", err) + return false + } + + if exists > 0 { + return true + } + + return true +} + +func (c *EventHubConnector) SetupMetadataTables() error { + ms := c.pgMetadata + + // start a transaction + tx, err := ms.pool.Begin(c.ctx) + if err != nil { + log.Errorf("failed to start transaction: %v", err) + return err + } + + // create the schema + _, err = tx.Exec(c.ctx, "CREATE SCHEMA IF NOT EXISTS "+metadataSchema) + if err != nil { + log.Errorf("failed to create schema: %v", err) + return err + } + + // create the last sync state table + _, err = tx.Exec(c.ctx, ` + CREATE TABLE IF NOT EXISTS `+metadataSchema+`.`+lastSyncStateTableName+` ( + job_name TEXT PRIMARY KEY NOT NULL, + last_offset BIGINT NOT NULL, + updated_at TIMESTAMP NOT NULL DEFAULT NOW() + ) + `) + if err != nil { + log.Errorf("failed to create last sync state table: %v", err) + return err + } + + // commit the transaction + err = tx.Commit(c.ctx) + if err != nil { + log.Errorf("failed to commit transaction: %v", err) + return err + } + + return nil +} + +func (c *EventHubConnector) GetLastOffset(jobName string) (*protos.LastSyncState, error) { + ms := c.pgMetadata + + rows := ms.pool.QueryRow(c.ctx, ` + SELECT last_offset + FROM `+metadataSchema+`.`+lastSyncStateTableName+` + WHERE job_name = $1 + `, jobName) + + var offset int64 + err := rows.Scan(&offset) + if err != nil { + // if the job doesn't exist, return 0 + if err.Error() == "no rows in result set" { + return &protos.LastSyncState{ + Checkpoint: 0, + }, nil + } + + log.Errorf("failed to get last offset: %v", err) + return nil, err + } + + log.Infof("got last offset for job `%s`: %d", jobName, offset) + + return &protos.LastSyncState{ + Checkpoint: offset, + }, nil +} + +// update offset for a job +func (c *EventHubConnector) UpdateLastOffset(jobName string, offset int64) error { + ms := c.pgMetadata + + // start a transaction + tx, err := ms.pool.Begin(c.ctx) + if err != nil { + log.Errorf("failed to start transaction: %v", err) + return err + } + + // update the last offset + log.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) + ON CONFLICT (job_name) + DO UPDATE SET last_offset = $2, updated_at = NOW() + `, jobName, offset) + + if err != nil { + log.Errorf("failed to update last offset: %v", err) + return err + } + + // commit the transaction + err = tx.Commit(c.ctx) + if err != nil { + log.Errorf("failed to commit transaction: %v", err) + return err + } + + return nil +} diff --git a/flow/connectors/eventhub/qrep.go b/flow/connectors/eventhub/qrep.go new file mode 100644 index 0000000000..4aaa75984e --- /dev/null +++ b/flow/connectors/eventhub/qrep.go @@ -0,0 +1,33 @@ +package conneventhub + +import ( + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/model" +) + +func (c *EventHubConnector) SetupQRepMetadataTables(config *protos.QRepConfig) error { + panic("setup qrep metadata tables not implemented for eventhub") +} + +func (c *EventHubConnector) GetQRepPartitions( + config *protos.QRepConfig, last *protos.QRepPartition) ([]*protos.QRepPartition, error) { + panic("get qrep partitions not implemented for eventhub") +} + +func (c *EventHubConnector) PullQRepRecords( + config *protos.QRepConfig, partition *protos.QRepPartition) (*model.QRecordBatch, error) { + panic("pull qrep records not implemented for eventhub") +} + +func (c *EventHubConnector) SyncQRepRecords( + config *protos.QRepConfig, partition *protos.QRepPartition, records *model.QRecordBatch) (int, error) { + panic("sync qrep records not implemented for eventhub") +} + +func (c *EventHubConnector) ConsolidateQRepPartitions(config *protos.QRepConfig) error { + panic("consolidate qrep partitions not implemented for eventhub") +} + +func (c *EventHubConnector) CleanupQRepFlow(config *protos.QRepConfig) error { + panic("cleanup qrep flow not implemented for eventhub") +} diff --git a/flow/connectors/postgres/cdc.go b/flow/connectors/postgres/cdc.go index a71e229887..b1e1824f65 100644 --- a/flow/connectors/postgres/cdc.go +++ b/flow/connectors/postgres/cdc.go @@ -13,6 +13,7 @@ import ( "github.com/jackc/pgx/v5/pgproto3" "github.com/jackc/pgx/v5/pgtype" "github.com/jackc/pgx/v5/pgxpool" + "github.com/lib/pq/oid" log "github.com/sirupsen/logrus" ) @@ -234,6 +235,10 @@ func (p *PostgresCDCSource) consumeStream( result.LastCheckPointID = int64(xld.WALStart) clientXLogPos = xld.WALStart + pglogrepl.LSN(len(xld.WALData)) + + if result.Records != nil && len(result.Records) == int(req.MaxBatchSize) { + return result, nil + } } } } @@ -443,6 +448,12 @@ func (p *PostgresCDCSource) decodeColumnData(data []byte, dataType uint32, forma return nil, err } return retVal, nil + } else if dataType == uint32(oid.T_timetz) { // ugly TIMETZ workaround for CDC decoding. + retVal, err := parseFieldFromPostgresOID(dataType, string(data)) + if err != nil { + return nil, err + } + return retVal, nil } return &qvalue.QValue{Kind: qvalue.QValueKindString, Value: string(data)}, nil } diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index 909423a3ae..8e892cfd5e 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -5,6 +5,7 @@ import ( "fmt" "strings" + "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" @@ -33,8 +34,7 @@ func (t *SchemaTable) String() string { // NewPostgresConnector creates a new instance of PostgresConnector. func NewPostgresConnector(ctx context.Context, pgConfig *protos.PostgresConfig) (*PostgresConnector, error) { - connectionString := fmt.Sprintf("host=%s port=%d user=%s password=%s dbname=%s", - pgConfig.Host, pgConfig.Port, pgConfig.User, pgConfig.Password, pgConfig.Database) + connectionString := utils.GetPGConnectionString(pgConfig) // create a separate connection pool for non-replication queries as replication connections cannot // be used for extended query protocol, i.e. prepared statements @@ -83,13 +83,6 @@ func (c *PostgresConnector) GetLastOffset(jobName string) (*protos.LastSyncState panic("not implemented") } -func (c *PostgresConnector) GetLastSyncBatchID(jobName string) (int64, error) { - panic("not implemented") -} - -func (c *PostgresConnector) GetLastNormalizeBatchID(jobName string) (int64, error) { - panic("not implemented") -} func (c *PostgresConnector) GetDistinctTableNamesInBatch(flowJobName string, syncBatchID int64, normalizeBatchID int64) ([]string, error) { panic("not implemented") @@ -424,14 +417,27 @@ func (c *PostgresConnector) getPrimaryKeyColumn(schemaTable *SchemaTable) (strin // Get the primary key column name var pkCol string - err = c.pool.QueryRow(c.ctx, + rows, err := c.pool.Query(c.ctx, `SELECT a.attname FROM pg_index i JOIN pg_attribute a ON a.attrelid = i.indrelid AND a.attnum = ANY(i.indkey) WHERE i.indrelid = $1 AND i.indisprimary`, - relID).Scan(&pkCol) + relID) if err != nil { return "", fmt.Errorf("error getting primary key column for table %s: %w", schemaTable, err) } + defer rows.Close() + // 0 rows returned, table has no primary keys + if !rows.Next() { + return "", fmt.Errorf("table %s has no primary keys", schemaTable) + } + err = rows.Scan(&pkCol) + if err != nil { + return "", fmt.Errorf("error scanning primary key column for table %s: %w", schemaTable, err) + } + // more than 1 row returned, table has more than 1 primary key + if rows.Next() { + return "", fmt.Errorf("table %s has more than one primary key", schemaTable) + } return pkCol, nil } diff --git a/flow/connectors/postgres/postgres_cdc_test.go b/flow/connectors/postgres/postgres_cdc_test.go index af32f112fc..b5ad5b8d5d 100644 --- a/flow/connectors/postgres/postgres_cdc_test.go +++ b/flow/connectors/postgres/postgres_cdc_test.go @@ -10,6 +10,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" + "github.com/jackc/pgx/v5" "github.com/stretchr/testify/suite" ) @@ -38,10 +39,13 @@ func (suite *PostgresCDCTestSuite) insertSimpleRecords(srcTableName string) { func (suite *PostgresCDCTestSuite) validateInsertedSimpleRecords(records []model.Record, srcTableName string, dstTableName string) { suite.Equal(3, len(records)) - matchData := []map[string]interface{}{ - {"id": int32(2), "name": "quick"}, - {"id": int32(4), "name": "brown"}, - {"id": int32(8), "name": "fox"}, + 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"}}, } for idx, record := range records { suite.IsType(&model.InsertRecord{}, record) @@ -57,7 +61,9 @@ func (suite *PostgresCDCTestSuite) mutateSimpleRecords(srcTableName string) { suite.failTestError(err) defer func() { err := mutateRecordsTx.Rollback(context.Background()) - suite.failTestError(err) + if err != pgx.ErrTxClosed { + suite.failTestError(err) + } }() _, err = mutateRecordsTx.Exec(context.Background(), @@ -77,14 +83,16 @@ func (suite *PostgresCDCTestSuite) validateSimpleMutatedRecords(records []model. updateRecord := records[0].(*model.UpdateRecord) suite.Equal(srcTableName, updateRecord.SourceTableName) suite.Equal(dstTableName, updateRecord.DestinationTableName) - suite.Equal(map[string]interface{}{}, updateRecord.OldItems) - suite.Equal(map[string]interface{}{"id": int32(2), "name": "slow"}, updateRecord.NewItems) + 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) suite.IsType(&model.DeleteRecord{}, records[1]) deleteRecord := records[1].(*model.DeleteRecord) suite.Equal(srcTableName, deleteRecord.SourceTableName) suite.Equal(dstTableName, deleteRecord.DestinationTableName) - suite.Equal(map[string]interface{}{"id": int32(8), "name": nil}, deleteRecord.Items) + suite.Equal(model.RecordItems{"id": qvalue.QValue{Kind: qvalue.QValueKindInt32, Value: int32(8)}, + "name": qvalue.QValue{Kind: qvalue.QValueKindInvalid, Value: nil}}, deleteRecord.Items) } func (suite *PostgresCDCTestSuite) randBytea(n int) []byte { @@ -110,7 +118,9 @@ func (suite *PostgresCDCTestSuite) insertToastRecords(srcTableName string) { suite.failTestError(err) defer func() { err := insertRecordsTx.Rollback(context.Background()) - suite.failTestError(err) + if err != pgx.ErrTxClosed { + suite.failTestError(err) + } }() for i := 0; i < 4; i++ { @@ -147,7 +157,9 @@ func (suite *PostgresCDCTestSuite) mutateToastRecords(srcTableName string) { suite.failTestError(err) defer func() { err := mutateRecordsTx.Rollback(context.Background()) - suite.failTestError(err) + if err != pgx.ErrTxClosed { + suite.failTestError(err) + } }() _, err = mutateRecordsTx.Exec(context.Background(), @@ -177,6 +189,7 @@ func (suite *PostgresCDCTestSuite) validateMutatedToastRecords(records []model.R 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))) suite.Equal(3, len(updateRecord.UnchangedToastColumns)) suite.True(updateRecord.UnchangedToastColumns["lz4_t"]) @@ -188,7 +201,9 @@ func (suite *PostgresCDCTestSuite) validateMutatedToastRecords(records []model.R suite.Equal(srcTableName, updateRecord.SourceTableName) suite.Equal(dstTableName, updateRecord.DestinationTableName) suite.Equal(2, len(updateRecord.NewItems)) + suite.Equal(qvalue.QValueKindInt32, updateRecord.NewItems["id"].Kind) suite.Equal(int32(2), updateRecord.NewItems["id"].Value.(int32)) + suite.Equal(qvalue.QValueKindBytes, updateRecord.NewItems["lz4_b"].Kind) suite.Equal(65536, len(updateRecord.NewItems["lz4_b"].Value.([]byte))) suite.Equal(3, len(updateRecord.UnchangedToastColumns)) suite.True(updateRecord.UnchangedToastColumns["lz4_t"]) @@ -201,6 +216,7 @@ func (suite *PostgresCDCTestSuite) validateMutatedToastRecords(records []model.R suite.Equal(dstTableName, updateRecord.DestinationTableName) suite.Equal(2, len(updateRecord.NewItems)) suite.Equal(int32(3), updateRecord.NewItems["id"].Value.(int32)) + suite.Equal(qvalue.QValueKindBytes, updateRecord.NewItems["n_b"].Kind) suite.Equal(65536, len(updateRecord.NewItems["n_b"].Value.([]byte))) suite.Equal(3, len(updateRecord.UnchangedToastColumns)) suite.True(updateRecord.UnchangedToastColumns["lz4_t"]) @@ -213,10 +229,14 @@ func (suite *PostgresCDCTestSuite) validateMutatedToastRecords(records []model.R suite.Equal(dstTableName, deleteRecord.DestinationTableName) suite.Equal(5, len(deleteRecord.Items)) suite.Equal(int32(3), deleteRecord.Items["id"].Value.(int32)) - suite.Nil(deleteRecord.Items["n_t"]) - suite.Nil(deleteRecord.Items["lz4_t"]) - suite.Nil(deleteRecord.Items["n_b"]) - suite.Nil(deleteRecord.Items["lz4_b"]) + 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) } func (suite *PostgresCDCTestSuite) SetupSuite() { @@ -236,7 +256,9 @@ func (suite *PostgresCDCTestSuite) SetupSuite() { suite.failTestError(err) defer func() { err := setupTx.Rollback(context.Background()) - suite.failTestError(err) + if err != pgx.ErrTxClosed { + suite.failTestError(err) + } }() _, err = setupTx.Exec(context.Background(), "DROP SCHEMA IF EXISTS pgpeer_test CASCADE") suite.failTestError(err) @@ -251,7 +273,9 @@ func (suite *PostgresCDCTestSuite) TearDownSuite() { suite.failTestError(err) defer func() { err := teardownTx.Rollback(context.Background()) - suite.failTestError(err) + if err != pgx.ErrTxClosed { + suite.failTestError(err) + } }() _, err = teardownTx.Exec(context.Background(), "DROP SCHEMA IF EXISTS pgpeer_test CASCADE") suite.failTestError(err) @@ -292,14 +316,6 @@ func (suite *PostgresCDCTestSuite) TestNonImplementedFunctions() { //nolint:errcheck suite.connector.GetLastOffset("offset_panic") }, "not implemented") - suite.Panicsf(func() { - //nolint:errcheck - suite.connector.GetLastSyncBatchID("sync_batch_id_panic") - }, "not implemented") - suite.Panicsf(func() { - //nolint:errcheck - suite.connector.GetLastNormalizeBatchID("normalize_batch_id_panic") - }, "not implemented") suite.Panicsf(func() { //nolint:errcheck suite.connector.GetDistinctTableNamesInBatch("distinct_table_names_in_batch_panic", 0, 0) @@ -606,8 +622,8 @@ func (suite *PostgresCDCTestSuite) TestAllTypesHappyFlow() { Columns: map[string]string{ "id": string(qvalue.QValueKindInt64), "c1": string(qvalue.QValueKindInt64), - "c2": model.ColumnHexBit, - "c3": model.ColumnHexBit, + "c2": string(qvalue.QValueKindBit), + "c3": string(qvalue.QValueKindBit), "c4": string(qvalue.QValueKindBoolean), "c6": string(qvalue.QValueKindBytes), "c7": string(qvalue.QValueKindString), @@ -616,29 +632,29 @@ func (suite *PostgresCDCTestSuite) TestAllTypesHappyFlow() { "c11": string(qvalue.QValueKindDate), "c12": string(qvalue.QValueKindFloat64), "c13": string(qvalue.QValueKindFloat64), - "c14": model.ColumnTypeString, + "c14": string(qvalue.QValueKindString), "c15": string(qvalue.QValueKindInt32), - "c16": model.ColumnTypeInterval, + "c16": string(qvalue.QValueKindString), "c17": string(qvalue.QValueKindJSON), "c18": string(qvalue.QValueKindJSON), - "c21": model.ColumnTypeString, - "c22": model.ColumnTypeString, + "c21": string(qvalue.QValueKindString), + "c22": string(qvalue.QValueKindString), "c23": string(qvalue.QValueKindNumeric), - "c24": string(qvalue.QValueKindInt64), + "c24": string(qvalue.QValueKindString), "c28": string(qvalue.QValueKindFloat32), "c29": string(qvalue.QValueKindInt16), "c30": string(qvalue.QValueKindInt16), "c31": string(qvalue.QValueKindInt32), - "c32": model.ColumnTypeString, + "c32": string(qvalue.QValueKindString), "c33": string(qvalue.QValueKindTimestamp), "c34": string(qvalue.QValueKindTimestampTZ), "c35": string(qvalue.QValueKindTime), "c36": string(qvalue.QValueKindTimeTZ), - "c37": model.ColumnTypeString, - "c38": model.ColumnTypeString, - "c39": model.ColumnTypeString, - "c40": model.ColumnTypeString, - "c41": model.ColumnTypeString, + "c37": string(qvalue.QValueKindString), + "c38": string(qvalue.QValueKindString), + "c39": string(qvalue.QValueKindString), + "c40": string(qvalue.QValueKindUUID), + "c41": string(qvalue.QValueKindString), }, PrimaryKeyColumn: "id", }, tableNameSchema) diff --git a/flow/connectors/postgres/qrep.go b/flow/connectors/postgres/qrep.go index 10f1d4249b..85021edbad 100644 --- a/flow/connectors/postgres/qrep.go +++ b/flow/connectors/postgres/qrep.go @@ -26,14 +26,16 @@ func (c *PostgresConnector) GetQRepPartitions( var partitions []*protos.QRepPartition switch v := minValue.(type) { - case int32, int64: + case int64: maxValue := maxValue.(int64) + 1 - fmt.Println("minValue", minValue) - fmt.Println("maxValue", maxValue) - partitions, err = c.getIntPartitions(v.(int64), maxValue, config.BatchSizeInt) + partitions, err = c.getIntPartitions(v, maxValue, config.BatchSizeInt) case time.Time: maxValue := maxValue.(time.Time).Add(time.Microsecond) partitions, err = c.getTimePartitions(v, maxValue, config.BatchDurationSeconds) + // only hit when there is no data in the source table + case nil: + log.Warnf("no records to replicate for flow job %s, returning", config.FlowJobName) + return make([]*protos.QRepPartition, 0), nil default: return nil, fmt.Errorf("unsupported type: %T", v) } @@ -51,12 +53,24 @@ func (c *PostgresConnector) getMinMaxValues( ) (interface{}, interface{}, error) { var minValue, maxValue interface{} quotedWatermarkColumn := fmt.Sprintf("\"%s\"", config.WatermarkColumn) + // Get the maximum value from the database + maxQuery := fmt.Sprintf("SELECT MAX(%[1]s) FROM %[2]s", quotedWatermarkColumn, config.WatermarkTable) + row := c.pool.QueryRow(c.ctx, maxQuery) + if err := row.Scan(&maxValue); err != nil { + return nil, nil, fmt.Errorf("failed to query for max value: %w", err) + } if last != nil && last.Range != nil { // If there's a last partition, start from its end switch lastRange := last.Range.Range.(type) { case *protos.PartitionRange_IntRange: minValue = lastRange.IntRange.End + switch v := maxValue.(type) { + case int16: + maxValue = int64(v) + case int32: + maxValue = int64(v) + } case *protos.PartitionRange_TimestampRange: minValue = lastRange.TimestampRange.End.AsTime() } @@ -68,13 +82,15 @@ func (c *PostgresConnector) getMinMaxValues( log.Errorf("failed to query [%s] for min value: %v", minQuery, err) return nil, nil, fmt.Errorf("failed to query for min value: %w", err) } - } - // Get the maximum value from the database - maxQuery := fmt.Sprintf("SELECT MAX(%[1]s) FROM %[2]s", quotedWatermarkColumn, config.WatermarkTable) - row := c.pool.QueryRow(c.ctx, maxQuery) - if err := row.Scan(&maxValue); err != nil { - return nil, nil, fmt.Errorf("failed to query for max value: %w", err) + switch v := minValue.(type) { + case int16: + minValue = int64(v) + maxValue = int64(maxValue.(int16)) + case int32: + minValue = int64(v) + maxValue = int64(maxValue.(int32)) + } } return minValue, maxValue, nil diff --git a/flow/connectors/postgres/qrep_query_executor_test.go b/flow/connectors/postgres/qrep_query_executor_test.go index 1cf641ed82..3b9f0eabb9 100644 --- a/flow/connectors/postgres/qrep_query_executor_test.go +++ b/flow/connectors/postgres/qrep_query_executor_test.go @@ -239,7 +239,7 @@ func TestAllDataTypes(t *testing.T) { t.Fatalf("expected %v, got %v", expectedBytea, record.Entries[6].Value) } - expectedJSON := `{"key": "value"}` + expectedJSON := `{"key":"value"}` if record.Entries[7].Value.(string) != expectedJSON { t.Fatalf("expected %v, got %v", expectedJSON, record.Entries[7].Value) } diff --git a/flow/connectors/postgres/qvalue_convert.go b/flow/connectors/postgres/qvalue_convert.go index d9b821f2a2..64861fb780 100644 --- a/flow/connectors/postgres/qvalue_convert.go +++ b/flow/connectors/postgres/qvalue_convert.go @@ -11,11 +11,12 @@ import ( "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/jackc/pgx/v5/pgtype" + "github.com/lib/pq/oid" log "github.com/sirupsen/logrus" ) -func getQValueKindForPostgresOID(oid uint32) qvalue.QValueKind { - switch oid { +func getQValueKindForPostgresOID(recvOID uint32) qvalue.QValueKind { + switch recvOID { case pgtype.BoolOID: return qvalue.QValueKindBoolean case pgtype.Int2OID: @@ -46,18 +47,30 @@ func getQValueKindForPostgresOID(oid uint32) qvalue.QValueKind { return qvalue.QValueKindTimestampTZ case pgtype.NumericOID: return qvalue.QValueKindNumeric + case pgtype.BitOID, pgtype.VarbitOID: + return qvalue.QValueKindBit default: - typeName, ok := pgtype.NewMap().TypeForOID(oid) + typeName, ok := pgtype.NewMap().TypeForOID(recvOID) if !ok { - // workaround for TIMETZ not being defined by this pgtype - if oid == 1266 { + // workaround for some types not being defined by pgtype + if recvOID == uint32(oid.T_timetz) { return qvalue.QValueKindTimeTZ + } else if recvOID == uint32(oid.T_xml) { // XML + return qvalue.QValueKindString + } else if recvOID == uint32(oid.T_money) { // MONEY + return qvalue.QValueKindString + } else if recvOID == uint32(oid.T_txid_snapshot) { // TXID_SNAPSHOT + return qvalue.QValueKindString + } else if recvOID == uint32(oid.T_tsvector) { // TSVECTOR + return qvalue.QValueKindString + } else if recvOID == uint32(oid.T_tsquery) { // TSQUERY + return qvalue.QValueKindString } - log.Warnf("failed to get type name for oid: %v", oid) + log.Warnf("failed to get type name for oid: %v", recvOID) return qvalue.QValueKindInvalid } else { - log.Warnf("unsupported field type: %v - type name - %s", oid, typeName.Name) - return qvalue.QValueKindInvalid + log.Warnf("unsupported field type: %v - type name - %s; returning as string", recvOID, typeName.Name) + return qvalue.QValueKindString } } } @@ -65,6 +78,11 @@ func getQValueKindForPostgresOID(oid uint32) qvalue.QValueKind { func parseFieldFromQValueKind(qvalueKind qvalue.QValueKind, value interface{}) (*qvalue.QValue, error) { var val *qvalue.QValue = nil + if value == nil { + val = &qvalue.QValue{Kind: qvalueKind, Value: nil} + return val, nil + } + switch qvalueKind { case qvalue.QValueKindTimestamp: timestamp := value.(time.Time) @@ -96,12 +114,14 @@ func parseFieldFromQValueKind(qvalueKind qvalue.QValueKind, value interface{}) ( timeVal := value.(string) // edge case, Postgres supports this extreme value for time timeVal = strings.Replace(timeVal, "24:00:00.000000", "23:59:59.999999", 1) + // edge case, Postgres prints +0000 as +00 + timeVal = strings.Replace(timeVal, "+00", "+0000", 1) t, err := time.Parse("15:04:05.999999-0700", timeVal) if err != nil { return nil, fmt.Errorf("failed to parse time: %w", err) } t = t.AddDate(1970, 0, 0) - val = &qvalue.QValue{Kind: qvalue.QValueKindTime, Value: t} + val = &qvalue.QValue{Kind: qvalue.QValueKindTimeTZ, Value: t} case qvalue.QValueKindBoolean: boolVal := value.(bool) @@ -130,8 +150,9 @@ func parseFieldFromQValueKind(qvalueKind qvalue.QValueKind, value interface{}) ( floatVal := value.(float64) val = &qvalue.QValue{Kind: qvalue.QValueKindFloat64, Value: floatVal} case qvalue.QValueKindString: - textVal := value.(string) - val = &qvalue.QValue{Kind: qvalue.QValueKindString, Value: textVal} + // handling all unsupported types with strings as well for now. + textVal := value + val = &qvalue.QValue{Kind: qvalue.QValueKindString, Value: fmt.Sprint(textVal)} case qvalue.QValueKindUUID: switch value.(type) { case string: @@ -144,7 +165,11 @@ func parseFieldFromQValueKind(qvalueKind qvalue.QValueKind, value interface{}) ( case qvalue.QValueKindBytes: rawBytes := value.([]byte) val = &qvalue.QValue{Kind: qvalue.QValueKindBytes, Value: rawBytes} - // TODO: check for handling of QValueKindBit + case qvalue.QValueKindBit: + bitsVal := value.(pgtype.Bits) + if bitsVal.Valid { + val = &qvalue.QValue{Kind: qvalue.QValueKindBit, Value: bitsVal.Bytes} + } case qvalue.QValueKindNumeric: numVal := value.(pgtype.Numeric) if numVal.Valid { diff --git a/flow/connectors/snowflake/avro_file_writer_test.go b/flow/connectors/snowflake/avro_file_writer_test.go index 41b76513c6..a15c378236 100644 --- a/flow/connectors/snowflake/avro_file_writer_test.go +++ b/flow/connectors/snowflake/avro_file_writer_test.go @@ -35,15 +35,13 @@ func createQValue(t *testing.T, kind qvalue.QValueKind, placeHolder int) qvalue. value = big.NewRat(int64(placeHolder), 1) case qvalue.QValueKindUUID: value = uuid.New() // assuming you have the github.com/google/uuid package - // case qvalue.QValueKindBit: - // value = placeHolder % 2 // case qvalue.QValueKindArray: // value = []int{1, 2, 3} // placeholder array, replace with actual logic // case qvalue.QValueKindStruct: // value = map[string]interface{}{"key": "value"} // placeholder struct, replace with actual logic // case qvalue.QValueKindJSON: // value = `{"key": "value"}` // placeholder JSON, replace with actual logic - case qvalue.QValueKindBytes: + case qvalue.QValueKindBytes, qvalue.QValueKindBit: value = []byte("sample bytes") // placeholder bytes, replace with actual logic default: require.Failf(t, "unsupported QValueKind", "unsupported QValueKind: %s", kind) @@ -75,7 +73,7 @@ func generateRecords(t *testing.T, nullable bool, numRows uint32) *model.QRecord qvalue.QValueKindBytes, qvalue.QValueKindUUID, // qvalue.QValueKindJSON, - // qvalue.QValueKindBit, + qvalue.QValueKindBit, } numKinds := len(allQValueKinds) diff --git a/flow/connectors/snowflake/client.go b/flow/connectors/snowflake/client.go index ed64a9adb6..5a198a3c16 100644 --- a/flow/connectors/snowflake/client.go +++ b/flow/connectors/snowflake/client.go @@ -116,6 +116,7 @@ func (s *SnowflakeClient) RecreateSchema(schema string) error { // DropSchema drops the schema. func (s *SnowflakeClient) DropSchema(schema string) error { + fmt.Println("dropping schema: ", schema) exists, err := s.schemaExists(schema) if err != nil { return fmt.Errorf("failed to check if schema %s exists: %w", schema, err) @@ -177,53 +178,84 @@ func (s *SnowflakeClient) CheckNull(schema string, tableName string, colNames [] func toQValue(kind qvalue.QValueKind, val interface{}) (qvalue.QValue, error) { switch kind { case qvalue.QValueKindInt32: - if v, ok := val.(*int); ok && v != nil { - return qvalue.QValue{Kind: qvalue.QValueKindInt32, Value: *v}, nil + if v, ok := val.(*sql.NullInt32); ok { + if v.Valid { + return qvalue.QValue{Kind: qvalue.QValueKindInt32, Value: v.Int32}, nil + } else { + return qvalue.QValue{Kind: qvalue.QValueKindInt32, Value: nil}, nil + } } case qvalue.QValueKindInt64: - if v, ok := val.(*int64); ok && v != nil { - return qvalue.QValue{Kind: qvalue.QValueKindInt64, Value: *v}, nil + if v, ok := val.(*sql.NullInt64); ok { + if v.Valid { + return qvalue.QValue{Kind: qvalue.QValueKindInt64, Value: v.Int64}, nil + } else { + return qvalue.QValue{Kind: qvalue.QValueKindInt64, Value: nil}, nil + } } case qvalue.QValueKindFloat32: - if v, ok := val.(*float32); ok && v != nil { - return qvalue.QValue{Kind: qvalue.QValueKindFloat32, Value: *v}, nil + if v, ok := val.(*sql.NullFloat64); ok { + if v.Valid { + return qvalue.QValue{Kind: qvalue.QValueKindFloat32, Value: float32(v.Float64)}, nil + } else { + return qvalue.QValue{Kind: qvalue.QValueKindFloat32, Value: nil}, nil + } } case qvalue.QValueKindFloat64: - if v, ok := val.(*float64); ok && v != nil { - return qvalue.QValue{Kind: qvalue.QValueKindFloat64, Value: *v}, nil + if v, ok := val.(*sql.NullFloat64); ok { + if v.Valid { + return qvalue.QValue{Kind: qvalue.QValueKindFloat64, Value: v.Float64}, nil + } else { + return qvalue.QValue{Kind: qvalue.QValueKindFloat64, Value: nil}, nil + } } case qvalue.QValueKindString: - if v, ok := val.(*string); ok && v != nil { - return qvalue.QValue{Kind: qvalue.QValueKindString, Value: *v}, nil + if v, ok := val.(*sql.NullString); ok { + if v.Valid { + return qvalue.QValue{Kind: qvalue.QValueKindString, Value: v.String}, nil + } else { + return qvalue.QValue{Kind: qvalue.QValueKindString, Value: nil}, nil + } } case qvalue.QValueKindBoolean: - if v, ok := val.(*bool); ok && v != nil { - return qvalue.QValue{Kind: qvalue.QValueKindBoolean, Value: *v}, nil - } - case qvalue.QValueKindNumeric: - // convert string to big.Rat - if v, ok := val.(*string); ok && v != nil { - //nolint:gosec - ratVal, ok := new(big.Rat).SetString(*v) - if !ok { - return qvalue.QValue{}, fmt.Errorf("failed to convert string to big.Rat: %s", *v) + if v, ok := val.(*sql.NullBool); ok { + if v.Valid { + return qvalue.QValue{Kind: qvalue.QValueKindBoolean, Value: v.Bool}, nil + } else { + return qvalue.QValue{Kind: qvalue.QValueKindBoolean, Value: nil}, nil } - return qvalue.QValue{ - Kind: qvalue.QValueKindNumeric, - Value: ratVal, - }, nil } case qvalue.QValueKindTimestamp, qvalue.QValueKindTimestampTZ, qvalue.QValueKindDate, qvalue.QValueKindTime, qvalue.QValueKindTimeTZ: - if t, ok := val.(*time.Time); ok && t != nil { - return qvalue.QValue{ - Kind: kind, - Value: *t, - }, nil + if t, ok := val.(*sql.NullTime); ok { + if t.Valid { + return qvalue.QValue{ + Kind: kind, + Value: t.Time, + }, nil + } else { + return qvalue.QValue{ + Kind: kind, + Value: nil, + }, nil + } + } + case qvalue.QValueKindNumeric: + if v, ok := val.(*sql.NullString); ok { + if v.Valid { + numeric := new(big.Rat) + //nolint:gosec + if _, ok := numeric.SetString(v.String); !ok { + return qvalue.QValue{}, fmt.Errorf("failed to parse numeric: %v", v.String) + } + return qvalue.QValue{Kind: qvalue.QValueKindNumeric, Value: numeric}, nil + } else { + return qvalue.QValue{Kind: qvalue.QValueKindNumeric, Value: nil}, nil + } } - case qvalue.QValueKindBytes: + case qvalue.QValueKindBytes, qvalue.QValueKindBit: if v, ok := val.(*[]byte); ok && v != nil { - return qvalue.QValue{Kind: qvalue.QValueKindBytes, Value: *v}, nil + return qvalue.QValue{Kind: kind, Value: *v}, nil } } @@ -283,25 +315,34 @@ func (s *SnowflakeClient) ExecuteAndProcessQuery(query string) (*model.QRecordBa switch qfields[i].Type { case qvalue.QValueKindTimestamp, qvalue.QValueKindTimestampTZ, qvalue.QValueKindTime, qvalue.QValueKindTimeTZ, qvalue.QValueKindDate: - values[i] = new(time.Time) + var t sql.NullTime + values[i] = &t case qvalue.QValueKindInt16: - values[i] = new(int16) + var n sql.NullInt16 + values[i] = &n case qvalue.QValueKindInt32: - values[i] = new(int32) + var n sql.NullInt32 + values[i] = &n case qvalue.QValueKindInt64: - values[i] = new(int64) + var n sql.NullInt64 + values[i] = &n case qvalue.QValueKindFloat32: - values[i] = new(float32) + var f sql.NullFloat64 + values[i] = &f case qvalue.QValueKindFloat64: - values[i] = new(float64) + var f sql.NullFloat64 + values[i] = &f case qvalue.QValueKindBoolean: - values[i] = new(bool) + var b sql.NullBool + values[i] = &b case qvalue.QValueKindString: - values[i] = new(string) - case qvalue.QValueKindBytes: + var s sql.NullString + values[i] = &s + case qvalue.QValueKindBytes, qvalue.QValueKindBit: values[i] = new([]byte) case qvalue.QValueKindNumeric: - values[i] = new(string) + var s sql.NullString + values[i] = &s default: values[i] = new(interface{}) } diff --git a/flow/connectors/snowflake/qrep_avro_sync.go b/flow/connectors/snowflake/qrep_avro_sync.go index c69e736408..c47889ae51 100644 --- a/flow/connectors/snowflake/qrep_avro_sync.go +++ b/flow/connectors/snowflake/qrep_avro_sync.go @@ -142,8 +142,8 @@ func CopyStageToDestination( appendMode := true if config.WriteMode != nil { - wirteType := config.WriteMode.WriteType - if wirteType == protos.QRepWriteType_QREP_WRITE_MODE_UPSERT { + writeType := config.WriteMode.WriteType + if writeType == protos.QRepWriteType_QREP_WRITE_MODE_UPSERT { appendMode = false } } diff --git a/flow/connectors/snowflake/snowflake.go b/flow/connectors/snowflake/snowflake.go index 0ce38070e1..40b2dd18d3 100644 --- a/flow/connectors/snowflake/snowflake.go +++ b/flow/connectors/snowflake/snowflake.go @@ -210,11 +210,11 @@ func (c *SnowflakeConnector) GetLastOffset(jobName string) (*protos.LastSyncStat return nil, fmt.Errorf("error querying Snowflake peer for last syncedID: %w", err) } - var result int64 if !rows.Next() { log.Warnf("No row found for job %s, returning nil", jobName) return nil, nil } + var result int64 err = rows.Scan(&result) if err != nil { return nil, fmt.Errorf("error while reading result row: %w", err) @@ -725,13 +725,9 @@ func (c *SnowflakeConnector) generateAndExecuteMergeStatement(destinationTableId for columnName, genericColumnType := range normalizedTableSchema.Columns { sfType := qValueKindToSnowflakeType(genericColumnType) switch qvalue.QValueKind(genericColumnType) { - case qvalue.QValueKindBytes: + case qvalue.QValueKindBytes, qvalue.QValueKindBit: flattenedCastsSQLArray = append(flattenedCastsSQLArray, fmt.Sprintf("BASE64_DECODE_BINARY(%s:%s) "+ "AS %s,", toVariantColumnName, columnName, columnName)) - case qvalue.QValueKindBit: - // "c2": {"Bytes": "gA==", "Len": 1,"Valid": true} - flattenedCastsSQLArray = append(flattenedCastsSQLArray, fmt.Sprintf("BASE64_DECODE_BINARY(%s:%s:Bytes) "+ - "AS %s,", toVariantColumnName, columnName, columnName)) // TODO: https://github.com/PeerDB-io/peerdb/issues/189 - handle time types and interval types // case model.ColumnTypeTime: // flattenedCastsSQLArray = append(flattenedCastsSQLArray, fmt.Sprintf("TIME_FROM_PARTS(0,0,0,%s:%s:"+ @@ -845,8 +841,8 @@ func (c *SnowflakeConnector) updateNormalizeMetadata(flowJobName string, normali return nil } -func (c *SnowflakeConnector) createPeerDBInternalSchema(createsSchemaTx *sql.Tx) error { - _, err := createsSchemaTx.ExecContext(c.ctx, fmt.Sprintf(createPeerDBInternalSchemaSQL, peerDBInternalSchema)) +func (c *SnowflakeConnector) createPeerDBInternalSchema(createSchemaTx *sql.Tx) error { + _, err := createSchemaTx.ExecContext(c.ctx, fmt.Sprintf(createPeerDBInternalSchemaSQL, peerDBInternalSchema)) if err != nil { return fmt.Errorf("error while creating internal schema for PeerDB: %w", err) } diff --git a/flow/connectors/utils/azure.go b/flow/connectors/utils/azure.go new file mode 100644 index 0000000000..cded09815e --- /dev/null +++ b/flow/connectors/utils/azure.go @@ -0,0 +1,15 @@ +package utils + +import ( + "fmt" + "os" +) + +func GetAzureSubscriptionID() (string, error) { + // get this from env + id := os.Getenv("AZURE_SUBSCRIPTION_ID") + if id == "" { + return "", fmt.Errorf("AZURE_SUBSCRIPTION_ID is not set") + } + return id, nil +} diff --git a/flow/connectors/utils/postgres.go b/flow/connectors/utils/postgres.go new file mode 100644 index 0000000000..883cf36791 --- /dev/null +++ b/flow/connectors/utils/postgres.go @@ -0,0 +1,22 @@ +package utils + +import ( + "fmt" + "net/url" + + "github.com/PeerDB-io/peer-flow/generated/protos" +) + +func GetPGConnectionString(pgConfig *protos.PostgresConfig) string { + passwordEscaped := url.QueryEscape(pgConfig.Password) + // for a url like postgres://user:password@host:port/dbname + connString := fmt.Sprintf( + "postgres://%s:%s@%s:%d/%s", + pgConfig.User, + passwordEscaped, + pgConfig.Host, + pgConfig.Port, + pgConfig.Database, + ) + return connString +} diff --git a/flow/e2e/bigquery_helper.go b/flow/e2e/bigquery_helper.go index b15bf04e38..95fc2ef7d4 100644 --- a/flow/e2e/bigquery_helper.go +++ b/flow/e2e/bigquery_helper.go @@ -218,6 +218,8 @@ func toQValue(bqValue bigquery.Value) (qvalue.QValue, error) { return qvalue.QValue{Kind: qvalue.QValueKindNumeric, Value: v}, nil case []uint8: return qvalue.QValue{Kind: qvalue.QValueKindBytes, Value: v}, nil + case nil: + return qvalue.QValue{Kind: qvalue.QValueKindInvalid, Value: nil}, nil default: // If type is unsupported, return error return qvalue.QValue{}, fmt.Errorf("bqHelper unsupported type %T", v) @@ -363,7 +365,7 @@ func qValueKindToBqColTypeString(val qvalue.QValueKind) (string, error) { return "BOOL", nil case qvalue.QValueKindTimestamp: return "TIMESTAMP", nil - case qvalue.QValueKindBytes: + case qvalue.QValueKindBytes, qvalue.QValueKindBit: return "BYTES", nil case qvalue.QValueKindNumeric: return "NUMERIC", nil diff --git a/flow/e2e/eventhub_helper.go b/flow/e2e/eventhub_helper.go new file mode 100644 index 0000000000..2457361cb6 --- /dev/null +++ b/flow/e2e/eventhub_helper.go @@ -0,0 +1,129 @@ +package e2e + +import ( + "context" + "sync" + + "github.com/Azure/azure-amqp-common-go/v4/aad" + "github.com/Azure/azure-amqp-common-go/v4/auth" + eventhub "github.com/Azure/azure-event-hubs-go/v3" + "github.com/Azure/azure-sdk-for-go/sdk/azidentity" + "github.com/PeerDB-io/peer-flow/generated/protos" + log "github.com/sirupsen/logrus" +) + +type EventHubTestHelper struct { + creds *azidentity.DefaultAzureCredential + ehConfig *protos.EventHubConfig + tokenProvider auth.TokenProvider +} + +func NewEventHubTestHelper(pgConf *protos.PostgresConfig) (*EventHubTestHelper, error) { + defaultAzureCreds, err := azidentity.NewDefaultAzureCredential(nil) + if err != nil { + log.Errorf("failed to get default azure credentials: %v", err) + return nil, err + } + log.Info("got default azure credentials") + + jwtTokenProvider, err := aad.NewJWTProvider(aad.JWTProviderWithEnvironmentVars()) + if err != nil { + log.Errorf("failed to get jwt token provider: %v", err) + return nil, err + } + log.Info("got jwt token provider") + + ehConfig := &protos.EventHubConfig{ + Namespace: "peerdb-dev", + ResourceGroup: "peerdb-resource", + Location: "eastus", + MetadataDb: pgConf, + } + + return &EventHubTestHelper{ + creds: defaultAzureCreds, + tokenProvider: jwtTokenProvider, + ehConfig: ehConfig, + }, nil +} + +func (h *EventHubTestHelper) GetPeer() *protos.Peer { + return &protos.Peer{ + Name: "test_eh_peer", + Type: protos.DBType_EVENTHUB, + Config: &protos.Peer_EventhubConfig{ + EventhubConfig: h.ehConfig, + }, + } +} + +// consume all messages from the eventhub with the given name. +// returns as a list of strings. +func (h *EventHubTestHelper) ConsumeAllMessages( + ctx context.Context, + name string, + expectedNum int, +) ([]string, error) { + hub, err := eventhub.NewHub(h.ehConfig.Namespace, name, h.tokenProvider) + if err != nil { + log.Errorf("failed to create eventhub hub [%s]: %v", name, err) + return nil, err + } + + var messages []string + + // create a WaitGroup to wait for all messages to be consumed + wg := sync.WaitGroup{} + wg.Add(expectedNum) + + handler := func(c context.Context, event *eventhub.Event) error { + messages = append(messages, string(event.Data)) + log.Infof("received message: %s", string(event.Data)) + wg.Done() + return nil + } + + // listen to each partition of the Event Hub + runtimeInfo, err := hub.GetRuntimeInformation(ctx) + if err != nil { + log.Errorf("failed to get runtime info for eventhub [%s]: %v", name, err) + return nil, err + } + + var listenerHandles []*eventhub.ListenerHandle + + for _, partitionID := range runtimeInfo.PartitionIDs { + // Start receiving messages + // + // Receive blocks while attempting to connect to hub, then runs until listenerHandle.Close() is called + // <- listenerHandle.Done() signals listener has stopped + // listenerHandle.Err() provides the last error the receiver encountered + listenerHandle, err := hub.Receive(ctx, partitionID, handler) + if err != nil { + log.Errorf("failed to receive messages from eventhub [%s]: %v", name, err) + return nil, err + } + + listenerHandles = append(listenerHandles, listenerHandle) + } + + // wait for all messages to be consumed + wg.Wait() + + // close all the listeners + for _, listenerHandle := range listenerHandles { + listenerHandle.Close(ctx) + } + + err = hub.Close(ctx) + if err != nil { + log.Errorf("failed to close eventhub [%s]: %v", name, err) + return nil, err + } + + return messages, nil +} + +func (h *EventHubTestHelper) CleanUp() error { + return nil +} diff --git a/flow/e2e/peer_flow_eh_test.go b/flow/e2e/peer_flow_eh_test.go new file mode 100644 index 0000000000..b1f0a8fdf6 --- /dev/null +++ b/flow/e2e/peer_flow_eh_test.go @@ -0,0 +1,103 @@ +package e2e + +import ( + "context" + "fmt" + "os" + "time" + + util "github.com/PeerDB-io/peer-flow/utils" + peerflow "github.com/PeerDB-io/peer-flow/workflows" + "github.com/stretchr/testify/require" +) + +func (s *E2EPeerFlowTestSuite) setupEventHub() error { + enableEHT := os.Getenv("ENABLE_EVENT_HUB_TESTS") + if enableEHT == "" { + return nil + } + + pgConf := GetTestPostgresConf() + helper, err := NewEventHubTestHelper(pgConf) + if err != nil { + return err + } + + s.ehHelper = helper + return nil +} + +func (s *E2EPeerFlowTestSuite) Test_Complete_Simple_Flow_EH() { + if s.ehHelper == nil { + s.T().Skip("Skipping EventHub test") + } + + env := s.NewTestWorkflowEnvironment() + registerWorkflowsAndActivities(env) + + ru, err := util.RandomUInt64() + s.NoError(err) + + jobName := fmt.Sprintf("test_complete_single_col_flow_eh_%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) + + connectionGen := FlowConnectionGenerationConfig{ + FlowJobName: jobName, + TableNameMapping: map[string]string{schemaQualifiedName: jobName}, + PostgresPort: postgresPort, + Destination: s.ehHelper.GetPeer(), + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + peerFlowInput := 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 `+schemaQualifiedName+` (key, value) VALUES ($1, $2) + `, testKey, testValue) + s.NoError(err) + } + fmt.Println("Inserted 10 rows into the source table") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &peerFlowInput, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + s.NoError(err) + + // Verify that the destination table has 10 rows + // make context with timeout + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + defer cancel() + + msgs, err := s.ehHelper.ConsumeAllMessages(ctx, jobName, 10) + + require.NoError(s.T(), err) + + require.Equal(s.T(), 10, len(msgs)) + + env.AssertExpectations(s.T()) +} diff --git a/flow/e2e/peer_flow_test.go b/flow/e2e/peer_flow_test.go index e10744972e..86afb00ea4 100644 --- a/flow/e2e/peer_flow_test.go +++ b/flow/e2e/peer_flow_test.go @@ -8,12 +8,13 @@ import ( "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/mock" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" "go.temporal.io/sdk/testsuite" @@ -23,9 +24,12 @@ type E2EPeerFlowTestSuite struct { suite.Suite testsuite.WorkflowTestSuite - pool *pgxpool.Pool + pgConnStr string + pool *pgxpool.Pool + bqHelper *BigQueryTestHelper sfHelper *SnowflakeTestHelper + ehHelper *EventHubTestHelper } func TestE2EPeerFlowTestSuite(t *testing.T) { @@ -33,13 +37,27 @@ func TestE2EPeerFlowTestSuite(t *testing.T) { } const ( - postgresPort = 7132 - postgresJdbcURL = "postgres://postgres:postgres@localhost:7132/postgres" + 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 { - pool, err := pgxpool.New(context.Background(), postgresJdbcURL) + 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) } @@ -161,6 +179,11 @@ func (s *E2EPeerFlowTestSuite) SetupSuite() { if err != nil { s.Fail("failed to setup snowflake", err) } + + err = s.setupEventHub() + if err != nil { + s.Fail("failed to setup eventhub", err) + } } // Implement TearDownAllSuite interface to tear down the test suite @@ -185,6 +208,15 @@ func (s *E2EPeerFlowTestSuite) TearDownSuite() { 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) + } } } @@ -204,6 +236,7 @@ func registerWorkflowsAndActivities(env *testsuite.TestWorkflowEnvironment) { env.SetTestTimeout(300 * time.Second) env.RegisterWorkflow(peerflow.PeerFlowWorkflow) + env.RegisterWorkflow(peerflow.PeerFlowWorkflowWithConfig) env.RegisterWorkflow(peerflow.SyncFlowWorkflow) env.RegisterWorkflow(peerflow.SetupFlowWorkflow) env.RegisterWorkflow(peerflow.NormalizeFlowWorkflow) @@ -217,17 +250,13 @@ func (s *E2EPeerFlowTestSuite) Test_Invalid_Connection_Config() { env := s.NewTestWorkflowEnvironment() registerWorkflowsAndActivities(env) - env.OnActivity("FetchConfig", mock.Anything, mock.Anything).Return(nil, nil) - // TODO (kaushikiska): ensure flow name can only be alpha numeric and underscores. - peerFlowInput := peerflow.PeerFlowWorkflowInput{ - PeerFlowName: "invalid_connection_config", - CatalogJdbcURL: postgresJdbcURL, + limits := peerflow.PeerFlowLimits{ TotalSyncFlows: 1, MaxBatchSize: 1, } - env.ExecuteWorkflow(peerflow.PeerFlowWorkflow, &peerFlowInput) + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, nil, &limits, nil) // Verify workflow completes s.True(env.IsWorkflowCompleted()) @@ -263,16 +292,12 @@ func (s *E2EPeerFlowTestSuite) Test_Complete_Flow_No_Data() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - env.OnActivity("FetchConfig", mock.Anything, mock.Anything).Return(flowConnConfig, nil) - - peerFlowInput := peerflow.PeerFlowWorkflowInput{ - PeerFlowName: connectionGen.FlowJobName, - CatalogJdbcURL: postgresJdbcURL, + limits := peerflow.PeerFlowLimits{ TotalSyncFlows: 1, MaxBatchSize: 1, } - env.ExecuteWorkflow(peerflow.PeerFlowWorkflow, &peerFlowInput) + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -307,16 +332,12 @@ func (s *E2EPeerFlowTestSuite) Test_Char_ColType_Error() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - env.OnActivity("FetchConfig", mock.Anything, mock.Anything).Return(flowConnConfig, nil) - - peerFlowInput := peerflow.PeerFlowWorkflowInput{ - PeerFlowName: connectionGen.FlowJobName, - CatalogJdbcURL: postgresJdbcURL, + limits := peerflow.PeerFlowLimits{ TotalSyncFlows: 1, MaxBatchSize: 1, } - env.ExecuteWorkflow(peerflow.PeerFlowWorkflow, &peerFlowInput) + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -354,11 +375,7 @@ func (s *E2EPeerFlowTestSuite) Test_Complete_Simple_Flow_BQ() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - env.OnActivity("FetchConfig", mock.Anything, mock.Anything).Return(flowConnConfig, nil) - - peerFlowInput := peerflow.PeerFlowWorkflowInput{ - PeerFlowName: connectionGen.FlowJobName, - CatalogJdbcURL: postgresJdbcURL, + limits := peerflow.PeerFlowLimits{ TotalSyncFlows: 2, MaxBatchSize: 100, } @@ -379,7 +396,7 @@ func (s *E2EPeerFlowTestSuite) Test_Complete_Simple_Flow_BQ() { fmt.Println("Inserted 10 rows into the source table") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflow, &peerFlowInput) + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -422,11 +439,7 @@ func (s *E2EPeerFlowTestSuite) Test_Toast_BQ() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - env.OnActivity("FetchConfig", mock.Anything, mock.Anything).Return(flowConnConfig, nil) - - peerFlowInput := peerflow.PeerFlowWorkflowInput{ - PeerFlowName: connectionGen.FlowJobName, - CatalogJdbcURL: postgresJdbcURL, + limits := peerflow.PeerFlowLimits{ TotalSyncFlows: 1, MaxBatchSize: 100, } @@ -453,7 +466,7 @@ func (s *E2EPeerFlowTestSuite) Test_Toast_BQ() { fmt.Println("Executed a transaction touching toast columns") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflow, &peerFlowInput) + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -493,11 +506,7 @@ func (s *E2EPeerFlowTestSuite) Test_Toast_Nochanges_BQ() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - env.OnActivity("FetchConfig", mock.Anything, mock.Anything).Return(flowConnConfig, nil) - - peerFlowInput := peerflow.PeerFlowWorkflowInput{ - PeerFlowName: connectionGen.FlowJobName, - CatalogJdbcURL: postgresJdbcURL, + limits := peerflow.PeerFlowLimits{ TotalSyncFlows: 1, MaxBatchSize: 100, } @@ -517,7 +526,7 @@ func (s *E2EPeerFlowTestSuite) Test_Toast_Nochanges_BQ() { fmt.Println("Executed a transaction touching toast columns") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflow, &peerFlowInput) + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -558,11 +567,7 @@ func (s *E2EPeerFlowTestSuite) Test_Toast_Advance_1_BQ() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - env.OnActivity("FetchConfig", mock.Anything, mock.Anything).Return(flowConnConfig, nil) - - peerFlowInput := peerflow.PeerFlowWorkflowInput{ - PeerFlowName: connectionGen.FlowJobName, - CatalogJdbcURL: postgresJdbcURL, + limits := peerflow.PeerFlowLimits{ TotalSyncFlows: 1, MaxBatchSize: 100, } @@ -594,7 +599,7 @@ func (s *E2EPeerFlowTestSuite) Test_Toast_Advance_1_BQ() { fmt.Println("Executed a transaction touching toast columns") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflow, &peerFlowInput) + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -660,11 +665,7 @@ func (s *E2EPeerFlowTestSuite) Test_Toast_Advance_2_BQ() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - env.OnActivity("FetchConfig", mock.Anything, mock.Anything).Return(flowConnConfig, nil) - - peerFlowInput := peerflow.PeerFlowWorkflowInput{ - PeerFlowName: connectionGen.FlowJobName, - CatalogJdbcURL: postgresJdbcURL, + limits := peerflow.PeerFlowLimits{ TotalSyncFlows: 1, MaxBatchSize: 100, } @@ -691,7 +692,7 @@ func (s *E2EPeerFlowTestSuite) Test_Toast_Advance_2_BQ() { fmt.Println("Executed a transaction touching toast columns") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflow, &peerFlowInput) + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -731,11 +732,8 @@ func (s *E2EPeerFlowTestSuite) Test_Toast_Advance_3_BQ() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - env.OnActivity("FetchConfig", mock.Anything, mock.Anything).Return(flowConnConfig, nil) + limits := peerflow.PeerFlowLimits{ - peerFlowInput := peerflow.PeerFlowWorkflowInput{ - PeerFlowName: connectionGen.FlowJobName, - CatalogJdbcURL: postgresJdbcURL, TotalSyncFlows: 1, MaxBatchSize: 100, } @@ -761,7 +759,7 @@ func (s *E2EPeerFlowTestSuite) Test_Toast_Advance_3_BQ() { fmt.Println("Executed a transaction touching toast columns") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflow, &peerFlowInput) + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -806,11 +804,8 @@ func (s *E2EPeerFlowTestSuite) Test_Types_BQ() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - env.OnActivity("FetchConfig", mock.Anything, mock.Anything).Return(flowConnConfig, nil) + limits := peerflow.PeerFlowLimits{ - peerFlowInput := peerflow.PeerFlowWorkflowInput{ - PeerFlowName: connectionGen.FlowJobName, - CatalogJdbcURL: postgresJdbcURL, TotalSyncFlows: 1, MaxBatchSize: 100, } @@ -835,7 +830,7 @@ func (s *E2EPeerFlowTestSuite) Test_Types_BQ() { fmt.Println("Executed an insert with all types") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflow, &peerFlowInput) + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -877,11 +872,7 @@ func (s *E2EPeerFlowTestSuite) Test_Multi_Table_BQ() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - env.OnActivity("FetchConfig", mock.Anything, mock.Anything).Return(flowConnConfig, nil) - - peerFlowInput := peerflow.PeerFlowWorkflowInput{ - PeerFlowName: connectionGen.FlowJobName, - CatalogJdbcURL: postgresJdbcURL, + limits := peerflow.PeerFlowLimits{ TotalSyncFlows: 1, MaxBatchSize: 100, } @@ -899,7 +890,7 @@ func (s *E2EPeerFlowTestSuite) Test_Multi_Table_BQ() { fmt.Println("Executed an insert with all types") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflow, &peerFlowInput) + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -941,11 +932,7 @@ func (s *E2EPeerFlowTestSuite) Test_Complete_Simple_Flow_SF() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - env.OnActivity("FetchConfig", mock.Anything, mock.Anything).Return(flowConnConfig, nil) - - peerFlowInput := peerflow.PeerFlowWorkflowInput{ - PeerFlowName: connectionGen.FlowJobName, - CatalogJdbcURL: postgresJdbcURL, + limits := peerflow.PeerFlowLimits{ TotalSyncFlows: 2, MaxBatchSize: 100, } @@ -966,7 +953,7 @@ func (s *E2EPeerFlowTestSuite) Test_Complete_Simple_Flow_SF() { fmt.Println("Inserted 10 rows into the source table") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflow, &peerFlowInput) + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -1014,11 +1001,7 @@ func (s *E2EPeerFlowTestSuite) Test_Toast_SF() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - env.OnActivity("FetchConfig", mock.Anything, mock.Anything).Return(flowConnConfig, nil) - - peerFlowInput := peerflow.PeerFlowWorkflowInput{ - PeerFlowName: connectionGen.FlowJobName, - CatalogJdbcURL: postgresJdbcURL, + limits := peerflow.PeerFlowLimits{ TotalSyncFlows: 1, MaxBatchSize: 100, } @@ -1045,7 +1028,7 @@ func (s *E2EPeerFlowTestSuite) Test_Toast_SF() { fmt.Println("Executed a transaction touching toast columns") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflow, &peerFlowInput) + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -1053,7 +1036,7 @@ func (s *E2EPeerFlowTestSuite) Test_Toast_SF() { s.NoError(err) - s.compareTableContentsSF("test_toast_sf_1", "id,t1,t2,k") + s.compareTableContentsSF("test_toast_sf_1", `id,t1,t2,k`, false) env.AssertExpectations(s.T()) } @@ -1086,11 +1069,7 @@ func (s *E2EPeerFlowTestSuite) Test_Toast_Nochanges_SF() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - env.OnActivity("FetchConfig", mock.Anything, mock.Anything).Return(flowConnConfig, nil) - - peerFlowInput := peerflow.PeerFlowWorkflowInput{ - PeerFlowName: connectionGen.FlowJobName, - CatalogJdbcURL: postgresJdbcURL, + limits := peerflow.PeerFlowLimits{ TotalSyncFlows: 1, MaxBatchSize: 100, } @@ -1110,7 +1089,7 @@ func (s *E2EPeerFlowTestSuite) Test_Toast_Nochanges_SF() { fmt.Println("Executed a transaction touching toast columns") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflow, &peerFlowInput) + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -1119,7 +1098,7 @@ func (s *E2EPeerFlowTestSuite) Test_Toast_Nochanges_SF() { // assert that error contains "invalid connection configs" s.NoError(err) - s.compareTableContentsSF("test_toast_sf_2", "id,t1,t2,k") + s.compareTableContentsSF("test_toast_sf_2", `id,t1,t2,k`, false) env.AssertExpectations(s.T()) } @@ -1152,11 +1131,7 @@ func (s *E2EPeerFlowTestSuite) Test_Toast_Advance_1_SF() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - env.OnActivity("FetchConfig", mock.Anything, mock.Anything).Return(flowConnConfig, nil) - - peerFlowInput := peerflow.PeerFlowWorkflowInput{ - PeerFlowName: connectionGen.FlowJobName, - CatalogJdbcURL: postgresJdbcURL, + limits := peerflow.PeerFlowLimits{ TotalSyncFlows: 2, MaxBatchSize: 100, } @@ -1188,7 +1163,7 @@ func (s *E2EPeerFlowTestSuite) Test_Toast_Advance_1_SF() { fmt.Println("Executed a transaction touching toast columns") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflow, &peerFlowInput) + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -1196,7 +1171,7 @@ func (s *E2EPeerFlowTestSuite) Test_Toast_Advance_1_SF() { s.NoError(err) - s.compareTableContentsSF("test_toast_sf_3", "id,t1,t2,k") + s.compareTableContentsSF("test_toast_sf_3", `id,t1,t2,k`, false) env.AssertExpectations(s.T()) } @@ -1228,11 +1203,7 @@ func (s *E2EPeerFlowTestSuite) Test_Toast_Advance_2_SF() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - env.OnActivity("FetchConfig", mock.Anything, mock.Anything).Return(flowConnConfig, nil) - - peerFlowInput := peerflow.PeerFlowWorkflowInput{ - PeerFlowName: connectionGen.FlowJobName, - CatalogJdbcURL: postgresJdbcURL, + limits := peerflow.PeerFlowLimits{ TotalSyncFlows: 1, MaxBatchSize: 100, } @@ -1259,7 +1230,7 @@ func (s *E2EPeerFlowTestSuite) Test_Toast_Advance_2_SF() { fmt.Println("Executed a transaction touching toast columns") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflow, &peerFlowInput) + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -1267,7 +1238,7 @@ func (s *E2EPeerFlowTestSuite) Test_Toast_Advance_2_SF() { s.NoError(err) - s.compareTableContentsSF("test_toast_sf_4", "id,t1,k") + s.compareTableContentsSF("test_toast_sf_4", `id,t1,k`, false) env.AssertExpectations(s.T()) } @@ -1300,11 +1271,7 @@ func (s *E2EPeerFlowTestSuite) Test_Toast_Advance_3_SF() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - env.OnActivity("FetchConfig", mock.Anything, mock.Anything).Return(flowConnConfig, nil) - - peerFlowInput := peerflow.PeerFlowWorkflowInput{ - PeerFlowName: connectionGen.FlowJobName, - CatalogJdbcURL: postgresJdbcURL, + limits := peerflow.PeerFlowLimits{ TotalSyncFlows: 1, MaxBatchSize: 100, } @@ -1330,7 +1297,7 @@ func (s *E2EPeerFlowTestSuite) Test_Toast_Advance_3_SF() { fmt.Println("Executed a transaction touching toast columns") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflow, &peerFlowInput) + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -1338,7 +1305,7 @@ func (s *E2EPeerFlowTestSuite) Test_Toast_Advance_3_SF() { s.NoError(err) - s.compareTableContentsSF("test_toast_sf_5", "id,t1,t2,k") + s.compareTableContentsSF("test_toast_sf_5", `id,t1,t2,k`, false) env.AssertExpectations(s.T()) } @@ -1376,11 +1343,7 @@ func (s *E2EPeerFlowTestSuite) Test_Types_SF() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - env.OnActivity("FetchConfig", mock.Anything, mock.Anything).Return(flowConnConfig, nil) - - peerFlowInput := peerflow.PeerFlowWorkflowInput{ - PeerFlowName: connectionGen.FlowJobName, - CatalogJdbcURL: postgresJdbcURL, + limits := peerflow.PeerFlowLimits{ TotalSyncFlows: 1, MaxBatchSize: 100, } @@ -1405,7 +1368,7 @@ func (s *E2EPeerFlowTestSuite) Test_Types_SF() { fmt.Println("Executed an insert with all types") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflow, &peerFlowInput) + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -1449,11 +1412,7 @@ func (s *E2EPeerFlowTestSuite) Test_Multi_Table_SF() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - env.OnActivity("FetchConfig", mock.Anything, mock.Anything).Return(flowConnConfig, nil) - - peerFlowInput := peerflow.PeerFlowWorkflowInput{ - PeerFlowName: connectionGen.FlowJobName, - CatalogJdbcURL: postgresJdbcURL, + limits := peerflow.PeerFlowLimits{ TotalSyncFlows: 1, MaxBatchSize: 100, } @@ -1471,7 +1430,7 @@ func (s *E2EPeerFlowTestSuite) Test_Multi_Table_SF() { fmt.Println("Executed an insert with all types") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflow, &peerFlowInput) + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) diff --git a/flow/e2e/qrep_flow_test.go b/flow/e2e/qrep_flow_test.go index 0a210d1656..624ec78d20 100644 --- a/flow/e2e/qrep_flow_test.go +++ b/flow/e2e/qrep_flow_test.go @@ -59,7 +59,7 @@ func (s *E2EPeerFlowTestSuite) createSourceTable(tableName string) { } func (s *E2EPeerFlowTestSuite) populateSourceTable(tableName string, rowCount int) { - for i := 0; i < rowCount; i++ { + for i := 0; i < rowCount-1; i++ { _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` INSERT INTO e2e_test.%s ( id, card_id, "from", price, created_at, @@ -82,6 +82,18 @@ func (s *E2EPeerFlowTestSuite) populateSourceTable(tableName string, rowCount in uuid.New().String(), uuid.New().String(), uuid.New().String(), uuid.New().String())) s.NoError(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) } func (s *E2EPeerFlowTestSuite) setupSourceTable(tableName string, rowCount int) { @@ -200,7 +212,7 @@ func (s *E2EPeerFlowTestSuite) compareTableContentsBQ(tableName string, colsStri s.True(pgRows.Equals(bqRows), "rows from source and destination tables are not equal") } -func (s *E2EPeerFlowTestSuite) compareTableContentsSF(tableName string, selector string) { +func (s *E2EPeerFlowTestSuite) compareTableContentsSF(tableName string, selector string, caseSensitive bool) { // read rows from source table pgQueryExecutor := connpostgres.NewQRepQueryExecutor(s.pool, context.Background()) pgRows, err := pgQueryExecutor.ExecuteAndProcessQuery( @@ -210,7 +222,12 @@ func (s *E2EPeerFlowTestSuite) compareTableContentsSF(tableName string, selector // read rows from destination table qualifiedTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, tableName) - sfSelQuery := fmt.Sprintf(`SELECT %s FROM %s ORDER BY "id"`, selector, qualifiedTableName) + 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) sfRows, err := s.sfHelper.ExecuteAndProcessQuery(sfSelQuery) require.NoError(s.T(), err) @@ -330,7 +347,7 @@ func (s *E2EPeerFlowTestSuite) Test_Complete_QRep_Flow_Avro_SF() { env := s.NewTestWorkflowEnvironment() registerWorkflowsAndActivities(env) - numRows := 1 + numRows := 10 tblName := "test_qrep_flow_avro_sf" s.setupSourceTable(tblName, numRows) @@ -359,7 +376,7 @@ func (s *E2EPeerFlowTestSuite) Test_Complete_QRep_Flow_Avro_SF() { s.NoError(err) sel := getOwnersSelectorString() - s.compareTableContentsSF(tblName, sel) + s.compareTableContentsSF(tblName, sel, true) env.AssertExpectations(s.T()) } @@ -401,7 +418,7 @@ func (s *E2EPeerFlowTestSuite) Test_Complete_QRep_Flow_Avro_SF_Upsert_Simple() { s.NoError(err) sel := getOwnersSelectorString() - s.compareTableContentsSF(tblName, sel) + s.compareTableContentsSF(tblName, sel, true) env.AssertExpectations(s.T()) } @@ -485,7 +502,7 @@ func (s *E2EPeerFlowTestSuite) Test_Complete_QRep_Flow_Avro_SF_S3() { s.NoError(err) sel := getOwnersSelectorString() - s.compareTableContentsSF(tblName, sel) + s.compareTableContentsSF(tblName, sel, true) env.AssertExpectations(s.T()) } @@ -527,7 +544,7 @@ func (s *E2EPeerFlowTestSuite) Test_Complete_QRep_Flow_Avro_SF_S3_Integration() s.NoError(err) sel := getOwnersSelectorString() - s.compareTableContentsSF(tblName, sel) + s.compareTableContentsSF(tblName, sel, true) env.AssertExpectations(s.T()) } diff --git a/flow/generated/protos/flow.pb.go b/flow/generated/protos/flow.pb.go index 40dd807c91..0c1c05a2cb 100644 --- a/flow/generated/protos/flow.pb.go +++ b/flow/generated/protos/flow.pb.go @@ -181,6 +181,10 @@ type FlowConnectionConfigs struct { TableNameMapping map[string]string `protobuf:"bytes,5,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"` SrcTableIdNameMapping map[uint32]string `protobuf:"bytes,6,rep,name=src_table_id_name_mapping,json=srcTableIdNameMapping,proto3" json:"src_table_id_name_mapping,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` TableNameSchemaMapping map[string]*TableSchema `protobuf:"bytes,7,rep,name=table_name_schema_mapping,json=tableNameSchemaMapping,proto3" json:"table_name_schema_mapping,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + // This is an optional peer that will be used to hold metadata in cases where + // the destination isn't ideal for holding metadata. + MetadataPeer *Peer `protobuf:"bytes,8,opt,name=metadata_peer,json=metadataPeer,proto3" json:"metadata_peer,omitempty"` + MaxBatchSize uint32 `protobuf:"varint,9,opt,name=max_batch_size,json=maxBatchSize,proto3" json:"max_batch_size,omitempty"` } func (x *FlowConnectionConfigs) Reset() { @@ -264,6 +268,20 @@ func (x *FlowConnectionConfigs) GetTableNameSchemaMapping() map[string]*TableSch return nil } +func (x *FlowConnectionConfigs) GetMetadataPeer() *Peer { + if x != nil { + return x.MetadataPeer + } + return nil +} + +func (x *FlowConnectionConfigs) GetMaxBatchSize() uint32 { + if x != nil { + return x.MaxBatchSize + } + return 0 +} + type SyncFlowOptions struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -870,8 +888,9 @@ type CreateRawTableInput struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - PeerConnectionConfig *Peer `protobuf:"bytes,1,opt,name=peer_connection_config,json=peerConnectionConfig,proto3" json:"peer_connection_config,omitempty"` - FlowJobName string `protobuf:"bytes,2,opt,name=flow_job_name,json=flowJobName,proto3" json:"flow_job_name,omitempty"` + PeerConnectionConfig *Peer `protobuf:"bytes,1,opt,name=peer_connection_config,json=peerConnectionConfig,proto3" json:"peer_connection_config,omitempty"` + 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"` } func (x *CreateRawTableInput) Reset() { @@ -920,6 +939,13 @@ func (x *CreateRawTableInput) GetFlowJobName() string { return "" } +func (x *CreateRawTableInput) GetTableNameMapping() map[string]string { + if x != nil { + return x.TableNameMapping + } + return nil +} + type CreateRawTableOutput struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -1783,7 +1809,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, 0xaa, 0x06, + 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x89, 0x07, 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, @@ -1819,275 +1845,292 @@ var file_flow_proto_rawDesc = []byte{ 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, 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, 0x2e, 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, 0x2e, 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, 0x2e, 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, 0x2e, 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, 0x2e, 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, 0x2e, 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, 0x2e, 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, 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, - 0x2e, 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, 0x2e, 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, 0xb2, 0x02, 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, 0x2e, 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, 0x2e, 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, + 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, 0x2e, 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, 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, 0x2e, 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, 0x2e, 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, 0x2e, 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, 0x2e, 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, 0x2e, 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, 0x2e, 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, + 0x2e, 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, 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, 0x2e, + 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, 0x2e, 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, + 0xb2, 0x02, 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, 0x2e, 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, 0x2e, 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, 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, 0xae, 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, 0x2e, 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, 0x2e, + 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, 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, 0x83, 0x01, 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, 0x2e, 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, 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, 0x8a, - 0x01, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, - 0x61, 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, 0x2e, - 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, 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, - 0x2e, 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, 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, 0x2e, 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, 0x2e, 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, 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, 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, 0xa9, 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, 0x2e, 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, 0x2e, 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, 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, - 0x2e, 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, 0xe1, 0x05, 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, 0x2e, 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, + 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, 0x8a, 0x01, 0x0a, 0x13, 0x47, 0x65, 0x74, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 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, 0x2e, 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, 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, 0x2e, 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, 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, 0x2e, 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, 0x2e, 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, 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, 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, 0xa9, 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, 0x2e, + 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, 0x2e, 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, 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, 0x2e, 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, 0xe1, 0x05, 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, 0x2e, 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, 0x2e, 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, 0x2e, - 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, - 0x22, 0x65, 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, 0x2e, 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, 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, 0x2e, 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, 0x12, 0x5a, 0x10, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x64, 0x2f, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x73, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 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, 0x2e, 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, 0x2e, 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, 0x2e, 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, 0x22, 0x65, 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, 0x2e, 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, 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, 0x2e, 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, 0x12, 0x5a, 0x10, + 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x64, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x73, + 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -2103,7 +2146,7 @@ func file_flow_proto_rawDescGZIP() []byte { } var file_flow_proto_enumTypes = make([]protoimpl.EnumInfo, 2) -var file_flow_proto_msgTypes = make([]protoimpl.MessageInfo, 32) +var file_flow_proto_msgTypes = make([]protoimpl.MessageInfo, 33) var file_flow_proto_goTypes = []interface{}{ (QRepSyncMode)(0), // 0: peerdb.flow.QRepSyncMode (QRepWriteType)(0), // 1: peerdb.flow.QRepWriteType @@ -2138,50 +2181,53 @@ var file_flow_proto_goTypes = []interface{}{ nil, // 30: peerdb.flow.FlowConnectionConfigs.SrcTableIdNameMappingEntry nil, // 31: peerdb.flow.FlowConnectionConfigs.TableNameSchemaMappingEntry nil, // 32: peerdb.flow.SetupReplicationInput.TableNameMappingEntry - nil, // 33: peerdb.flow.TableSchema.ColumnsEntry - (*Peer)(nil), // 34: peerdb.peers.Peer - (*timestamppb.Timestamp)(nil), // 35: google.protobuf.Timestamp + nil, // 33: peerdb.flow.CreateRawTableInput.TableNameMappingEntry + nil, // 34: peerdb.flow.TableSchema.ColumnsEntry + (*Peer)(nil), // 35: peerdb.peers.Peer + (*timestamppb.Timestamp)(nil), // 36: google.protobuf.Timestamp } var file_flow_proto_depIdxs = []int32{ - 34, // 0: peerdb.flow.FlowConnectionConfigs.source:type_name -> peerdb.peers.Peer - 34, // 1: peerdb.flow.FlowConnectionConfigs.destination:type_name -> peerdb.peers.Peer + 35, // 0: peerdb.flow.FlowConnectionConfigs.source:type_name -> peerdb.peers.Peer + 35, // 1: peerdb.flow.FlowConnectionConfigs.destination:type_name -> peerdb.peers.Peer 18, // 2: peerdb.flow.FlowConnectionConfigs.table_schema:type_name -> peerdb.flow.TableSchema 29, // 3: peerdb.flow.FlowConnectionConfigs.table_name_mapping:type_name -> peerdb.flow.FlowConnectionConfigs.TableNameMappingEntry 30, // 4: peerdb.flow.FlowConnectionConfigs.src_table_id_name_mapping:type_name -> peerdb.flow.FlowConnectionConfigs.SrcTableIdNameMappingEntry 31, // 5: peerdb.flow.FlowConnectionConfigs.table_name_schema_mapping:type_name -> peerdb.flow.FlowConnectionConfigs.TableNameSchemaMappingEntry - 35, // 6: peerdb.flow.LastSyncState.last_synced_at:type_name -> google.protobuf.Timestamp - 6, // 7: peerdb.flow.StartFlowInput.last_sync_state:type_name -> peerdb.flow.LastSyncState - 3, // 8: peerdb.flow.StartFlowInput.flow_connection_configs:type_name -> peerdb.flow.FlowConnectionConfigs - 4, // 9: peerdb.flow.StartFlowInput.sync_flow_options:type_name -> peerdb.flow.SyncFlowOptions - 3, // 10: peerdb.flow.StartNormalizeInput.flow_connection_configs:type_name -> peerdb.flow.FlowConnectionConfigs - 34, // 11: peerdb.flow.GetLastSyncedIDInput.peer_connection_config:type_name -> peerdb.peers.Peer - 34, // 12: peerdb.flow.EnsurePullabilityInput.peer_connection_config:type_name -> peerdb.peers.Peer - 11, // 13: peerdb.flow.TableIdentifier.postgres_table_identifier:type_name -> peerdb.flow.PostgresTableIdentifier - 12, // 14: peerdb.flow.EnsurePullabilityOutput.table_identifier:type_name -> peerdb.flow.TableIdentifier - 34, // 15: peerdb.flow.SetupReplicationInput.peer_connection_config:type_name -> peerdb.peers.Peer - 32, // 16: peerdb.flow.SetupReplicationInput.table_name_mapping:type_name -> peerdb.flow.SetupReplicationInput.TableNameMappingEntry - 34, // 17: peerdb.flow.CreateRawTableInput.peer_connection_config:type_name -> peerdb.peers.Peer - 34, // 18: peerdb.flow.GetTableSchemaInput.peer_connection_config:type_name -> peerdb.peers.Peer - 33, // 19: peerdb.flow.TableSchema.columns:type_name -> peerdb.flow.TableSchema.ColumnsEntry - 34, // 20: peerdb.flow.SetupNormalizedTableInput.peer_connection_config:type_name -> peerdb.peers.Peer - 18, // 21: peerdb.flow.SetupNormalizedTableInput.source_table_schema:type_name -> peerdb.flow.TableSchema - 35, // 22: peerdb.flow.TimestampPartitionRange.start:type_name -> google.protobuf.Timestamp - 35, // 23: peerdb.flow.TimestampPartitionRange.end:type_name -> google.protobuf.Timestamp - 21, // 24: peerdb.flow.PartitionRange.int_range:type_name -> peerdb.flow.IntPartitionRange - 22, // 25: peerdb.flow.PartitionRange.timestamp_range:type_name -> peerdb.flow.TimestampPartitionRange - 1, // 26: peerdb.flow.QRepWriteMode.write_type:type_name -> peerdb.flow.QRepWriteType - 34, // 27: peerdb.flow.QRepConfig.source_peer:type_name -> peerdb.peers.Peer - 34, // 28: peerdb.flow.QRepConfig.destination_peer:type_name -> peerdb.peers.Peer - 0, // 29: peerdb.flow.QRepConfig.sync_mode:type_name -> peerdb.flow.QRepSyncMode - 24, // 30: peerdb.flow.QRepConfig.write_mode:type_name -> peerdb.flow.QRepWriteMode - 23, // 31: peerdb.flow.QRepPartition.range:type_name -> peerdb.flow.PartitionRange - 26, // 32: peerdb.flow.QRepParitionResult.partitions:type_name -> peerdb.flow.QRepPartition - 18, // 33: peerdb.flow.FlowConnectionConfigs.TableNameSchemaMappingEntry.value:type_name -> peerdb.flow.TableSchema - 34, // [34:34] is the sub-list for method output_type - 34, // [34:34] is the sub-list for method input_type - 34, // [34:34] is the sub-list for extension type_name - 34, // [34:34] is the sub-list for extension extendee - 0, // [0:34] is the sub-list for field type_name + 35, // 6: peerdb.flow.FlowConnectionConfigs.metadata_peer:type_name -> peerdb.peers.Peer + 36, // 7: peerdb.flow.LastSyncState.last_synced_at:type_name -> google.protobuf.Timestamp + 6, // 8: peerdb.flow.StartFlowInput.last_sync_state:type_name -> peerdb.flow.LastSyncState + 3, // 9: peerdb.flow.StartFlowInput.flow_connection_configs:type_name -> peerdb.flow.FlowConnectionConfigs + 4, // 10: peerdb.flow.StartFlowInput.sync_flow_options:type_name -> peerdb.flow.SyncFlowOptions + 3, // 11: peerdb.flow.StartNormalizeInput.flow_connection_configs:type_name -> peerdb.flow.FlowConnectionConfigs + 35, // 12: peerdb.flow.GetLastSyncedIDInput.peer_connection_config:type_name -> peerdb.peers.Peer + 35, // 13: peerdb.flow.EnsurePullabilityInput.peer_connection_config:type_name -> peerdb.peers.Peer + 11, // 14: peerdb.flow.TableIdentifier.postgres_table_identifier:type_name -> peerdb.flow.PostgresTableIdentifier + 12, // 15: peerdb.flow.EnsurePullabilityOutput.table_identifier:type_name -> peerdb.flow.TableIdentifier + 35, // 16: peerdb.flow.SetupReplicationInput.peer_connection_config:type_name -> peerdb.peers.Peer + 32, // 17: peerdb.flow.SetupReplicationInput.table_name_mapping:type_name -> peerdb.flow.SetupReplicationInput.TableNameMappingEntry + 35, // 18: peerdb.flow.CreateRawTableInput.peer_connection_config:type_name -> peerdb.peers.Peer + 33, // 19: peerdb.flow.CreateRawTableInput.table_name_mapping:type_name -> peerdb.flow.CreateRawTableInput.TableNameMappingEntry + 35, // 20: peerdb.flow.GetTableSchemaInput.peer_connection_config:type_name -> peerdb.peers.Peer + 34, // 21: peerdb.flow.TableSchema.columns:type_name -> peerdb.flow.TableSchema.ColumnsEntry + 35, // 22: peerdb.flow.SetupNormalizedTableInput.peer_connection_config:type_name -> peerdb.peers.Peer + 18, // 23: peerdb.flow.SetupNormalizedTableInput.source_table_schema:type_name -> peerdb.flow.TableSchema + 36, // 24: peerdb.flow.TimestampPartitionRange.start:type_name -> google.protobuf.Timestamp + 36, // 25: peerdb.flow.TimestampPartitionRange.end:type_name -> google.protobuf.Timestamp + 21, // 26: peerdb.flow.PartitionRange.int_range:type_name -> peerdb.flow.IntPartitionRange + 22, // 27: peerdb.flow.PartitionRange.timestamp_range:type_name -> peerdb.flow.TimestampPartitionRange + 1, // 28: peerdb.flow.QRepWriteMode.write_type:type_name -> peerdb.flow.QRepWriteType + 35, // 29: peerdb.flow.QRepConfig.source_peer:type_name -> peerdb.peers.Peer + 35, // 30: peerdb.flow.QRepConfig.destination_peer:type_name -> peerdb.peers.Peer + 0, // 31: peerdb.flow.QRepConfig.sync_mode:type_name -> peerdb.flow.QRepSyncMode + 24, // 32: peerdb.flow.QRepConfig.write_mode:type_name -> peerdb.flow.QRepWriteMode + 23, // 33: peerdb.flow.QRepPartition.range:type_name -> peerdb.flow.PartitionRange + 26, // 34: peerdb.flow.QRepParitionResult.partitions:type_name -> peerdb.flow.QRepPartition + 18, // 35: peerdb.flow.FlowConnectionConfigs.TableNameSchemaMappingEntry.value:type_name -> peerdb.flow.TableSchema + 36, // [36:36] is the sub-list for method output_type + 36, // [36:36] is the sub-list for method input_type + 36, // [36:36] is the sub-list for extension type_name + 36, // [36:36] is the sub-list for extension extendee + 0, // [0:36] is the sub-list for field type_name } func init() { file_flow_proto_init() } @@ -2529,7 +2575,7 @@ func file_flow_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_flow_proto_rawDesc, NumEnums: 2, - NumMessages: 32, + NumMessages: 33, NumExtensions: 0, NumServices: 0, }, diff --git a/flow/generated/protos/peers.pb.go b/flow/generated/protos/peers.pb.go index 252879d508..acb05637ad 100644 --- a/flow/generated/protos/peers.pb.go +++ b/flow/generated/protos/peers.pb.go @@ -27,6 +27,7 @@ const ( DBType_SNOWFLAKE DBType = 1 DBType_MONGO DBType = 2 DBType_POSTGRES DBType = 3 + DBType_EVENTHUB DBType = 4 ) // Enum value maps for DBType. @@ -36,12 +37,14 @@ var ( 1: "SNOWFLAKE", 2: "MONGO", 3: "POSTGRES", + 4: "EVENTHUB", } DBType_value = map[string]int32{ "BIGQUERY": 0, "SNOWFLAKE": 1, "MONGO": 2, "POSTGRES": 3, + "EVENTHUB": 4, } ) @@ -460,6 +463,77 @@ func (x *PostgresConfig) GetDatabase() string { return "" } +type EventHubConfig struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` + ResourceGroup string `protobuf:"bytes,2,opt,name=resource_group,json=resourceGroup,proto3" json:"resource_group,omitempty"` + Location string `protobuf:"bytes,3,opt,name=location,proto3" json:"location,omitempty"` + MetadataDb *PostgresConfig `protobuf:"bytes,4,opt,name=metadata_db,json=metadataDb,proto3" json:"metadata_db,omitempty"` +} + +func (x *EventHubConfig) Reset() { + *x = EventHubConfig{} + if protoimpl.UnsafeEnabled { + mi := &file_peers_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *EventHubConfig) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*EventHubConfig) ProtoMessage() {} + +func (x *EventHubConfig) ProtoReflect() protoreflect.Message { + mi := &file_peers_proto_msgTypes[4] + 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 EventHubConfig.ProtoReflect.Descriptor instead. +func (*EventHubConfig) Descriptor() ([]byte, []int) { + return file_peers_proto_rawDescGZIP(), []int{4} +} + +func (x *EventHubConfig) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *EventHubConfig) GetResourceGroup() string { + if x != nil { + return x.ResourceGroup + } + return "" +} + +func (x *EventHubConfig) GetLocation() string { + if x != nil { + return x.Location + } + return "" +} + +func (x *EventHubConfig) GetMetadataDb() *PostgresConfig { + if x != nil { + return x.MetadataDb + } + return nil +} + type Peer struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -473,13 +547,14 @@ type Peer struct { // *Peer_BigqueryConfig // *Peer_MongoConfig // *Peer_PostgresConfig + // *Peer_EventhubConfig Config isPeer_Config `protobuf_oneof:"config"` } func (x *Peer) Reset() { *x = Peer{} if protoimpl.UnsafeEnabled { - mi := &file_peers_proto_msgTypes[4] + mi := &file_peers_proto_msgTypes[5] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -492,7 +567,7 @@ func (x *Peer) String() string { func (*Peer) ProtoMessage() {} func (x *Peer) ProtoReflect() protoreflect.Message { - mi := &file_peers_proto_msgTypes[4] + mi := &file_peers_proto_msgTypes[5] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -505,7 +580,7 @@ func (x *Peer) ProtoReflect() protoreflect.Message { // Deprecated: Use Peer.ProtoReflect.Descriptor instead. func (*Peer) Descriptor() ([]byte, []int) { - return file_peers_proto_rawDescGZIP(), []int{4} + return file_peers_proto_rawDescGZIP(), []int{5} } func (x *Peer) GetName() string { @@ -557,6 +632,13 @@ func (x *Peer) GetPostgresConfig() *PostgresConfig { return nil } +func (x *Peer) GetEventhubConfig() *EventHubConfig { + if x, ok := x.GetConfig().(*Peer_EventhubConfig); ok { + return x.EventhubConfig + } + return nil +} + type isPeer_Config interface { isPeer_Config() } @@ -577,6 +659,10 @@ type Peer_PostgresConfig struct { PostgresConfig *PostgresConfig `protobuf:"bytes,6,opt,name=postgres_config,json=postgresConfig,proto3,oneof"` } +type Peer_EventhubConfig struct { + EventhubConfig *EventHubConfig `protobuf:"bytes,7,opt,name=eventhub_config,json=eventhubConfig,proto3,oneof"` +} + func (*Peer_SnowflakeConfig) isPeer_Config() {} func (*Peer_BigqueryConfig) isPeer_Config() {} @@ -585,6 +671,8 @@ func (*Peer_MongoConfig) isPeer_Config() {} func (*Peer_PostgresConfig) isPeer_Config() {} +func (*Peer_EventhubConfig) isPeer_Config() {} + var File_peers_proto protoreflect.FileDescriptor var file_peers_proto_rawDesc = []byte{ @@ -650,36 +738,53 @@ var file_peers_proto_rawDesc = []byte{ 0x52, 0x04, 0x75, 0x73, 0x65, 0x72, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x64, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x18, 0x05, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x64, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x22, 0xec, - 0x02, 0x0a, 0x04, 0x50, 0x65, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x28, 0x0a, 0x04, 0x74, - 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x14, 0x2e, 0x70, 0x65, 0x65, 0x72, - 0x64, 0x62, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x44, 0x42, 0x54, 0x79, 0x70, 0x65, 0x52, - 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x4a, 0x0a, 0x10, 0x73, 0x6e, 0x6f, 0x77, 0x66, 0x6c, 0x61, - 0x6b, 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x1d, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x53, - 0x6e, 0x6f, 0x77, 0x66, 0x6c, 0x61, 0x6b, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, - 0x52, 0x0f, 0x73, 0x6e, 0x6f, 0x77, 0x66, 0x6c, 0x61, 0x6b, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x12, 0x47, 0x0a, 0x0f, 0x62, 0x69, 0x67, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x63, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, - 0x72, 0x64, 0x62, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x42, 0x69, 0x67, 0x71, 0x75, 0x65, - 0x72, 0x79, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, 0x52, 0x0e, 0x62, 0x69, 0x67, 0x71, - 0x75, 0x65, 0x72, 0x79, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x3e, 0x0a, 0x0c, 0x6d, 0x6f, - 0x6e, 0x67, 0x6f, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x19, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, - 0x4d, 0x6f, 0x6e, 0x67, 0x6f, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, 0x52, 0x0b, 0x6d, - 0x6f, 0x6e, 0x67, 0x6f, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x47, 0x0a, 0x0f, 0x70, 0x6f, - 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x06, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x2e, 0x70, 0x65, 0x65, - 0x72, 0x73, 0x2e, 0x50, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x43, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x48, 0x00, 0x52, 0x0e, 0x70, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x43, 0x6f, 0x6e, - 0x66, 0x69, 0x67, 0x42, 0x08, 0x0a, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x2a, 0x3e, 0x0a, - 0x06, 0x44, 0x42, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0c, 0x0a, 0x08, 0x42, 0x49, 0x47, 0x51, 0x55, - 0x45, 0x52, 0x59, 0x10, 0x00, 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x4e, 0x4f, 0x57, 0x46, 0x4c, 0x41, - 0x4b, 0x45, 0x10, 0x01, 0x12, 0x09, 0x0a, 0x05, 0x4d, 0x4f, 0x4e, 0x47, 0x4f, 0x10, 0x02, 0x12, - 0x0c, 0x0a, 0x08, 0x50, 0x4f, 0x53, 0x54, 0x47, 0x52, 0x45, 0x53, 0x10, 0x03, 0x42, 0x12, 0x5a, - 0x10, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x64, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x73, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x64, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x22, 0xb0, + 0x01, 0x0a, 0x0e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x48, 0x75, 0x62, 0x43, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, + 0x25, 0x0a, 0x0e, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x67, 0x72, 0x6f, 0x75, + 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x1a, 0x0a, 0x08, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x12, 0x3d, 0x0a, 0x0b, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x64, + 0x62, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, + 0x2e, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x43, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x0a, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x44, + 0x62, 0x22, 0xb5, 0x03, 0x0a, 0x04, 0x50, 0x65, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x28, + 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x14, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x44, 0x42, 0x54, 0x79, + 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x4a, 0x0a, 0x10, 0x73, 0x6e, 0x6f, 0x77, + 0x66, 0x6c, 0x61, 0x6b, 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x2e, 0x70, 0x65, 0x65, 0x72, + 0x73, 0x2e, 0x53, 0x6e, 0x6f, 0x77, 0x66, 0x6c, 0x61, 0x6b, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x48, 0x00, 0x52, 0x0f, 0x73, 0x6e, 0x6f, 0x77, 0x66, 0x6c, 0x61, 0x6b, 0x65, 0x43, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x12, 0x47, 0x0a, 0x0f, 0x62, 0x69, 0x67, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, + 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x42, 0x69, 0x67, + 0x71, 0x75, 0x65, 0x72, 0x79, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, 0x52, 0x0e, 0x62, + 0x69, 0x67, 0x71, 0x75, 0x65, 0x72, 0x79, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x3e, 0x0a, + 0x0c, 0x6d, 0x6f, 0x6e, 0x67, 0x6f, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x2e, 0x70, 0x65, 0x65, + 0x72, 0x73, 0x2e, 0x4d, 0x6f, 0x6e, 0x67, 0x6f, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, + 0x52, 0x0b, 0x6d, 0x6f, 0x6e, 0x67, 0x6f, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x47, 0x0a, + 0x0f, 0x70, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x2e, + 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x43, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, 0x52, 0x0e, 0x70, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, + 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x47, 0x0a, 0x0f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x68, + 0x75, 0x62, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x45, + 0x76, 0x65, 0x6e, 0x74, 0x48, 0x75, 0x62, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, 0x52, + 0x0e, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x68, 0x75, 0x62, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x42, + 0x08, 0x0a, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x2a, 0x4c, 0x0a, 0x06, 0x44, 0x42, 0x54, + 0x79, 0x70, 0x65, 0x12, 0x0c, 0x0a, 0x08, 0x42, 0x49, 0x47, 0x51, 0x55, 0x45, 0x52, 0x59, 0x10, + 0x00, 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x4e, 0x4f, 0x57, 0x46, 0x4c, 0x41, 0x4b, 0x45, 0x10, 0x01, + 0x12, 0x09, 0x0a, 0x05, 0x4d, 0x4f, 0x4e, 0x47, 0x4f, 0x10, 0x02, 0x12, 0x0c, 0x0a, 0x08, 0x50, + 0x4f, 0x53, 0x54, 0x47, 0x52, 0x45, 0x53, 0x10, 0x03, 0x12, 0x0c, 0x0a, 0x08, 0x45, 0x56, 0x45, + 0x4e, 0x54, 0x48, 0x55, 0x42, 0x10, 0x04, 0x42, 0x12, 0x5a, 0x10, 0x67, 0x65, 0x6e, 0x65, 0x72, + 0x61, 0x74, 0x65, 0x64, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x73, 0x62, 0x06, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x33, } var ( @@ -695,26 +800,29 @@ func file_peers_proto_rawDescGZIP() []byte { } var file_peers_proto_enumTypes = make([]protoimpl.EnumInfo, 1) -var file_peers_proto_msgTypes = make([]protoimpl.MessageInfo, 5) +var file_peers_proto_msgTypes = make([]protoimpl.MessageInfo, 6) var file_peers_proto_goTypes = []interface{}{ (DBType)(0), // 0: peerdb.peers.DBType (*SnowflakeConfig)(nil), // 1: peerdb.peers.SnowflakeConfig (*BigqueryConfig)(nil), // 2: peerdb.peers.BigqueryConfig (*MongoConfig)(nil), // 3: peerdb.peers.MongoConfig (*PostgresConfig)(nil), // 4: peerdb.peers.PostgresConfig - (*Peer)(nil), // 5: peerdb.peers.Peer + (*EventHubConfig)(nil), // 5: peerdb.peers.EventHubConfig + (*Peer)(nil), // 6: peerdb.peers.Peer } var file_peers_proto_depIdxs = []int32{ - 0, // 0: peerdb.peers.Peer.type:type_name -> peerdb.peers.DBType - 1, // 1: peerdb.peers.Peer.snowflake_config:type_name -> peerdb.peers.SnowflakeConfig - 2, // 2: peerdb.peers.Peer.bigquery_config:type_name -> peerdb.peers.BigqueryConfig - 3, // 3: peerdb.peers.Peer.mongo_config:type_name -> peerdb.peers.MongoConfig - 4, // 4: peerdb.peers.Peer.postgres_config:type_name -> peerdb.peers.PostgresConfig - 5, // [5:5] is the sub-list for method output_type - 5, // [5:5] 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 + 4, // 0: peerdb.peers.EventHubConfig.metadata_db:type_name -> peerdb.peers.PostgresConfig + 0, // 1: peerdb.peers.Peer.type:type_name -> peerdb.peers.DBType + 1, // 2: peerdb.peers.Peer.snowflake_config:type_name -> peerdb.peers.SnowflakeConfig + 2, // 3: peerdb.peers.Peer.bigquery_config:type_name -> peerdb.peers.BigqueryConfig + 3, // 4: peerdb.peers.Peer.mongo_config:type_name -> peerdb.peers.MongoConfig + 4, // 5: peerdb.peers.Peer.postgres_config:type_name -> peerdb.peers.PostgresConfig + 5, // 6: peerdb.peers.Peer.eventhub_config:type_name -> peerdb.peers.EventHubConfig + 7, // [7:7] is the sub-list for method output_type + 7, // [7:7] is the sub-list for method input_type + 7, // [7:7] is the sub-list for extension type_name + 7, // [7:7] is the sub-list for extension extendee + 0, // [0:7] is the sub-list for field type_name } func init() { file_peers_proto_init() } @@ -772,6 +880,18 @@ func file_peers_proto_init() { } } file_peers_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*EventHubConfig); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_peers_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*Peer); i { case 0: return &v.state @@ -784,11 +904,12 @@ func file_peers_proto_init() { } } } - file_peers_proto_msgTypes[4].OneofWrappers = []interface{}{ + file_peers_proto_msgTypes[5].OneofWrappers = []interface{}{ (*Peer_SnowflakeConfig)(nil), (*Peer_BigqueryConfig)(nil), (*Peer_MongoConfig)(nil), (*Peer_PostgresConfig)(nil), + (*Peer_EventhubConfig)(nil), } type x struct{} out := protoimpl.TypeBuilder{ @@ -796,7 +917,7 @@ func file_peers_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_peers_proto_rawDesc, NumEnums: 1, - NumMessages: 5, + NumMessages: 6, NumExtensions: 0, NumServices: 0, }, diff --git a/flow/go.mod b/flow/go.mod index e815d6132b..6042ee0ae1 100644 --- a/flow/go.mod +++ b/flow/go.mod @@ -3,59 +3,77 @@ module github.com/PeerDB-io/peer-flow go 1.19 require ( - cloud.google.com/go/bigquery v1.51.2 - cloud.google.com/go/storage v1.30.1 - github.com/aws/aws-sdk-go v1.44.293 + cloud.google.com/go v0.110.4 + cloud.google.com/go/bigquery v1.52.0 + cloud.google.com/go/storage v1.31.0 + github.com/Azure/azure-amqp-common-go/v4 v4.2.0 + github.com/Azure/azure-event-hubs-go/v3 v3.6.0 + github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.3.0 + github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/eventhub/armeventhub v1.1.1 + github.com/aws/aws-sdk-go v1.44.298 github.com/gin-gonic/gin v1.9.1 github.com/google/uuid v1.3.0 github.com/hashicorp/go-multierror v1.1.1 - github.com/jackc/pglogrepl v0.0.0-20230428004623-0c5b98f52784 - github.com/jackc/pgx/v5 v5.3.1 + github.com/jackc/pglogrepl v0.0.0-20230630212501-5fd22a600b50 + github.com/jackc/pgx/v5 v5.4.1 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/sirupsen/logrus v1.9.3 - github.com/snowflakedb/gosnowflake v1.6.21 + github.com/snowflakedb/gosnowflake v1.6.22 github.com/stretchr/testify v1.8.4 - github.com/urfave/cli/v2 v2.25.5 + github.com/urfave/cli/v2 v2.25.7 go.temporal.io/api v1.23.0 - go.temporal.io/sdk v1.23.0 - google.golang.org/api v0.126.0 - google.golang.org/protobuf v1.30.0 + go.temporal.io/sdk v1.23.1 + google.golang.org/api v0.130.0 + google.golang.org/protobuf v1.31.0 ) require ( - cloud.google.com/go v0.110.2 // indirect - cloud.google.com/go/compute v1.20.0 // indirect + cloud.google.com/go/compute v1.20.1 // indirect cloud.google.com/go/compute/metadata v0.2.3 // indirect - cloud.google.com/go/iam v1.1.0 // indirect + cloud.google.com/go/iam v1.1.1 // 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.6.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.0.0 // indirect + github.com/Azure/go-amqp v1.0.1 // indirect + github.com/Azure/go-autorest v14.2.0+incompatible // indirect + github.com/Azure/go-autorest/autorest v0.11.29 // indirect + github.com/Azure/go-autorest/autorest/adal v0.9.23 // indirect + github.com/Azure/go-autorest/autorest/azure/auth v0.5.12 // indirect + github.com/Azure/go-autorest/autorest/date v0.3.0 // indirect + github.com/Azure/go-autorest/autorest/to v0.4.0 // indirect + 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.0.0 // 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.0 // 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.18.0 // indirect + github.com/aws/aws-sdk-go-v2 v1.18.1 // indirect github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.10 // indirect - github.com/aws/aws-sdk-go-v2/credentials v1.13.24 // indirect - github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.67 // indirect - github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.33 // indirect - github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.27 // indirect - github.com/aws/aws-sdk-go-v2/internal/v4a v1.0.25 // indirect + github.com/aws/aws-sdk-go-v2/credentials v1.13.26 // indirect + github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.71 // indirect + github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.34 // indirect + github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.28 // indirect + github.com/aws/aws-sdk-go-v2/internal/v4a v1.0.26 // indirect github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.11 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.28 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.27 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.14.2 // indirect - github.com/aws/aws-sdk-go-v2/service/s3 v1.33.1 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.29 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.28 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.14.3 // indirect + github.com/aws/aws-sdk-go-v2/service/s3 v1.36.0 // indirect github.com/aws/smithy-go v1.13.5 // indirect - github.com/bytedance/sonic v1.9.1 // indirect + github.com/bytedance/sonic v1.9.2 // indirect github.com/chenzhuoyu/base64x v0.0.0-20221115062448-fe3a3abad311 // indirect github.com/cpuguy83/go-md2man/v2 v2.0.2 // indirect github.com/danieljoos/wincred v1.2.0 // indirect github.com/davecgh/go-spew v1.1.1 // indirect + github.com/devigned/tab v0.1.1 // indirect github.com/dvsekhvalnov/jose2go v1.5.0 // indirect github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a // indirect github.com/form3tech-oss/jwt-go v3.2.5+incompatible // indirect @@ -69,6 +87,7 @@ require ( github.com/gogo/googleapis v1.4.1 // indirect github.com/gogo/protobuf v1.3.2 // indirect github.com/gogo/status v1.1.1 // indirect + github.com/golang-jwt/jwt/v4 v4.5.0 // indirect github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da // indirect github.com/golang/mock v1.6.0 // indirect github.com/golang/protobuf v1.5.3 // indirect @@ -76,8 +95,8 @@ require ( 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/googleapis/enterprise-certificate-proxy v0.2.4 // indirect - github.com/googleapis/gax-go/v2 v2.10.0 // 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 github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c // indirect github.com/hashicorp/errwrap v1.1.0 // indirect @@ -86,20 +105,23 @@ require ( github.com/jackc/pgservicefile v0.0.0-20221227161230-091c0ba34f0a // indirect github.com/jackc/puddle/v2 v2.2.0 // indirect github.com/jmespath/go-jmespath v0.4.0 // indirect + github.com/jpillora/backoff v1.0.0 // indirect github.com/json-iterator/go v1.1.12 // indirect github.com/klauspost/asmfmt v1.3.2 // indirect - github.com/klauspost/compress v1.16.5 // indirect + github.com/klauspost/compress v1.16.7 // indirect github.com/klauspost/cpuid/v2 v2.2.5 // indirect + github.com/kylelemons/godebug v1.1.0 // indirect github.com/leodido/go-urn v1.2.4 // indirect github.com/mattn/go-isatty v0.0.19 // indirect github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8 // indirect github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3 // indirect + github.com/mitchellh/mapstructure v1.5.0 // indirect github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/modern-go/reflect2 v1.0.2 // indirect github.com/mtibben/percent v0.2.1 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pelletier/go-toml/v2 v2.0.8 // indirect - github.com/pierrec/lz4/v4 v4.1.17 // indirect + github.com/pierrec/lz4/v4 v4.1.18 // indirect github.com/pkg/browser v0.0.0-20210911075715-681adbf594b8 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect @@ -111,24 +133,23 @@ require ( github.com/zeebo/xxh3 v1.0.2 // indirect go.opencensus.io v0.24.0 // indirect go.uber.org/atomic v1.11.0 // indirect - golang.org/x/arch v0.3.0 // indirect - golang.org/x/crypto v0.9.0 // indirect - golang.org/x/exp v0.0.0-20230522175609-2e198f4a06a1 // indirect - golang.org/x/mod v0.10.0 // indirect - golang.org/x/net v0.10.0 // indirect - golang.org/x/oauth2 v0.8.0 // indirect - golang.org/x/sync v0.2.0 // indirect - golang.org/x/sys v0.8.0 // indirect - golang.org/x/term v0.8.0 // indirect - golang.org/x/text v0.9.0 // indirect + golang.org/x/arch v0.4.0 // indirect + golang.org/x/crypto v0.11.0 // indirect + golang.org/x/exp v0.0.0-20230626212559-97b1e661b5df // indirect + golang.org/x/mod v0.12.0 // indirect + golang.org/x/net v0.12.0 // indirect + golang.org/x/oauth2 v0.10.0 // indirect + golang.org/x/sync v0.3.0 // indirect + golang.org/x/sys v0.10.0 // indirect + golang.org/x/term v0.10.0 // indirect + golang.org/x/text v0.11.0 // indirect golang.org/x/time v0.3.0 // indirect - golang.org/x/tools v0.9.3 // indirect + golang.org/x/tools v0.11.0 // 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-20230530153820-e85fd2cbaebc // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20230530153820-e85fd2cbaebc // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20230530153820-e85fd2cbaebc // indirect - google.golang.org/grpc v1.55.0 // indirect - gopkg.in/linkedin/goavro.v1 v1.0.5 // indirect + google.golang.org/genproto v0.0.0-20230706204954-ccb25ca9f130 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20230706204954-ccb25ca9f130 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20230706204954-ccb25ca9f130 // indirect + google.golang.org/grpc v1.56.2 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/flow/go.sum b/flow/go.sum index f1718406f4..159095bb24 100644 --- a/flow/go.sum +++ b/flow/go.sum @@ -36,8 +36,8 @@ 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.2 h1:sdFPBr6xG9/wkBbfhmUz/JmZC7X6LavQgcrVINrKiVA= -cloud.google.com/go v0.110.2/go.mod h1:k04UEeEtb6ZBRTv3dZz4CeJC3jKGxyhl0sAiVVquxiw= +cloud.google.com/go v0.110.4 h1:1JYyxKMN9hd5dR2MYTPWkGUgcoxVVhg0LKNKEo0qvmk= +cloud.google.com/go v0.110.4/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= @@ -127,8 +127,8 @@ 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.51.2 h1:p6SZQJBh64rNJB/9V5O0jvMBI8O/XV5rJKlhmmCU+2o= -cloud.google.com/go/bigquery v1.51.2/go.mod h1:6YYSJ37dAY1HyMDq/+XByPmzsC52MgzNXhxjlTzIVCM= +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/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= @@ -175,8 +175,8 @@ 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.20.0 h1:cUOcywWuowO9It2i1KX1lIb0HH7gLv6nENKuZGnlcSo= -cloud.google.com/go/compute v1.20.0/go.mod h1:kn5BhC++qUWR/AM3Dn21myV7QbgqejW04cAOrtppaQI= +cloud.google.com/go/compute v1.20.1 h1:6aKEtlUiwEpJzM001l0yFkpXmUVXaN8W+fbkb2AZNbg= +cloud.google.com/go/compute v1.20.1/go.mod h1:4tCnrn48xsqlwSAiLf1HXMQk8CONslYbdiEZc9FEIbM= cloud.google.com/go/compute/metadata v0.1.0/go.mod h1:Z1VN+bulIf6bt4P/C37K4DyZYZEXYonfTBHHFPO/4UU= cloud.google.com/go/compute/metadata v0.2.0/go.mod h1:zFmK7XCadkQkj6TtorcaGlCW1hT1fIilQDwofLpJ20k= cloud.google.com/go/compute/metadata v0.2.1/go.mod h1:jgHgmJd2RKBGzXqF5LR2EZMGxBkeanZ9wwa75XHJgOM= @@ -201,8 +201,8 @@ cloud.google.com/go/datacatalog v1.7.0/go.mod h1:9mEl4AuDYWw81UGc41HonIHH7/sn52H cloud.google.com/go/datacatalog v1.8.0/go.mod h1:KYuoVOv9BM8EYz/4eMFxrr4DUKhGIOXxZoKYF5wdISM= 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 h1:4H5IJiyUE0X6ShQBqgFFZvGGcrwGVndTwUSLP4c52gw= 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/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= @@ -316,8 +316,8 @@ 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.1.0 h1:67gSqaPukx7O8WLLHMa0PNs3EBGd2eE4d+psbO/CO94= -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/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= @@ -350,7 +350,7 @@ cloud.google.com/go/logging v1.7.0/go.mod h1:3xjP2CjkM3ZkO73aj4ASA5wRPGGCRrPIAeN 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 h1:WDKiiNXFTaQ6qz/G8FCOkuY9kJmOJGY67wPUC1M2RbE= +cloud.google.com/go/longrunning v0.5.1 h1:Fr7TXftcqTudoyRJa113hyaqlGdiBQkp0Gq7tErFDWI= 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= @@ -537,8 +537,8 @@ 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 h1:uOdMxAs8HExqBlnLtnQyP0YkvbiDpdGShGKtx6U/oNM= -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/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= @@ -611,14 +611,54 @@ github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4 h1:/vQbFIOMb github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4/go.mod h1:hN7oaIRCjzsZ2dE+yG5k+rsdt3qcwykqK6HVGcKwsw4= github.com/99designs/keyring v1.2.2 h1:pZd3neh/EmUzWONb35LxQfvuY7kiSXAq3HQd97+XBn0= 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-sdk-for-go v68.0.0+incompatible h1:fcYLmCpyNYRnvJbPerq7U0hS+6+I79yEDJBqVNcqUzU= +github.com/Azure/azure-sdk-for-go v68.0.0+incompatible/go.mod h1:9XXNKU+eRnpl9moKnB4QOLf1HestfXbmab5FXxiDBjc= github.com/Azure/azure-sdk-for-go/sdk/azcore v1.6.1 h1:SEy2xmstIphdPwNBUi7uhvjyjhVKISfwjfOJmuy7kg4= github.com/Azure/azure-sdk-for-go/sdk/azcore v1.6.1/go.mod h1:bjGvMhVMb+EEm3VRNQawDMUyMMjo+S5ewNjflkep/0Q= -github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.1.0 h1:QkAcEIAKbNL4KoFr4SathZPhDhF4mVwpBMFlYjyAqy8= +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/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= +github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/eventhub/armeventhub v1.1.1/go.mod h1:7fQVOnRA11ScLE8dOCWanXHQa2NMFOM2i0u/1VRICXA= +github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/internal v1.1.2 h1:mLY+pNLjCUeKhgnAJWAKhEUQM+RJQo2H1fuGSw1Ky1E= +github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/resources/armresources v1.0.0 h1:ECsQtyERDVz3NP3kvDOTLvbQhqWp/x9EsGKtb4ogUr8= github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.0.0 h1:u/LLAOFgsMv7HmNL4Qufg58y+qElGOt5qv0z1mURkRY= github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.0.0/go.mod h1:2e8rMJtl2+2j+HXbTBwnyGpm5Nou7KhvSfxOq8JpTag= -github.com/AzureAD/microsoft-authentication-library-for-go v0.5.1 h1:BWe8a+f/t+7KY7zH2mqygeUD0t8hNFXe08p1Pb3/jKE= +github.com/Azure/go-amqp v1.0.1 h1:Jf8OQCKzRDMZ3pCiH4onM7yrhl5curkRSGkRLTyP35o= +github.com/Azure/go-amqp v1.0.1/go.mod h1:+bg0x3ce5+Q3ahCEXnCsGG3ETpDQe3MEVnOuT2ywPwc= +github.com/Azure/go-autorest v14.2.0+incompatible h1:V5VMDjClD3GiElqLWO7mz2MxNAK/vTfRHdAubSIPRgs= +github.com/Azure/go-autorest v14.2.0+incompatible/go.mod h1:r+4oMnoxhatjLLJ6zxSWATqVooLgysK6ZNox3g/xq24= +github.com/Azure/go-autorest/autorest v0.11.24/go.mod h1:G6kyRlFnTuSbEYkQGawPfsCswgme4iYf6rfSKUDzbCc= +github.com/Azure/go-autorest/autorest v0.11.29 h1:I4+HL/JDvErx2LjyzaVxllw2lRDB5/BT2Bm4g20iqYw= +github.com/Azure/go-autorest/autorest v0.11.29/go.mod h1:ZtEzC4Jy2JDrZLxvWs8LrBWEBycl1hbT1eknI8MtfAs= +github.com/Azure/go-autorest/autorest/adal v0.9.18/go.mod h1:XVVeme+LZwABT8K5Lc3hA4nAe8LDBVle26gTrguhhPQ= +github.com/Azure/go-autorest/autorest/adal v0.9.22/go.mod h1:XuAbAEUv2Tta//+voMI038TrJBqjKam0me7qR+L8Cmk= +github.com/Azure/go-autorest/autorest/adal v0.9.23 h1:Yepx8CvFxwNKpH6ja7RZ+sKX+DWYNldbLiALMC3BTz8= +github.com/Azure/go-autorest/autorest/adal v0.9.23/go.mod h1:5pcMqFkdPhviJdlEy3kC/v1ZLnQl0MH6XA5YCcMhy4c= +github.com/Azure/go-autorest/autorest/azure/auth v0.5.12 h1:wkAZRgT/pn8HhFyzfe9UnqOjJYqlembgCTi72Bm/xKk= +github.com/Azure/go-autorest/autorest/azure/auth v0.5.12/go.mod h1:84w/uV8E37feW2NCJ08uT9VBfjfUHpgLVnG2InYD6cg= +github.com/Azure/go-autorest/autorest/azure/cli v0.4.5 h1:0W/yGmFdTIT77fvdlGZ0LMISoLHFJ7Tx4U0yeB+uFs4= +github.com/Azure/go-autorest/autorest/azure/cli v0.4.5/go.mod h1:ADQAXrkgm7acgWVUNamOgh8YNrv4p27l3Wc55oVfpzg= +github.com/Azure/go-autorest/autorest/date v0.3.0 h1:7gUk1U5M/CQbp9WoqinNzJar+8KY+LPI6wiWrP/myHw= +github.com/Azure/go-autorest/autorest/date v0.3.0/go.mod h1:BI0uouVdmngYNUzGWeSYnokU+TrmwEsOqdt8Y6sso74= +github.com/Azure/go-autorest/autorest/mocks v0.4.1/go.mod h1:LTp+uSrOhSkaKrUy935gNZuuIPPVsHlr9DSOxSayd+k= +github.com/Azure/go-autorest/autorest/mocks v0.4.2 h1:PGN4EDXnuQbojHbU0UWoNvmu9AGVwYHG9/fkDYhtAfw= +github.com/Azure/go-autorest/autorest/mocks v0.4.2/go.mod h1:Vy7OitM9Kei0i1Oj+LvyAWMXJHeKH1MVlzFugfVrmyU= +github.com/Azure/go-autorest/autorest/to v0.4.0 h1:oXVqrxakqqV1UZdSazDOPOLvOIz+XA683u8EctwboHk= +github.com/Azure/go-autorest/autorest/to v0.4.0/go.mod h1:fE8iZBn7LQR7zH/9XU2NcPR4o9jEImooCeWJcYV/zLE= +github.com/Azure/go-autorest/autorest/validation v0.3.1 h1:AgyqjAd94fwNAoTjl/WQXg4VvFeRFpO+UhNyRXqF1ac= +github.com/Azure/go-autorest/autorest/validation v0.3.1/go.mod h1:yhLgjC0Wda5DYXl6JAsWyUe4KVNffhoDhG0zVzUMo3E= +github.com/Azure/go-autorest/logger v0.2.1 h1:IG7i4p/mDa2Ce4TRyAO8IHnVhAVF3RFU+ZtXWSmf4Tg= +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/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= @@ -634,57 +674,57 @@ 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 h1:xtZE63VWl7qLdB0JObIXvvhGjoVNrQ9ciIHG2OK5cmc= -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.293 h1:oBPrQqsyMYe61Sl/xKVvQFflXjPwYH11aKi8QR3Nhts= -github.com/aws/aws-sdk-go v1.44.293/go.mod h1:aVsgQcEevwlmQ7qHE9I3h+dtQgpqhFB+i8Phjh7fkwI= -github.com/aws/aws-sdk-go-v2 v1.18.0 h1:882kkTpSFhdgYRKVZ/VCgf7sd0ru57p2JCxz4/oN5RY= -github.com/aws/aws-sdk-go-v2 v1.18.0/go.mod h1:uzbQtefpm44goOPmdKyAlXSNcwlRgF3ePWVW6EtJvvw= +github.com/aws/aws-sdk-go v1.44.298 h1:5qTxdubgV7PptZJmp/2qDwD2JL187ePL7VOxsSh1i3g= +github.com/aws/aws-sdk-go v1.44.298/go.mod h1:aVsgQcEevwlmQ7qHE9I3h+dtQgpqhFB+i8Phjh7fkwI= +github.com/aws/aws-sdk-go-v2 v1.18.1 h1:+tefE750oAb7ZQGzla6bLkOwfcQCEtC5y2RqoqCeqKo= +github.com/aws/aws-sdk-go-v2 v1.18.1/go.mod h1:uzbQtefpm44goOPmdKyAlXSNcwlRgF3ePWVW6EtJvvw= 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/config v1.18.25 h1:JuYyZcnMPBiFqn87L2cRppo+rNwgah6YwD3VuyvaW6Q= -github.com/aws/aws-sdk-go-v2/config v1.18.25/go.mod h1:dZnYpD5wTW/dQF0rRNLVypB396zWCcPiBIvdvSWHEg4= -github.com/aws/aws-sdk-go-v2/credentials v1.13.24 h1:PjiYyls3QdCrzqUN35jMWtUK1vqVZ+zLfdOa/UPFDp0= -github.com/aws/aws-sdk-go-v2/credentials v1.13.24/go.mod h1:jYPYi99wUOPIFi0rhiOvXeSEReVOzBqFNOX5bXYoG2o= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.3 h1:jJPgroehGvjrde3XufFIJUZVK5A2L9a3KwSFgKy9n8w= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.3/go.mod h1:4Q0UFP0YJf0NrsEuEYHpM9fTSEVnD16Z3uyEF7J9JGM= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.67 h1:fI9/5BDEaAv/pv1VO1X1n3jfP9it+IGqWsCuuBQI8wM= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.67/go.mod h1:zQClPRIwQZfJlZq6WZve+s4Tb4JW+3V6eS+4+KrYeP8= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.33 h1:kG5eQilShqmJbv11XL1VpyDbaEJzWxd4zRiCG30GSn4= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.33/go.mod h1:7i0PF1ME/2eUPFcjkVIwq+DOygHEoK92t5cDqNgYbIw= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.27 h1:vFQlirhuM8lLlpI7imKOMsjdQLuN9CPi+k44F/OFVsk= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.27/go.mod h1:UrHnn3QV/d0pBZ6QBAEQcqFLf8FAzLmoUfPVIueOvoM= -github.com/aws/aws-sdk-go-v2/internal/ini v1.3.34 h1:gGLG7yKaXG02/jBlg210R7VgQIotiQntNhsCFejawx8= -github.com/aws/aws-sdk-go-v2/internal/ini v1.3.34/go.mod h1:Etz2dj6UHYuw+Xw830KfzCfWGMzqvUTCjUj5b76GVDc= -github.com/aws/aws-sdk-go-v2/internal/v4a v1.0.25 h1:AzwRi5OKKwo4QNqPf7TjeO+tK8AyOK3GVSwmRPo7/Cs= -github.com/aws/aws-sdk-go-v2/internal/v4a v1.0.25/go.mod h1:SUbB4wcbSEyCvqBxv/O/IBf93RbEze7U7OnoTlpPB+g= +github.com/aws/aws-sdk-go-v2/config v1.18.27 h1:Az9uLwmssTE6OGTpsFqOnaGpLnKDqNYOJzWuC6UAYzA= +github.com/aws/aws-sdk-go-v2/config v1.18.27/go.mod h1:0My+YgmkGxeqjXZb5BYme5pc4drjTnM+x1GJ3zv42Nw= +github.com/aws/aws-sdk-go-v2/credentials v1.13.26 h1:qmU+yhKmOCyujmuPY7tf5MxR/RKyZrOPO3V4DobiTUk= +github.com/aws/aws-sdk-go-v2/credentials v1.13.26/go.mod h1:GoXt2YC8jHUBbA4jr+W3JiemnIbkXOfxSXcisUsZ3os= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.4 h1:LxK/bitrAr4lnh9LnIS6i7zWbCOdMsfzKFBI6LUCS0I= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.4/go.mod h1:E1hLXN/BL2e6YizK1zFlYd8vsfi2GTjbjBazinMmeaM= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.71 h1:SAB1UAVaf6nGCu3zyIrV+VWsendXrms1GqtW4zBotKA= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.71/go.mod h1:ZNo5H4PR3/fwsXYqb+Ld5YAfvHcYCbltaTTtSay4l2o= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.34 h1:A5UqQEmPaCFpedKouS4v+dHCTUo2sKqhoKO9U5kxyWo= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.34/go.mod h1:wZpTEecJe0Btj3IYnDx/VlUzor9wm3fJHyvLpQF0VwY= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.28 h1:srIVS45eQuewqz6fKKu6ZGXaq6FuFg5NzgQBAM6g8Y4= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.28/go.mod h1:7VRpKQQedkfIEXb4k52I7swUnZP0wohVajJMRn3vsUw= +github.com/aws/aws-sdk-go-v2/internal/ini v1.3.35 h1:LWA+3kDM8ly001vJ1X1waCuLJdtTl48gwkPKWy9sosI= +github.com/aws/aws-sdk-go-v2/internal/ini v1.3.35/go.mod h1:0Eg1YjxE0Bhn56lx+SHJwCzhW+2JGtizsrx+lCqrfm0= +github.com/aws/aws-sdk-go-v2/internal/v4a v1.0.26 h1:wscW+pnn3J1OYnanMnza5ZVYXLX4cKk5rAvUAl4Qu+c= +github.com/aws/aws-sdk-go-v2/internal/v4a v1.0.26/go.mod h1:MtYiox5gvyB+OyP0Mr0Sm/yzbEAIPL9eijj/ouHAPw0= 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/service/internal/checksum v1.1.28 h1:vGWm5vTpMr39tEZfQeDiDAMgk+5qsnvRny3FjLpnH5w= -github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.28/go.mod h1:spfrICMD6wCAhjhzHuy6DOZZ+LAIY10UxhUmLzpJTTs= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.27 h1:0iKliEXAcCa2qVtRs7Ot5hItA2MsufrphbRFlz1Owxo= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.27/go.mod h1:EOwBD4J4S5qYszS5/3DpkejfuK+Z5/1uzICfPaZLtqw= -github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.14.2 h1:NbWkRxEEIRSCqxhsHQuMiTH7yo+JZW1gp8v3elSVMTQ= -github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.14.2/go.mod h1:4tfW5l4IAB32VWCDEBxCRtR9T4BWy4I4kr1spr8NgZM= -github.com/aws/aws-sdk-go-v2/service/s3 v1.33.1 h1:O+9nAy9Bb6bJFTpeNFtd9UfHbgxO1o4ZDAM9rQp5NsY= -github.com/aws/aws-sdk-go-v2/service/s3 v1.33.1/go.mod h1:J9kLNzEiHSeGMyN7238EjJmBpCniVzFda75Gxl/NqB8= -github.com/aws/aws-sdk-go-v2/service/sso v1.12.10 h1:UBQjaMTCKwyUYwiVnUt6toEJwGXsLBI6al083tpjJzY= -github.com/aws/aws-sdk-go-v2/service/sso v1.12.10/go.mod h1:ouy2P4z6sJN70fR3ka3wD3Ro3KezSxU6eKGQI2+2fjI= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.14.10 h1:PkHIIJs8qvq0e5QybnZoG1K/9QTrLr9OsqCIo59jOBA= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.14.10/go.mod h1:AFvkxc8xfBe8XA+5St5XIHHrQQtkxqrRincx4hmMHOk= -github.com/aws/aws-sdk-go-v2/service/sts v1.19.0 h1:2DQLAKDteoEDI8zpCzqBMaZlJuoE9iTYD0gFmXVax9E= -github.com/aws/aws-sdk-go-v2/service/sts v1.19.0/go.mod h1:BgQOMsg8av8jset59jelyPW7NoZcZXLVpDsXunGDrk8= +github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.29 h1:zZSLP3v3riMOP14H7b4XP0uyfREDQOYv2cqIrvTXDNQ= +github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.29/go.mod h1:z7EjRjVwZ6pWcWdI2H64dKttvzaP99jRIj5hphW0M5U= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.28 h1:bkRyG4a929RCnpVSTvLM2j/T4ls015ZhhYApbmYs15s= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.28/go.mod h1:jj7znCIg05jXlaGBlFMGP8+7UN3VtCkRBG2spnmRQkU= +github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.14.3 h1:dBL3StFxHtpBzJJ/mNEsjXVgfO+7jR0dAIEwLqMapEA= +github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.14.3/go.mod h1:f1QyiAsvIv4B49DmCqrhlXqyaR+0IxMmyX+1P+AnzOM= +github.com/aws/aws-sdk-go-v2/service/s3 v1.36.0 h1:lEmQ1XSD9qLk+NZXbgvLJI/IiTz7OIR2TYUTFH25EI4= +github.com/aws/aws-sdk-go-v2/service/s3 v1.36.0/go.mod h1:aVbf0sko/TsLWHx30c/uVu7c62+0EAJ3vbxaJga0xCw= +github.com/aws/aws-sdk-go-v2/service/sso v1.12.12 h1:nneMBM2p79PGWBQovYO/6Xnc2ryRMw3InnDJq1FHkSY= +github.com/aws/aws-sdk-go-v2/service/sso v1.12.12/go.mod h1:HuCOxYsF21eKrerARYO6HapNeh9GBNq7fius2AcwodY= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.14.12 h1:2qTR7IFk7/0IN/adSFhYu9Xthr0zVFTgBrmPldILn80= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.14.12/go.mod h1:E4VrHCPzmVB/KFXtqBGKb3c8zpbNBgKe3fisDNLAW5w= +github.com/aws/aws-sdk-go-v2/service/sts v1.19.2 h1:XFJ2Z6sNUUcAz9poj+245DMkrHE4h2j5I9/xD50RHfE= +github.com/aws/aws-sdk-go-v2/service/sts v1.19.2/go.mod h1:dp0yLPsLBOi++WTxzCjA/oZqi6NPIhoR+uF7GeMU9eg= 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/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/boombuler/barcode v1.0.0/go.mod h1:paBWMcWSl3LHKBqUq+rly7CNSldXjb2rDl3JlRe0mD8= github.com/boombuler/barcode v1.0.1/go.mod h1:paBWMcWSl3LHKBqUq+rly7CNSldXjb2rDl3JlRe0mD8= github.com/bytedance/sonic v1.5.0/go.mod h1:ED5hyg4y6t3/9Ku1R6dU/4KyJ48DZ4jPhfY1O2AihPM= -github.com/bytedance/sonic v1.9.1 h1:6iJ6NqdoxCDr6mbY8h18oSO+cShGSMRGCEo7F2h0x8s= -github.com/bytedance/sonic v1.9.1/go.mod h1:i736AoUSYt75HyZLoJW9ERYxcy6eaN6h4BZXU064P/U= +github.com/bytedance/sonic v1.9.2 h1:GDaNjuWSGu09guE9Oql0MSTNhNCLlWwO8y/xM5BzcbM= +github.com/bytedance/sonic v1.9.2/go.mod h1:i736AoUSYt75HyZLoJW9ERYxcy6eaN6h4BZXU064P/U= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/census-instrumentation/opencensus-proto v0.3.0/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/census-instrumentation/opencensus-proto v0.4.1/go.mod h1:4T9NM4+4Vw91VeyqjLS6ao50K5bOcLKN6Q42XnYaRYw= @@ -719,7 +759,11 @@ github.com/danieljoos/wincred v1.2.0/go.mod h1:FzQLLMKBFdvu+osBrnFODiv32YGwCfx0S github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/dnaeon/go-vcr v1.1.0 h1:ReYa/UBrRyQdant9B4fNHGoCNKw6qh6P0fsdGmZpR7c= +github.com/devigned/tab v0.1.1 h1:3mD6Kb1mUOYeLpJvTVSDwSg5ZsfSxfvxGRTxRsJsITA= +github.com/devigned/tab v0.1.1/go.mod h1:XG9mPq0dFghrYvoBF3xdRrJzSTX1b7IQrvaL9mzjeJY= +github.com/dimchansky/utfbom v1.1.1 h1:vV6w1AhK4VMnhBno/TPVCoK9U/LP0PkLCS9tbxHdi/U= +github.com/dimchansky/utfbom v1.1.1/go.mod h1:SxdoEBH5qIqFocHMyGOXVAybYJdr71b1Q/j0mACtrfE= +github.com/dnaeon/go-vcr v1.2.0 h1:zHCHvJYTMh1N7xnV7zf1m1GPBF9Ad0Jk/whtQ1663qI= 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/dvsekhvalnov/jose2go v1.5.0 h1:3j8ya4Z4kMCwT5nXIKFSV84YS+HdqSSO0VsTQxaLAeM= @@ -745,6 +789,7 @@ github.com/fogleman/gg v1.2.1-0.20190220221249-0403632d5b90/go.mod h1:R/bRT+9gY/ github.com/fogleman/gg v1.3.0/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= github.com/form3tech-oss/jwt-go v3.2.5+incompatible h1:/l4kBbb4/vGSsdtB5nUe8L7B9mImVMaBPw9L/0TBHU8= github.com/form3tech-oss/jwt-go v3.2.5+incompatible/go.mod h1:pbq4aXjuKjdthFRnoDwaVPLA+WlJuPGy+QneDUgJi2k= +github.com/fortytw2/leaktest v1.3.0 h1:u8491cBMTQ8ft8aeV+adlcytMZylmA5nnwwkRZjI8vw= github.com/gabriel-vasile/mimetype v1.4.2 h1:w5qFW6JKBz9Y393Y4q372O9A7cUSequkh1Q7OhCmWKU= github.com/gabriel-vasile/mimetype v1.4.2/go.mod h1:zApsH/mKG4w07erKIaJPFiX0Tsq9BFQgN3qGY5GnNgA= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= @@ -790,7 +835,10 @@ github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= github.com/gogo/status v1.1.1 h1:DuHXlSFHNKqTQ+/ACf5Vs6r4X/dH2EgIzR9Vr+H65kg= github.com/gogo/status v1.1.1/go.mod h1:jpG3dM5QPcqu19Hg8lkUhBFBa3TcLs1DG7+2Jqci7oU= -github.com/golang-jwt/jwt v3.2.1+incompatible h1:73Z+4BJcrTC+KczS6WvTPvRGOp1WmfEP4Q1lOd9Z/+c= +github.com/golang-jwt/jwt/v4 v4.0.0/go.mod h1:/xlHOz8bRuivTWchD4jCa+NbatV+wEUSzwAxVc6locg= +github.com/golang-jwt/jwt/v4 v4.2.0/go.mod h1:/xlHOz8bRuivTWchD4jCa+NbatV+wEUSzwAxVc6locg= +github.com/golang-jwt/jwt/v4 v4.5.0 h1:7cYmW1XlMY7h7ii7UhUyChSgS5wUJEnm9uZVTGqOWzg= +github.com/golang-jwt/jwt/v4 v4.5.0/go.mod h1:m21LjoU+eqJr34lmDMbreY2eSTRJ1cv77w39/MY0Ch0= github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0/go.mod h1:E/TSTwGwJL78qG/PmXZO1EjYhfJinVAhrmmHX6Z8B9k= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= github.com/golang/glog v1.0.0/go.mod h1:EWib/APOK0SL3dFbYqvxE3UYd8E6s1ouQ7iEp/0LWV4= @@ -889,8 +937,8 @@ github.com/googleapis/enterprise-certificate-proxy v0.1.0/go.mod h1:17drOmN3MwGY github.com/googleapis/enterprise-certificate-proxy v0.2.0/go.mod h1:8C0jb7/mgJe/9KK8Lm7X9ctZC2t60YyIpYEI16jx0Qg= github.com/googleapis/enterprise-certificate-proxy v0.2.1/go.mod h1:AwSRAtLfXpU5Nm3pW+v7rGDHp09LsPtGY9MduiEsR9k= github.com/googleapis/enterprise-certificate-proxy v0.2.3/go.mod h1:AwSRAtLfXpU5Nm3pW+v7rGDHp09LsPtGY9MduiEsR9k= -github.com/googleapis/enterprise-certificate-proxy v0.2.4 h1:uGy6JWR/uMIILU8wbf+OkstIrNiMjGpEIyhx8f6W7s4= -github.com/googleapis/enterprise-certificate-proxy v0.2.4/go.mod h1:AwSRAtLfXpU5Nm3pW+v7rGDHp09LsPtGY9MduiEsR9k= +github.com/googleapis/enterprise-certificate-proxy v0.2.5 h1:UR4rDjcgpgEnqpIEvkiqTYKBCKLNmlge2eVjoZfySzM= +github.com/googleapis/enterprise-certificate-proxy v0.2.5/go.mod h1:RxW0N9901Cko1VOCW3SXCpWP+mlIEkk2tP7jnHy9a3w= github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= github.com/googleapis/gax-go/v2 v2.1.0/go.mod h1:Q3nei7sK6ybPYH7twZdmQpAd1MKb7pfu6SK+H1/DsU0= @@ -902,8 +950,8 @@ 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.10.0 h1:ebSgKfMxynOdxw8QQuFOKMgomqeLGPqNLQox2bo42zg= -github.com/googleapis/gax-go/v2 v2.10.0/go.mod h1:4UOEnMCrxsSqQ940WnTiD6qJ63le2ev3xfyagutxiPw= +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= github.com/googleapis/google-cloud-go-testing v0.0.0-20200911160855-bcd43fbb19e8/go.mod h1:dvDLG8qkwmyD9a/MJJN3XJcT3xFxOKAvTZGvuZmac9g= github.com/grpc-ecosystem/go-grpc-middleware v1.3.0/go.mod h1:z0ButlSOZa5vEBq9m2m2hlwIgKw+rp3sdCBRoJY+30Y= @@ -926,16 +974,16 @@ 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-20230428004623-0c5b98f52784 h1:qk5+0FP+HrGuqOcrQB2ISaLduvBaPnXBUoJ2FsQE9wg= -github.com/jackc/pglogrepl v0.0.0-20230428004623-0c5b98f52784/go.mod h1:P5+MSYwllwjij1PDNGA4NF6hpomKWs0CmuagKUW9s0c= +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/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.3.1 h1:Fcr8QJ1ZeLi5zsPZqQeUZhNhxfkkKBOgJuYkJHoBOtU= -github.com/jackc/pgx/v5 v5.3.1/go.mod h1:t3JDKnCBlYIc0ewLF0Q7B8MXmoIaBOZj/ic7iHozM/8= +github.com/jackc/pgx/v5 v5.4.1 h1:oKfB/FhuVtit1bBM3zNRRsZ925ZkMN3HXL+LgLUM9lE= +github.com/jackc/pgx/v5 v5.4.1/go.mod h1:q6iHT8uDNXWiFNOlRqJzBTaSH3+2xCXkokxHZC5qWFY= 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= @@ -947,6 +995,8 @@ github.com/jmoiron/sqlx v1.3.5 h1:vFFPA71p1o5gAeqtEAwLU4dnX2napprKtHr7PYIcN3g= github.com/jmoiron/sqlx v1.3.5/go.mod h1:nRVWtLre0KfCLJvgxzCsLVMogSvQ1zNJtpYr2Ccp0mQ= github.com/joho/godotenv v1.5.1 h1:7eLL/+HRGLY0ldzfGMeQkb7vMd0as4CfYvUVzLqw0N0= github.com/joho/godotenv v1.5.1/go.mod h1:f4LDr5Voq0i2e/R5DDNOoa2zzDfwtkZa6DnEwAbqwq4= +github.com/jpillora/backoff v1.0.0 h1:uvFg412JmmHBHw7iwprIxkPMI+sGQ4kzOWsMeHnm2EA= +github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnrnM= github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= @@ -959,8 +1009,8 @@ github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+o github.com/klauspost/asmfmt v1.3.2 h1:4Ri7ox3EwapiOjCki+hw14RyKk201CN4rzyCJRFLpK4= github.com/klauspost/asmfmt v1.3.2/go.mod h1:AG8TuvYojzulgDAMCnYn50l/5QV3Bs/tp6j0HLHbNSE= github.com/klauspost/compress v1.15.9/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= -github.com/klauspost/compress v1.16.5 h1:IFV2oUNUzZaz+XyusxpLzpzS8Pt5rh0Z16For/djlyI= -github.com/klauspost/compress v1.16.5/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= +github.com/klauspost/compress v1.16.7 h1:2mk3MPGNzKyxErAw8YaohYh69+pa4sIQSC0fPGCFR9I= +github.com/klauspost/compress v1.16.7/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= github.com/klauspost/cpuid/v2 v2.2.5 h1:0E5MSMDEoAulmXNFquVs//DdoomxaoTY1kUhbc/qbZg= github.com/klauspost/cpuid/v2 v2.2.5/go.mod h1:Lcz8mBdAVJIBVzewtcLocK12l3Y+JytZYpaMropDUws= @@ -976,12 +1026,12 @@ github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/kylelemons/godebug v1.1.0 h1:RPNrshWIDI6G2gRW9EHilWtl7Z6Sb1BR0xunSBf0SNc= +github.com/kylelemons/godebug v1.1.0/go.mod h1:9/0rRGxNHcop5bhtWyNeEfOS8JIWk580+fNqagV/RAw= github.com/leodido/go-urn v1.2.4 h1:XlAE/cm/ms7TE/VMVoduSpNBoyc2dOxHs5MZSwAN63Q= github.com/leodido/go-urn v1.2.4/go.mod h1:7ZrI8mTSeBSHl/UaRyKQW1qZeMgak41ANeCNaVckg+4= -github.com/lib/pq v1.2.0 h1:LXpIM/LZ5xGFhOpXAQUIMM1HdyqzVYM13zNdjCEEcA0= github.com/lib/pq v1.2.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= -github.com/linkedin/goavro v2.1.0+incompatible h1:DV2aUlj2xZiuxQyvag8Dy7zjY69ENjS66bWkSfdpddY= -github.com/linkedin/goavro v2.1.0+incompatible/go.mod h1:bBCwI2eGYpUI/4820s67MElg9tdeLbINjLjiM2xZFYM= +github.com/lib/pq v1.10.9 h1:YXG7RB+JIjhP29X+OtkiDnYaXQwpS4JEWq7dtCCRUEw= +github.com/lib/pq v1.10.9/go.mod h1:AlVN5x4E4T544tWzH6hKfbfQvm3HdbOxrmggDNAPY9o= github.com/linkedin/goavro/v2 v2.12.0 h1:rIQQSj8jdAUlKQh6DttK8wCRv4t4QO09g1C4aBWXslg= github.com/linkedin/goavro/v2 v2.12.0/go.mod h1:KXx+erlq+RPlGSPmLF7xGo6SAbh8sCQ53x064+ioxhk= github.com/lyft/protoc-gen-star v0.6.0/go.mod h1:TGAoBVkt8w7MPG72TrKIu85MIdXwDuzJYeZuUPFPNwA= @@ -998,6 +1048,10 @@ github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8 h1:AMFGa4R4MiIpsp github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8/go.mod h1:mC1jAcsrzbxHt8iiaC+zU4b1ylILSosueou12R++wfY= github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3 h1:+n/aFZefKZp7spd8DFdX7uMikMLXX4oubIzJF4kv/wI= github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3/go.mod h1:RagcQ7I8IeTMnF8JTXieKnO4Z6JCsikNEzj0DwauVzE= +github.com/mitchellh/go-homedir v1.1.0 h1:lukF9ziXFxDFPkA1vsr5zpc1XuPDn/wFntq5mG+4E0Y= +github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= +github.com/mitchellh/mapstructure v1.5.0 h1:jeMsZIYE/09sWLaz43PL7Gy6RuMjD2eJVyuac5Z2hdY= +github.com/mitchellh/mapstructure v1.5.0/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= @@ -1015,8 +1069,8 @@ github.com/phpdave11/gofpdf v1.4.2/go.mod h1:zpO6xFn9yxo3YLyMvW8HcKWVdbNqgIfOOp2 github.com/phpdave11/gofpdi v1.0.12/go.mod h1:vBmVV0Do6hSBHC8uKUQ71JGW+ZGQq74llk/7bXwjDoI= github.com/phpdave11/gofpdi v1.0.13/go.mod h1:vBmVV0Do6hSBHC8uKUQ71JGW+ZGQq74llk/7bXwjDoI= github.com/pierrec/lz4/v4 v4.1.15/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= -github.com/pierrec/lz4/v4 v4.1.17 h1:kV4Ip+/hUBC+8T6+2EgburRtkE9ef4nbY3f4dFhGjMc= -github.com/pierrec/lz4/v4 v4.1.17/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= +github.com/pierrec/lz4/v4 v4.1.18 h1:xaKrnTkyoqfh1YItXl56+6KJNVYWlEEPuAQW9xsplYQ= +github.com/pierrec/lz4/v4 v4.1.18/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pkg/browser v0.0.0-20210911075715-681adbf594b8 h1:KoWmjvw+nsYOo29YJK9vDA65RGE3NrOnUtO7a+RF9HU= github.com/pkg/browser v0.0.0-20210911075715-681adbf594b8/go.mod h1:HKlIX3XHQyzLZPlr7++PzdhaXEj94dEiJgZDTsxEqUI= github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= @@ -1044,8 +1098,8 @@ github.com/ruudk/golang-pdf417 v0.0.0-20201230142125-a7e3863a1245/go.mod h1:pQAZ github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= 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.21 h1:OEn5/P+voj3P/STW+R/gGktJlEpfP127GzrxvtAJ5II= -github.com/snowflakedb/gosnowflake v1.6.21/go.mod h1:P2fE/xiD2kQXpr48OdgnazkzPsKD6aVtnHD3WP8yD9c= +github.com/snowflakedb/gosnowflake v1.6.22 h1:2crLpqmFVyV03NPAxxAtzQBMFn6wUPqOJ1uRl4ruOJ4= +github.com/snowflakedb/gosnowflake v1.6.22/go.mod h1:P2fE/xiD2kQXpr48OdgnazkzPsKD6aVtnHD3WP8yD9c= 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= @@ -1073,8 +1127,8 @@ github.com/twitchyliquid64/golang-asm v0.15.1 h1:SU5vSMR7hnwNxj24w34ZyCi/FmDZTkS github.com/twitchyliquid64/golang-asm v0.15.1/go.mod h1:a1lVb/DtPvCB8fslRZhAngC2+aY1QWCk3Cedj/Gdt08= github.com/ugorji/go/codec v1.2.11 h1:BMaWp1Bb6fHwEtbplGBGJ498wD+LKlNSl25MjdZY4dU= github.com/ugorji/go/codec v1.2.11/go.mod h1:UNopzCgEMSXjBc6AOMqYvWC1ktqTAfzJZUZgYf6w6lg= -github.com/urfave/cli/v2 v2.25.5 h1:d0NIAyhh5shGscroL7ek/Ya9QYQE0KNabJgiUinIQkc= -github.com/urfave/cli/v2 v2.25.5/go.mod h1:GHupkWPMM0M/sj1a2b4wUrWBPzazNrIjouW6fmdJLxc= +github.com/urfave/cli/v2 v2.25.7 h1:VAzn5oq403l5pHjc4OhD54+XGO9cdKVL/7lDjF+iKUs= +github.com/urfave/cli/v2 v2.25.7/go.mod h1:8qnjx1vcq5s2/wpsqoZFndg2CE5tNFyrTvS6SinrnYQ= github.com/xrash/smetrics v0.0.0-20201216005158-039620a65673 h1:bAn7/zixMGCfxrRTfdpNzjtPYqr8smhKouy9mxVdGPU= github.com/xrash/smetrics v0.0.0-20201216005158-039620a65673/go.mod h1:N3UwUGtsrSj3ccvlPHLoLsHnpR27oXr4ZE984MbSER8= github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= @@ -1103,8 +1157,8 @@ go.opentelemetry.io/proto/otlp v0.19.0/go.mod h1:H7XAot3MsfNsj7EXtrA2q5xSNQ10UqI 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/sdk v1.23.0 h1:oa9/1f3bbcBLiNGbYf9woIx7uWFJ153q0JOkPeZqJtQ= -go.temporal.io/sdk v1.23.0/go.mod h1:S7vWxU01lGcCny0sWx03bkkYw4VtVrpzeqBTn2A6y+E= +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.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= @@ -1116,8 +1170,8 @@ go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9i go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= go.uber.org/zap v1.18.1/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= golang.org/x/arch v0.0.0-20210923205945-b76863e36670/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= -golang.org/x/arch v0.3.0 h1:02VY4/ZcO/gBOH6PUaoiptASxtXU10jazRCP865E97k= -golang.org/x/arch v0.3.0/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= +golang.org/x/arch v0.4.0 h1:A8WCeEWhLwPBKNbFi5Wv5UTCBx5zzubnXDlMOFAzFMc= +golang.org/x/arch v0.4.0/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= @@ -1127,10 +1181,13 @@ golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20210421170649-83a5a9bb288b/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20211108221036-ceb1ce70b4fa/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= +golang.org/x/crypto v0.0.0-20211215153901-e495a2d5b3d3/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= 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.9.0 h1:LF6fAI+IutBocDJ2OT0Q1g8plpYljMZ4+lty+dsqw3g= -golang.org/x/crypto v0.9.0/go.mod h1:yrmDGqONDYtNj3tH8X9dzUun2m2lzPa9ngI6/RUPGR0= +golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= +golang.org/x/crypto v0.11.0 h1:6Ewdq3tDic1mg5xRO4milcWCfMVQhI4NkqWWvqejpuA= +golang.org/x/crypto v0.11.0/go.mod h1:xgJhtzW8F9jGdVFWZESrid1U1bjeNy4zgy5cRr/CIio= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -1146,8 +1203,8 @@ 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-20230522175609-2e198f4a06a1 h1:k/i9J1pBpvlfR+9QsetwPyERsqu1GIbi967PQMq3Ivc= -golang.org/x/exp v0.0.0-20230522175609-2e198f4a06a1/go.mod h1:V1LtkGg67GoY2N1AnLN78QLrzxkLyJw7RJb1gzOOz9w= +golang.org/x/exp v0.0.0-20230626212559-97b1e661b5df h1:UA2aFVmmsIlefxMk29Dp2juaUSth8Pyn3Tq5Y5mJGME= +golang.org/x/exp v0.0.0-20230626212559-97b1e661b5df/go.mod h1:FXUEEKJgO7OQYeo8N01OfiKP8RXMtf6e8aTskBGqWdc= 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= @@ -1190,8 +1247,8 @@ 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 h1:lFO9qtOdlre5W1jxS3r/4szv2/6iXxScdzjoBMXNhYk= -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= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -1250,8 +1307,9 @@ 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.10.0 h1:X2//UzNDwYmtCLn7To6G58Wr6f5ahEAQgKNzv9Y951M= 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/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -1280,8 +1338,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.8.0 h1:6dkIjl3j3LtZ/O3sTgZTMsLKSftL/B8Zgq4huOIIUu8= -golang.org/x/oauth2 v0.8.0/go.mod h1:yr7u4HXZRm1R1kBWqr/xKNqewf0plRYoB7sla+BCIXE= +golang.org/x/oauth2 v0.10.0 h1:zHCpF2Khkwy4mMB4bv0U37YtJdTGW8jI0glAApi0Kh8= +golang.org/x/oauth2 v0.10.0/go.mod h1:kTpgurOux7LqtuxjuyZa4Gj2gdezIt/jQtGnNFfypQI= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -1298,8 +1356,8 @@ 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 h1:PUR+T4wwASmuSTYdKjYHI5TD22Wy5ogLU5qZCOLxBrI= -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= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -1381,8 +1439,9 @@ 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.8.0 h1:EBmGv8NaZBZTWvrbjNoL6HVt+IVy3QDQpJs7VRIw3tU= 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/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.1.0/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= @@ -1391,8 +1450,9 @@ 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.8.0 h1:n5xxQn2i3PC0yLAbjTpNT85q/Kgzcr2gIoX9OrJUols= 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/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -1408,8 +1468,9 @@ golang.org/x/text v0.5.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= 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 h1:2sjJmO8cDvYveuX97RDLsxlyUxLl+GHoLxBiRdHllBE= 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/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -1481,8 +1542,8 @@ 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.9.3 h1:Gn1I8+64MsuTb/HpH+LmQtNas23LhUVr3rYZ0eKuaMM= -golang.org/x/tools v0.9.3/go.mod h1:owI94Op576fPu3cIGQeHs3joujW/2Oc6MtlxbF5dfNc= +golang.org/x/tools v0.11.0 h1:EMCa6U9S2LtZXLAMoWiR/R8dAQFRqbAitmbJ2UKhoi8= +golang.org/x/tools v0.11.0/go.mod h1:anzJrxPjNtfgiYQYirP2CPGzGLxrH2u2QBhn6Bf3qY8= 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= @@ -1558,8 +1619,8 @@ 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.126.0 h1:q4GJq+cAdMAC7XP7njvQ4tvohGLiSlytuL4BQxbIZ+o= -google.golang.org/api v0.126.0/go.mod h1:mBwVAtz+87bEN6CbA1GtZPDOqY2R5ONPqJeIlvyo4Aw= +google.golang.org/api v0.130.0 h1:A50ujooa1h9iizvfzA4rrJr2B7uRmWexwbekQ2+5FPQ= +google.golang.org/api v0.130.0/go.mod h1:J/LCJMYSDFvAVREGCbrESb53n4++NMBDetSHGL5I5RY= 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= @@ -1703,14 +1764,15 @@ google.golang.org/genproto v0.0.0-20230330154414-c0448cd141ea/go.mod h1:UUQDJDOl google.golang.org/genproto v0.0.0-20230331144136-dcfb400f0633/go.mod h1:UUQDJDOlWu4KYeJZffbWgBkS1YFobzKbLVfK69pe0Ak= 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-20230530153820-e85fd2cbaebc h1:8DyZCyvI8mE1IdLy/60bS+52xfymkE72wv1asokgtao= -google.golang.org/genproto v0.0.0-20230530153820-e85fd2cbaebc/go.mod h1:xZnkP7mREFX5MORlOPEzLMr+90PPZQ2QWzrVTWfAq64= +google.golang.org/genproto v0.0.0-20230706204954-ccb25ca9f130 h1:Au6te5hbKUV8pIYWHqOUZ1pva5qK/rwbIhoXEUB9Lu8= +google.golang.org/genproto v0.0.0-20230706204954-ccb25ca9f130/go.mod h1:O9kGHb51iE/nOGvQaDUuadVYqovW56s5emA88lQnj6Y= 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-20230530153820-e85fd2cbaebc h1:kVKPf/IiYSBWEWtkIn6wZXwWGCnLKcC8oWfZvXjsGnM= -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-20230706204954-ccb25ca9f130 h1:XVeBY8d/FaK4848myy41HBqnDwvxeV3zMZhwN1TvAMU= +google.golang.org/genproto/googleapis/api v0.0.0-20230706204954-ccb25ca9f130/go.mod h1:mPBs5jNgx2GuQGvFwUvVKqtn6HsUw9nP64BedgvqEsQ= 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-20230530153820-e85fd2cbaebc h1:XSJ8Vk1SWuNr8S18z1NZSziL0CPIXLCCMDOEFtHBOFc= 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-20230706204954-ccb25ca9f130 h1:2FZP5XuJY9zQyGM5N0rtovnoXjiMUEIUMvw0m9wlpLc= +google.golang.org/genproto/googleapis/rpc v0.0.0-20230706204954-ccb25ca9f130/go.mod h1:8mL13HKkDa+IuJ8yruA3ci0q+0vsUz4m//+ottjwS5o= 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= @@ -1752,8 +1814,9 @@ google.golang.org/grpc v1.51.0/go.mod h1:wgNDFcnuBGmxLKI/qn4T+m5BtEBYXJPvibbUPsA google.golang.org/grpc v1.52.0/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 h1:3Oj82/tFSCeUrRTg/5E/7d/W5A1tj6Ky1ABAuZuv5ag= 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/cmd/protoc-gen-go-grpc v1.1.0/go.mod h1:6Kw0yEErY5E/yWrBtf03jp27GLLJujG4z/JK95pnjjw= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= @@ -1771,16 +1834,15 @@ google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQ google.golang.org/protobuf v1.28.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= google.golang.org/protobuf v1.28.1/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= google.golang.org/protobuf v1.29.1/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= -google.golang.org/protobuf v1.30.0 h1:kPPoIgf3TsEvrm0PFe15JQ+570QVxYzEvvHqChK+cng= google.golang.org/protobuf v1.30.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= +google.golang.org/protobuf v1.31.0 h1:g0LDEJHgrBl9N9r17Ru3sqWhkIx2NB67okBHPwC7hs8= +google.golang.org/protobuf v1.31.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= -gopkg.in/linkedin/goavro.v1 v1.0.5 h1:BJa69CDh0awSsLUmZ9+BowBdokpduDZSM9Zk8oKHfN4= -gopkg.in/linkedin/goavro.v1 v1.0.5/go.mod h1:Aw5GdAbizjOEl0kAMHV9iHmA8reZzW/OKuJAl4Hb9F0= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= diff --git a/flow/model/model.go b/flow/model/model.go index e8e985826a..1085f1a242 100644 --- a/flow/model/model.go +++ b/flow/model/model.go @@ -2,6 +2,7 @@ package model import ( "encoding/json" + "math/big" "time" "github.com/PeerDB-io/peer-flow/generated/protos" @@ -47,6 +48,9 @@ func (r RecordItems) ToJSON() (string, error) { if err != nil { return "", err } + case qvalue.QValueKindNumeric: + bigRat := v.Value.(*big.Rat) + jsonStruct[k] = bigRat.FloatString(9) default: jsonStruct[k] = v.Value } diff --git a/flow/model/qrecord_batch.go b/flow/model/qrecord_batch.go index 385a16f980..b988cc2402 100644 --- a/flow/model/qrecord_batch.go +++ b/flow/model/qrecord_batch.go @@ -151,7 +151,7 @@ func (src *QRecordBatchCopyFromSource) Values() ([]interface{}, error) { // TODO: account for precision and scale issues. values[i] = v.FloatString(38) - case qvalue.QValueKindBytes: + case qvalue.QValueKindBytes, qvalue.QValueKindBit: v, ok := qValue.Value.([]byte) if !ok { src.err = fmt.Errorf("invalid Bytes value") diff --git a/flow/model/qvalue/avro_converter.go b/flow/model/qvalue/avro_converter.go index 8dcaca542f..8cf31558f3 100644 --- a/flow/model/qvalue/avro_converter.go +++ b/flow/model/qvalue/avro_converter.go @@ -54,7 +54,7 @@ func GetAvroSchemaFromQValueKind(kind QValueKind, nullable bool) (*QValueKindAvr return &QValueKindAvroSchema{ AvroLogicalSchema: "boolean", }, nil - case QValueKindBytes: + case QValueKindBytes, QValueKindBit: return &QValueKindAvroSchema{ AvroLogicalSchema: "bytes", }, nil @@ -73,7 +73,7 @@ func GetAvroSchemaFromQValueKind(kind QValueKind, nullable bool) (*QValueKindAvr "type": "string", }, }, nil - case QValueKindJSON, QValueKindArray, QValueKindStruct, QValueKindBit: + case QValueKindJSON, QValueKindArray, QValueKindStruct: return nil, fmt.Errorf("complex or unsupported types: %s", kind) default: return nil, errors.New("unsupported QValueKind type") @@ -136,7 +136,7 @@ func (c *QValueAvroConverter) ToAvroValue() (interface{}, error) { return nil, fmt.Errorf("QValueKindStruct not supported") case QValueKindNumeric: return c.processNumeric() - case QValueKindBytes: + case QValueKindBytes, QValueKindBit: return c.processBytes() case QValueKindJSON: jsonString, ok := c.Value.Value.(string) diff --git a/flow/model/qvalue/qvalue.go b/flow/model/qvalue/qvalue.go index cf0b6f11bc..7371ef23cd 100644 --- a/flow/model/qvalue/qvalue.go +++ b/flow/model/qvalue/qvalue.go @@ -59,10 +59,11 @@ func (q *QValue) Equals(other *QValue) bool { } func (q *QValue) GoTimeConvert() (string, error) { - if q.Kind == QValueKindTime { + if q.Kind == QValueKindTime || q.Kind == QValueKindTimeTZ { return q.Value.(time.Time).Format("15:04:05.999999"), nil - } else if q.Kind == QValueKindTimeTZ { - return q.Value.(time.Time).Format("15:04:05.999999-0700"), nil + // no connector supports time with timezone yet + // } else if q.Kind == QValueKindTimeTZ { + // return q.Value.(time.Time).Format("15:04:05.999999-0700"), nil } else if q.Kind == QValueKindDate { return q.Value.(time.Time).Format("2006-01-02"), nil } else if q.Kind == QValueKindTimestamp { @@ -81,12 +82,20 @@ func compareInt16(value1, value2 interface{}) bool { } func compareInt32(value1, value2 interface{}) bool { + if value1 == nil && value2 == nil { + return true + } + int1, ok1 := getInt32(value1) int2, ok2 := getInt32(value2) return ok1 && ok2 && int1 == int2 } func compareInt64(value1, value2 interface{}) bool { + if value1 == nil && value2 == nil { + return true + } + int1, ok1 := getInt64(value1) int2, ok2 := getInt64(value2) return ok1 && ok2 && int1 == int2 @@ -99,12 +108,20 @@ func compareFloat32(value1, value2 interface{}) bool { } func compareFloat64(value1, value2 interface{}) bool { + if value1 == nil && value2 == nil { + return true + } + float1, ok1 := getFloat64(value1) float2, ok2 := getFloat64(value2) return ok1 && ok2 && float1 == float2 } func compareGoTime(value1, value2 interface{}) bool { + if value1 == nil && value2 == nil { + return true + } + et1, ok1 := value1.(time.Time) et2, ok2 := value2.(time.Time) @@ -121,6 +138,10 @@ func compareGoTime(value1, value2 interface{}) bool { } func compareUUID(value1, value2 interface{}) bool { + if value1 == nil && value2 == nil { + return true + } + uuid1, ok1 := getUUID(value1) uuid2, ok2 := getUUID(value2) @@ -142,6 +163,10 @@ func compareBytes(value1, value2 interface{}) bool { } func compareNumeric(value1, value2 interface{}) bool { + if value1 == nil && value2 == nil { + return true + } + rat1, ok1 := getRat(value1) rat2, ok2 := getRat(value2) @@ -155,6 +180,10 @@ func compareNumeric(value1, value2 interface{}) bool { } func compareString(value1, value2 interface{}) bool { + if value1 == nil && value2 == nil { + return true + } + str1, ok1 := value1.(string) str2, ok2 := value2.(string) @@ -311,6 +340,9 @@ func getBytes(v interface{}) ([]byte, bool) { return value, true case string: return []byte(value), true + case nil: + // return empty byte array + return []byte{}, true default: return nil, false } @@ -346,6 +378,42 @@ func getRat(v interface{}) (*big.Rat, bool) { if ok { return parsed, true } + case float64: + rat := new(big.Rat) + return rat.SetFloat64(value), true + case int64: + rat := new(big.Rat) + return rat.SetInt64(value), true + case uint64: + rat := new(big.Rat) + return rat.SetUint64(value), true + case float32: + rat := new(big.Rat) + return rat.SetFloat64(float64(value)), true + case int32: + rat := new(big.Rat) + return rat.SetInt64(int64(value)), true + case uint32: + rat := new(big.Rat) + return rat.SetUint64(uint64(value)), true + case int: + rat := new(big.Rat) + return rat.SetInt64(int64(value)), true + case uint: + rat := new(big.Rat) + return rat.SetUint64(uint64(value)), true + case int8: + rat := new(big.Rat) + return rat.SetInt64(int64(value)), true + case uint8: + rat := new(big.Rat) + return rat.SetUint64(uint64(value)), true + case int16: + rat := new(big.Rat) + return rat.SetInt64(int64(value)), true + case uint16: + rat := new(big.Rat) + return rat.SetUint64(uint64(value)), true } return nil, false } diff --git a/flow/workflows/peer_flow.go b/flow/workflows/peer_flow.go index c6dc2eeb0e..39b9df3fc9 100644 --- a/flow/workflows/peer_flow.go +++ b/flow/workflows/peer_flow.go @@ -17,10 +17,26 @@ import ( ) const ( - PeerFlowStatusQuery = "q-peer-flow-status" + PeerFlowStatusQuery = "q-peer-flow-status" + maxSyncFlowsPerPeerFlow = 32 + maxNormalizeFlowsPerPeerFlow = 32 ) +type PeerFlowLimits struct { + // Number of sync flows to execute in total. + // If 0, the number of sync flows will be continuously executed until the peer flow is cancelled. + // This is typically non-zero for testing purposes. + TotalSyncFlows int + // Number of normalize flows to execute in total. + // If 0, the number of sync flows will be continuously executed until the peer flow is cancelled. + // This is typically non-zero for testing purposes. + TotalNormalizeFlows int + // Maximum number of rows in a sync flow batch. + MaxBatchSize int +} + type PeerFlowWorkflowInput struct { + PeerFlowLimits // The JDBC URL for the catalog database. CatalogJdbcURL string // The name of the peer flow to execute. @@ -38,8 +54,6 @@ type PeerFlowWorkflowInput struct { } type PeerFlowState struct { - // Input for the PeerFlowWorkflow. - PeerFlowWorkflowInput // Progress events for the peer flow. Progress []string // Accumulates status for sync flows spawned. @@ -56,36 +70,48 @@ type PeerFlowState struct { NormalizeFlowErrors error } +// returns a new empty PeerFlowState +func NewStartedPeerFlowState() *PeerFlowState { + return &PeerFlowState{ + Progress: []string{"started"}, + SyncFlowStatuses: nil, + NormalizeFlowStatuses: nil, + ActiveSignal: shared.NoopSignal, + SetupComplete: false, + SyncFlowErrors: nil, + NormalizeFlowErrors: nil, + } +} + // PeerFlowWorkflowExecution represents the state for execution of a peer flow. type PeerFlowWorkflowExecution struct { - // The state of the peer flow. - PeerFlowState flowExecutionID string logger log.Logger } // NewPeerFlowWorkflowExecution creates a new instance of PeerFlowWorkflowExecution. -func NewPeerFlowWorkflowExecution(ctx workflow.Context, state *PeerFlowState) *PeerFlowWorkflowExecution { +func NewPeerFlowWorkflowExecution(ctx workflow.Context) *PeerFlowWorkflowExecution { return &PeerFlowWorkflowExecution{ - PeerFlowState: *state, flowExecutionID: workflow.GetInfo(ctx).WorkflowExecution.ID, logger: workflow.GetLogger(ctx), } } // fetchConnectionConfigs fetches the connection configs for source and destination peers. -func (w *PeerFlowWorkflowExecution) fetchConnectionConfigs( +func fetchConnectionConfigs( ctx workflow.Context, + logger log.Logger, + input *PeerFlowWorkflowInput, ) (*protos.FlowConnectionConfigs, error) { - w.logger.Info("fetching connection configs for peer flow - ", w.PeerFlowName) + logger.Info("fetching connection configs for peer flow - ", input.PeerFlowName) ctx = workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ StartToCloseTimeout: 1 * time.Minute, }) fetchConfigActivityInput := &activities.FetchConfigActivityInput{ - CatalogJdbcURL: w.CatalogJdbcURL, - PeerFlowName: w.PeerFlowName, + CatalogJdbcURL: input.CatalogJdbcURL, + PeerFlowName: input.PeerFlowName, } configsFuture := workflow.ExecuteActivity(ctx, fetchConfig.FetchConfig, fetchConfigActivityInput) @@ -101,12 +127,11 @@ func (w *PeerFlowWorkflowExecution) fetchConnectionConfigs( return nil, fmt.Errorf("invalid connection configs") } - w.logger.Info("fetched connection configs for peer flow - ", w.PeerFlowName) + logger.Info("fetched connection configs for peer flow - ", input.PeerFlowName) return flowConnectionConfigs, nil } -// getChildWorkflowID returns the child workflow ID for a new sync flow. -func (w *PeerFlowWorkflowExecution) getChildWorkflowID( +func GetChildWorkflowID( ctx workflow.Context, prefix string, peerFlowName string, @@ -129,100 +154,140 @@ type PeerFlowWorkflowResult = PeerFlowState // PeerFlowWorkflow is the workflow that executes the specified peer flow. // This is the main entry point for the application. func PeerFlowWorkflow(ctx workflow.Context, input *PeerFlowWorkflowInput) (*PeerFlowWorkflowResult, error) { - w := NewPeerFlowWorkflowExecution(ctx, &PeerFlowState{ - PeerFlowWorkflowInput: *input, - Progress: []string{"started"}, - SyncFlowStatuses: []*model.SyncResponse{}, - NormalizeFlowStatuses: []*model.NormalizeResponse{}, - SetupComplete: false, - ActiveSignal: shared.NoopSignal, - SyncFlowErrors: nil, - }) + fconn, err := fetchConnectionConfigs(ctx, workflow.GetLogger(ctx), input) + if err != nil { + return nil, err + } + + fconn.FlowJobName = input.PeerFlowName + + peerflowWithConfigID, err := GetChildWorkflowID(ctx, "peer-flow-with-config", input.PeerFlowName) + if err != nil { + return nil, err + } + + peerflowWithConfigOpts := workflow.ChildWorkflowOptions{ + WorkflowID: peerflowWithConfigID, + ParentClosePolicy: enums.PARENT_CLOSE_POLICY_REQUEST_CANCEL, + RetryPolicy: &temporal.RetryPolicy{ + MaximumAttempts: 2, + }, + } + + limits := &PeerFlowLimits{ + TotalSyncFlows: input.TotalSyncFlows, + TotalNormalizeFlows: input.TotalNormalizeFlows, + MaxBatchSize: input.MaxBatchSize, + } + + state := NewStartedPeerFlowState() + peerflowWithConfigCtx := workflow.WithChildOptions(ctx, peerflowWithConfigOpts) + peerFlowWithConfigFuture := workflow.ExecuteChildWorkflow( + peerflowWithConfigCtx, PeerFlowWorkflowWithConfig, fconn, &limits, state) + + var res PeerFlowWorkflowResult + if err := peerFlowWithConfigFuture.Get(peerflowWithConfigCtx, &res); err != nil { + return nil, fmt.Errorf("failed to execute child workflow: %w", err) + } + + return &res, nil +} + +func PeerFlowWorkflowWithConfig( + ctx workflow.Context, + cfg *protos.FlowConnectionConfigs, + limits *PeerFlowLimits, + state *PeerFlowState, +) (*PeerFlowWorkflowResult, error) { + if state == nil { + state = NewStartedPeerFlowState() + } + + if cfg == nil { + return nil, fmt.Errorf("invalid connection configs") + } + + w := NewPeerFlowWorkflowExecution(ctx) + + if limits.TotalNormalizeFlows == 0 { + limits.TotalNormalizeFlows = maxNormalizeFlowsPerPeerFlow + } + + if limits.TotalSyncFlows == 0 { + limits.TotalSyncFlows = maxSyncFlowsPerPeerFlow + } // Support a Query for the current state of the peer flow. - err := workflow.SetQueryHandler(ctx, PeerFlowStatusQuery, func() (PeerFlowState, error) { - return w.PeerFlowState, nil + err := workflow.SetQueryHandler(ctx, PeerFlowStatusQuery, func(jobName string) (PeerFlowState, error) { + return *state, nil }) if err != nil { - return &w.PeerFlowState, fmt.Errorf("failed to set `%s` query handler: %w", PeerFlowStatusQuery, err) + return state, fmt.Errorf("failed to set `%s` query handler: %w", PeerFlowStatusQuery, err) } - selector := workflow.NewSelector(ctx) - signalChan := workflow.GetSignalChannel(ctx, shared.PeerFlowSignalName) signalHandler := func(_ workflow.Context, v shared.PeerFlowSignal) { w.logger.Info("received signal - ", v) - w.PeerFlowState.ActiveSignal = v + state.ActiveSignal = v } // Support a signal to pause the peer flow. + selector := workflow.NewSelector(ctx) selector.AddReceive(signalChan, func(c workflow.ReceiveChannel, more bool) { var signalVal shared.PeerFlowSignal c.Receive(ctx, &signalVal) signalHandler(ctx, signalVal) }) - // Fetch the connection configs for the source and destination peers. - flowConnectionConfigs, err := w.fetchConnectionConfigs(ctx) - if err != nil { - return &w.PeerFlowState, err - } - flowConnectionConfigs.FlowJobName = w.PeerFlowName - w.Progress = append(w.Progress, "fetched connection configs") - - { + if !state.SetupComplete { // start the SetupFlow workflow as a child workflow, and wait for it to complete // it should return the table schema for the source peer - setupFlowID, err := w.getChildWorkflowID(ctx, "setup-flow", w.PeerFlowName) + setupFlowID, err := GetChildWorkflowID(ctx, "setup-flow", cfg.FlowJobName) if err != nil { - return &w.PeerFlowState, err + return state, err } childSetupFlowOpts := workflow.ChildWorkflowOptions{ WorkflowID: setupFlowID, ParentClosePolicy: enums.PARENT_CLOSE_POLICY_REQUEST_CANCEL, RetryPolicy: &temporal.RetryPolicy{ - MaximumAttempts: 10, + MaximumAttempts: 2, }, } setupFlowCtx := workflow.WithChildOptions(ctx, childSetupFlowOpts) - setupFlowFuture := workflow.ExecuteChildWorkflow(setupFlowCtx, SetupFlowWorkflow, flowConnectionConfigs) - if err := setupFlowFuture.Get(setupFlowCtx, &flowConnectionConfigs); err != nil { - return &w.PeerFlowState, fmt.Errorf("failed to execute child workflow: %w", err) + setupFlowFuture := workflow.ExecuteChildWorkflow(setupFlowCtx, SetupFlowWorkflow, cfg) + if err := setupFlowFuture.Get(setupFlowCtx, &cfg); err != nil { + return state, fmt.Errorf("failed to execute child workflow: %w", err) } - w.SetupComplete = true - w.Progress = append(w.Progress, "executed setup flow") + state.SetupComplete = true + state.Progress = append(state.Progress, "executed setup flow") } syncFlowOptions := &protos.SyncFlowOptions{ - BatchSize: int32(input.MaxBatchSize), + BatchSize: int32(limits.MaxBatchSize), } - currentFlowNumber := 0 - currentNormalizeNumber := 0 + currentSyncFlowNum := 0 + currentNormalizeFlowNum := 0 for { // check if the peer flow has been shutdown - if w.PeerFlowState.ActiveSignal == shared.ShutdownSignal { + if state.ActiveSignal == shared.ShutdownSignal { w.logger.Info("peer flow has been shutdown") break } - /* - SyncFlow - sync raw cdc changes from source to target. - SyncFlow will always be running, even when Initial Load is going on. - */ // check if total sync flows have been completed - if input.TotalSyncFlows != 0 && currentFlowNumber == input.TotalSyncFlows { + if limits.TotalSyncFlows != 0 && currentSyncFlowNum == limits.TotalSyncFlows { w.logger.Info("All the syncflows have completed successfully, there was a"+ - " limit on the number of syncflows to be executed: ", input.TotalSyncFlows) - break + " limit on the number of syncflows to be executed: ", limits.TotalSyncFlows) + return state, nil } - currentFlowNumber++ + currentSyncFlowNum++ - syncFlowID, err := w.getChildWorkflowID(ctx, "sync-flow", w.PeerFlowName) + syncFlowID, err := GetChildWorkflowID(ctx, "sync-flow", cfg.FlowJobName) if err != nil { - return &w.PeerFlowState, err + return state, err } // execute the sync flow as a child workflow @@ -230,14 +295,14 @@ func PeerFlowWorkflow(ctx workflow.Context, input *PeerFlowWorkflowInput) (*Peer WorkflowID: syncFlowID, ParentClosePolicy: enums.PARENT_CLOSE_POLICY_REQUEST_CANCEL, RetryPolicy: &temporal.RetryPolicy{ - MaximumAttempts: 10, + MaximumAttempts: 2, }, } ctx = workflow.WithChildOptions(ctx, childSyncFlowOpts) childSyncFlowFuture := workflow.ExecuteChildWorkflow( ctx, SyncFlowWorkflow, - flowConnectionConfigs, + cfg, syncFlowOptions, ) @@ -245,9 +310,9 @@ func PeerFlowWorkflow(ctx workflow.Context, input *PeerFlowWorkflowInput) (*Peer var childSyncFlowRes *model.SyncResponse if err := f.Get(ctx, &childSyncFlowRes); err != nil { w.logger.Error("failed to execute sync flow: ", err) - w.SyncFlowErrors = multierror.Append(w.SyncFlowErrors, err) + state.SyncFlowErrors = multierror.Append(state.SyncFlowErrors, err) } else { - w.SyncFlowStatuses = append(w.SyncFlowStatuses, childSyncFlowRes) + state.SyncFlowStatuses = append(state.SyncFlowStatuses, childSyncFlowRes) } }) selector.Select(ctx) @@ -259,16 +324,16 @@ func PeerFlowWorkflow(ctx workflow.Context, input *PeerFlowWorkflowInput) (*Peer 1. Currently NormalizeFlow runs right after SyncFlow. We need to make it asynchronous NormalizeFlow will start only after Initial Load */ - if input.TotalNormalizeFlows != 0 && currentNormalizeNumber == input.TotalNormalizeFlows { + if limits.TotalNormalizeFlows != 0 && currentNormalizeFlowNum == limits.TotalNormalizeFlows { w.logger.Info("All the normalizer flows have completed successfully, there was a"+ - " limit on the number of normalizer to be executed: ", input.TotalNormalizeFlows) - break + " limit on the number of normalizer to be executed: ", limits.TotalNormalizeFlows) + return state, nil } - currentNormalizeNumber++ + currentNormalizeFlowNum++ - normalizeFlowID, err := w.getChildWorkflowID(ctx, "normalize-flow", w.PeerFlowName) + normalizeFlowID, err := GetChildWorkflowID(ctx, "normalize-flow", cfg.FlowJobName) if err != nil { - return &w.PeerFlowState, err + return state, err } // execute the normalize flow as a child workflow @@ -276,27 +341,27 @@ func PeerFlowWorkflow(ctx workflow.Context, input *PeerFlowWorkflowInput) (*Peer WorkflowID: normalizeFlowID, ParentClosePolicy: enums.PARENT_CLOSE_POLICY_REQUEST_CANCEL, RetryPolicy: &temporal.RetryPolicy{ - MaximumAttempts: 10, + MaximumAttempts: 2, }, } ctx = workflow.WithChildOptions(ctx, childNormalizeFlowOpts) childNormalizeFlowFuture := workflow.ExecuteChildWorkflow( ctx, NormalizeFlowWorkflow, - flowConnectionConfigs, + 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) - w.SyncFlowErrors = multierror.Append(w.SyncFlowErrors, err) + state.NormalizeFlowErrors = multierror.Append(state.NormalizeFlowErrors, err) } else { - w.NormalizeFlowStatuses = append(w.NormalizeFlowStatuses, childNormalizeFlowRes) + state.NormalizeFlowStatuses = append(state.NormalizeFlowStatuses, childNormalizeFlowRes) } }) selector.Select(ctx) } - return &w.PeerFlowState, w.SyncFlowErrors + return nil, workflow.NewContinueAsNewError(ctx, PeerFlowWorkflowWithConfig, cfg, limits, state) } diff --git a/flow/workflows/setup_flow.go b/flow/workflows/setup_flow.go index 0131bde450..063539e098 100644 --- a/flow/workflows/setup_flow.go +++ b/flow/workflows/setup_flow.go @@ -171,6 +171,7 @@ func (s *SetupFlowExecution) createRawTable( createRawTblInput := &protos.CreateRawTableInput{ PeerConnectionConfig: config.Destination, FlowJobName: s.PeerFlowName, + TableNameMapping: config.TableNameMapping, } rawTblFuture := workflow.ExecuteActivity(ctx, flowable.CreateRawTable, createRawTblInput) diff --git a/flow/workflows/sync_flow.go b/flow/workflows/sync_flow.go index daf9223d3a..db31da57b7 100644 --- a/flow/workflows/sync_flow.go +++ b/flow/workflows/sync_flow.go @@ -79,11 +79,11 @@ func (s *SyncFlowExecution) executeSyncFlow( s.logger.Info("no last synced ID from destination peer") } - syncFlowCtx := workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ - StartToCloseTimeout: 15 * time.Minute, + startFlowCtx := workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ + StartToCloseTimeout: 6 * time.Hour, // TODO: activity needs to call heartbeat. // see https://github.com/PeerDB-io/nexus/issues/216 - HeartbeatTimeout: 1 * time.Minute, + HeartbeatTimeout: 5 * time.Minute, }) // execute StartFlow on the peers to start the flow @@ -92,10 +92,10 @@ func (s *SyncFlowExecution) executeSyncFlow( LastSyncState: dstSyncState, SyncFlowOptions: opts, } - fStartFlow := workflow.ExecuteActivity(syncFlowCtx, flowable.StartFlow, startFlowInput) + fStartFlow := workflow.ExecuteActivity(startFlowCtx, flowable.StartFlow, startFlowInput) var syncRes *model.SyncResponse - if err := fStartFlow.Get(syncFlowCtx, &syncRes); err != nil { + if err := fStartFlow.Get(startFlowCtx, &syncRes); err != nil { return nil, fmt.Errorf("failed to flow: %w", err) } diff --git a/nexus/Cargo.lock b/nexus/Cargo.lock index 7e78f44185..08eb0d872d 100644 --- a/nexus/Cargo.lock +++ b/nexus/Cargo.lock @@ -17,6 +17,17 @@ version = "1.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f26201604c87b1e01bd3d98f8d5d9a8fcbb815e8cedb41ffccbeb4bf593a35fe" +[[package]] +name = "ahash" +version = "0.7.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fcb51a0695d8f838b1ee009b3fbf66bda078cd64590202a864a8f3e8c4315c47" +dependencies = [ + "getrandom", + "once_cell", + "version_check", +] + [[package]] name = "aho-corasick" version = "0.7.20" @@ -123,6 +134,12 @@ version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3d62b7694a562cdf5a74227903507c56ab2cc8bdd1f781ed5cb4cf9c9f810bfc" +[[package]] +name = "arrayvec" +version = "0.7.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" + [[package]] name = "async-compression" version = "0.3.15" @@ -303,6 +320,18 @@ version = "1.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" +[[package]] +name = "bitvec" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1bc2832c24239b0141d5674bb9174f9d68a8b5b3f2753311927c172ca46f7e9c" +dependencies = [ + "funty", + "radium", + "tap", + "wyz", +] + [[package]] name = "block-buffer" version = "0.10.4" @@ -312,12 +341,79 @@ dependencies = [ "generic-array", ] +[[package]] +name = "borsh" +version = "0.10.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4114279215a005bc675e386011e594e1d9b800918cea18fcadadcce864a2046b" +dependencies = [ + "borsh-derive", + "hashbrown", +] + +[[package]] +name = "borsh-derive" +version = "0.10.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0754613691538d51f329cce9af41d7b7ca150bc973056f1156611489475f54f7" +dependencies = [ + "borsh-derive-internal", + "borsh-schema-derive-internal", + "proc-macro-crate", + "proc-macro2", + "syn 1.0.109", +] + +[[package]] +name = "borsh-derive-internal" +version = "0.10.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "afb438156919598d2c7bad7e1c0adf3d26ed3840dbc010db1a882a65583ca2fb" +dependencies = [ + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "borsh-schema-derive-internal" +version = "0.10.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "634205cc43f74a1b9046ef87c4540ebda95696ec0f315024860cad7c5b0f5ccd" +dependencies = [ + "proc-macro2", + "quote", + "syn 1.0.109", +] + [[package]] name = "bumpalo" version = "3.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0d261e256854913907f67ed06efbc3338dfe6179796deefc1ff763fc1aee5535" +[[package]] +name = "bytecheck" +version = "0.6.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8b6372023ac861f6e6dc89c8344a8f398fb42aaba2b5dbc649ca0c0e9dbcb627" +dependencies = [ + "bytecheck_derive", + "ptr_meta", + "simdutf8", +] + +[[package]] +name = "bytecheck_derive" +version = "0.6.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a7ec4c6f261935ad534c0c22dbef2201b45918860eb1c574b972bd213a76af61" +dependencies = [ + "proc-macro2", + "quote", + "syn 1.0.109", +] + [[package]] name = "byteorder" version = "1.4.3" @@ -925,6 +1021,12 @@ dependencies = [ "percent-encoding", ] +[[package]] +name = "funty" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e6d5a32815ae3f33302d95fdcb2ce17862f8c65363dcfd29360480ba1001fc9c" + [[package]] name = "futures" version = "0.3.28" @@ -1113,6 +1215,9 @@ name = "hashbrown" version = "0.12.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" +dependencies = [ + "ahash", +] [[package]] name = "hdrhistogram" @@ -1790,13 +1895,14 @@ dependencies = [ "pgerror", "pgwire", "pt", + "rust_decimal", "serde", "serde_bytes", "serde_json", "sqlparser", "tokio", "tracing", - "uuid", + "uuid 0.8.2", "value", "yup-oauth2", ] @@ -1811,7 +1917,7 @@ dependencies = [ "tokio", "tokio-postgres", "tracing", - "uuid", + "uuid 0.8.2", ] [[package]] @@ -1842,6 +1948,7 @@ dependencies = [ "pgerror", "pgwire", "pt", + "rust_decimal", "serde", "serde_bytes", "serde_json", @@ -1850,7 +1957,7 @@ dependencies = [ "tokio-postgres", "tracing", "urlencoding", - "uuid", + "uuid 0.8.2", "value", ] @@ -1939,7 +2046,7 @@ dependencies = [ "tracing", "tracing-appender", "tracing-subscriber", - "uuid", + "uuid 0.8.2", ] [[package]] @@ -2144,7 +2251,7 @@ dependencies = [ "postgres-protocol", "serde", "serde_json", - "uuid", + "uuid 0.8.2", ] [[package]] @@ -2175,6 +2282,15 @@ dependencies = [ "syn 1.0.109", ] +[[package]] +name = "proc-macro-crate" +version = "0.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1d6ea3c4595b96363c13943497db34af4460fb474a95c43f4446ad341b8c9785" +dependencies = [ + "toml 0.5.11", +] + [[package]] name = "proc-macro-error" version = "1.0.4" @@ -2273,6 +2389,26 @@ dependencies = [ "sqlparser", ] +[[package]] +name = "ptr_meta" +version = "0.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0738ccf7ea06b608c10564b31debd4f5bc5e197fc8bfe088f68ae5ce81e7a4f1" +dependencies = [ + "ptr_meta_derive", +] + +[[package]] +name = "ptr_meta_derive" +version = "0.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "16b845dbfca988fa33db069c0e230574d15a3088f147a87b64c7589eb662c9ac" +dependencies = [ + "proc-macro2", + "quote", + "syn 1.0.109", +] + [[package]] name = "quick-error" version = "2.0.1" @@ -2288,6 +2424,12 @@ dependencies = [ "proc-macro2", ] +[[package]] +name = "radium" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dc33ff2d4973d518d823d61aa239014831e521c75da58e3df4840d3f47749d09" + [[package]] name = "rand" version = "0.8.5" @@ -2429,6 +2571,15 @@ version = "0.6.29" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f162c6dd7b008981e4d40210aca20b4bd0f9b60ca9271061b07f78537722f2e1" +[[package]] +name = "rend" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "581008d2099240d37fb08d77ad713bcaec2c4d89d50b5b21a8bb1996bbab68ab" +dependencies = [ + "bytecheck", +] + [[package]] name = "reqwest" version = "0.11.16" @@ -2494,6 +2645,34 @@ dependencies = [ "winapi", ] +[[package]] +name = "rkyv" +version = "0.7.42" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0200c8230b013893c0b2d6213d6ec64ed2b9be2e0e016682b7224ff82cff5c58" +dependencies = [ + "bitvec", + "bytecheck", + "hashbrown", + "ptr_meta", + "rend", + "rkyv_derive", + "seahash", + "tinyvec", + "uuid 1.4.0", +] + +[[package]] +name = "rkyv_derive" +version = "0.7.42" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b2e06b915b5c230a17d7a736d1e2e63ee753c256a8614ef3f5147b13a4f5541d" +dependencies = [ + "proc-macro2", + "quote", + "syn 1.0.109", +] + [[package]] name = "rsa" version = "0.6.1" @@ -2514,6 +2693,26 @@ dependencies = [ "zeroize", ] +[[package]] +name = "rust_decimal" +version = "1.30.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d0446843641c69436765a35a5a77088e28c2e6a12da93e84aa3ab1cd4aa5a042" +dependencies = [ + "arrayvec", + "borsh", + "bytecheck", + "byteorder", + "bytes", + "num-traits", + "postgres", + "rand", + "rkyv", + "serde", + "serde_json", + "tokio-postgres", +] + [[package]] name = "rustc-demangle" version = "0.1.23" @@ -2807,6 +3006,12 @@ version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5e1788eed21689f9cf370582dfc467ef36ed9c707f073528ddafa8d83e3b8500" +[[package]] +name = "simdutf8" +version = "0.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f27f6278552951f1f2b8cf9da965d10969b2efdea95a6ec47987ab46edfe263a" + [[package]] name = "simple_asn1" version = "0.6.2" @@ -2995,6 +3200,12 @@ version = "0.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2047c6ded9c721764247e62cd3b03c09ffc529b2ba5b10ec482ae507a4a70160" +[[package]] +name = "tap" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "55937e1799185b12863d447f42597ed69d9928686b8d88a1df17376a097d8369" + [[package]] name = "tar" version = "0.4.38" @@ -3527,6 +3738,12 @@ dependencies = [ "serde", ] +[[package]] +name = "uuid" +version = "1.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d023da39d1fde5a8a3fe1f3e01ca9632ada0a63e9797de55a879d6e2236277be" + [[package]] name = "valuable" version = "0.1.0" @@ -3544,9 +3761,10 @@ dependencies = [ "pgwire", "postgres", "postgres-types", + "rust_decimal", "serde", "serde_json", - "uuid", + "uuid 0.8.2", ] [[package]] @@ -3904,6 +4122,15 @@ dependencies = [ "winapi", ] +[[package]] +name = "wyz" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "05f360fc0b24296329c78fda852a1e9ae82de9cf7b27dae4b7f62f118f77b9ed" +dependencies = [ + "tap", +] + [[package]] name = "x509-certificate" version = "0.20.0" diff --git a/nexus/analyzer/src/lib.rs b/nexus/analyzer/src/lib.rs index 3cb13e3f04..4490e9943a 100644 --- a/nexus/analyzer/src/lib.rs +++ b/nexus/analyzer/src/lib.rs @@ -391,6 +391,7 @@ fn parse_db_options( let config = Config::PostgresConfig(postgres_config); Some(config) } + DbType::Eventhub => None, }; Ok(config) diff --git a/nexus/analyzer/src/qrep.rs b/nexus/analyzer/src/qrep.rs index 46653cb9a0..5f9aebdf5e 100644 --- a/nexus/analyzer/src/qrep.rs +++ b/nexus/analyzer/src/qrep.rs @@ -45,7 +45,7 @@ lazy_static::lazy_static! { QRepOptionType::String { name: "mode", default_val: Some("append"), - required: false, + required: true, accepted_values: Some(vec!["upsert", "append"]), }, QRepOptionType::StringArray { diff --git a/nexus/catalog/src/lib.rs b/nexus/catalog/src/lib.rs index e256cd2533..d0c6687054 100644 --- a/nexus/catalog/src/lib.rs +++ b/nexus/catalog/src/lib.rs @@ -144,6 +144,11 @@ impl Catalog { buf.reserve(config_len); postgres_config.encode(&mut buf)?; } + Config::EventhubConfig(eventhub_config) => { + let config_len = eventhub_config.encoded_len(); + buf.reserve(config_len); + eventhub_config.encode(&mut buf)?; + } }; buf @@ -233,6 +238,12 @@ impl Catalog { pt::peers::MongoConfig::decode(options.as_slice()).context(err)?; Some(Config::MongoConfig(mongo_config)) } + Some(DbType::Eventhub) => { + let err = format!("unable to decode {} options for peer {}", "eventhub", name); + let eventhub_config = + pt::peers::EventHubConfig::decode(options.as_slice()).context(err)?; + Some(Config::EventhubConfig(eventhub_config)) + } Some(DbType::Postgres) => { let err = format!("unable to decode {} options for peer {}", "postgres", name); let postgres_config = diff --git a/nexus/peer-bigquery/Cargo.toml b/nexus/peer-bigquery/Cargo.toml index 7d8147f5c1..f41d497d9e 100644 --- a/nexus/peer-bigquery/Cargo.toml +++ b/nexus/peer-bigquery/Cargo.toml @@ -16,6 +16,7 @@ peer-connections = { path = "../peer-connections" } pgerror = { path = "../pgerror" } pgwire = "0.15" pt = { path = "../pt" } +rust_decimal = { version = "1.30.0", features = [ "tokio-pg" ] } serde = { version = "1.0", features = ["derive"] } serde_json = "1.0" serde_bytes = "0.11" diff --git a/nexus/peer-bigquery/src/ast.rs b/nexus/peer-bigquery/src/ast.rs index d6b47fe631..dd020a6684 100644 --- a/nexus/peer-bigquery/src/ast.rs +++ b/nexus/peer-bigquery/src/ast.rs @@ -3,9 +3,9 @@ use std::ops::ControlFlow; use sqlparser::ast::Value::Number; use sqlparser::ast::{ - visit_expressions_mut, visit_function_arg_mut, visit_relations_mut, - visit_setexpr_mut, Array, BinaryOperator, DataType, DateTimeField, Expr, Function, FunctionArg, - FunctionArgExpr, Ident, ObjectName, Query, SetExpr, SetOperator, SetQuantifier, TimezoneInfo, + visit_expressions_mut, visit_function_arg_mut, visit_relations_mut, visit_setexpr_mut, Array, + BinaryOperator, DataType, DateTimeField, Expr, Function, FunctionArg, FunctionArgExpr, Ident, + ObjectName, Query, SetExpr, SetOperator, SetQuantifier, TimezoneInfo, }; #[derive(Default)] diff --git a/nexus/peer-bigquery/src/lib.rs b/nexus/peer-bigquery/src/lib.rs index 4ed659f750..851e8aa63c 100644 --- a/nexus/peer-bigquery/src/lib.rs +++ b/nexus/peer-bigquery/src/lib.rs @@ -261,8 +261,11 @@ impl QueryExecutor for BigQueryQueryExecutor { } async fn is_connection_valid(&self) -> anyhow::Result { let sql = "SELECT 1;"; - let test_stmt = parser::Parser::parse_sql(&GenericDialect {}, sql).unwrap(); - let _ = self.execute(&test_stmt[0]).await?; + let _result_set = self + .client + .job() + .query(&self.config.project_id, QueryRequest::new(sql)) + .await?; Ok(true) } } diff --git a/nexus/peer-bigquery/src/stream.rs b/nexus/peer-bigquery/src/stream.rs index 02417ea3b7..47ffb1af6e 100644 --- a/nexus/peer-bigquery/src/stream.rs +++ b/nexus/peer-bigquery/src/stream.rs @@ -1,5 +1,6 @@ use std::{ pin::Pin, + str::FromStr, task::{Context, Poll}, }; @@ -17,6 +18,7 @@ use pgwire::{ }, error::{PgWireError, PgWireResult}, }; +use rust_decimal::Decimal; use value::Value; #[derive(Debug)] @@ -127,9 +129,15 @@ impl BqRecordStream { FieldType::Float | FieldType::Float64 => { result_set.get_f64_by_name(field_name)?.map(Value::Double) } - FieldType::Bignumeric | FieldType::Numeric => result_set - .get_string_by_name(field_name)? - .map(Value::Numeric), + FieldType::Bignumeric | FieldType::Numeric => { + let result_string = result_set.get_string_by_name(field_name)?; + if let Some(result) = result_string { + let decimal = Decimal::from_str(&result)?; + Some(Value::Numeric(decimal)) + } else { + None + } + } FieldType::Boolean | FieldType::Bool => { result_set.get_bool_by_name(field_name)?.map(Value::Bool) } @@ -137,13 +145,15 @@ impl BqRecordStream { result_set.get_string_by_name(field_name)?.map(Value::Text) } FieldType::Timestamp => { - result_set.get_i64_by_name(field_name)?.map(|timestamp| { - Value::Timestamp(DateTime::::from_utc( - NaiveDateTime::from_timestamp_opt(timestamp, 0) - .expect("Invalid timestamp"), - Utc, - )) - }) + let timestamp = result_set + .get_i64_by_name(field_name)? + .ok_or(anyhow::Error::msg("Invalid timestamp"))?; + let naive_datetime = NaiveDateTime::from_timestamp_opt(timestamp, 0) + .ok_or(anyhow::Error::msg("Invalid timestamp"))?; + Some(Value::Timestamp(DateTime::::from_utc( + naive_datetime, + Utc, + ))) } FieldType::Record => todo!(), FieldType::Struct => todo!(), diff --git a/nexus/peer-cursor/src/util.rs b/nexus/peer-cursor/src/util.rs index 6b134d3ef8..afdf770896 100644 --- a/nexus/peer-cursor/src/util.rs +++ b/nexus/peer-cursor/src/util.rs @@ -6,7 +6,7 @@ use pgwire::{ api::results::{DataRowEncoder, FieldInfo, QueryResponse, Response}, error::{PgWireError, PgWireResult}, }; -use value::{Value}; +use value::Value; use crate::{Records, SchemaRef, SendableStream}; @@ -21,7 +21,7 @@ fn encode_value(value: &Value, builder: &mut DataRowEncoder) -> PgWireResult<()> Value::BigInt(v) => builder.encode_field(v), Value::Float(v) => builder.encode_field(v), Value::Double(v) => builder.encode_field(v), - Value::Numeric(v) => builder.encode_field(v), + Value::Numeric(v) => builder.encode_field(&v.to_string()), Value::Char(v) => builder.encode_field(&v.to_string()), Value::VarChar(v) => builder.encode_field(v), Value::Text(v) => builder.encode_field(v), diff --git a/nexus/peer-postgres/Cargo.toml b/nexus/peer-postgres/Cargo.toml index a1cce9849f..fe4c230523 100644 --- a/nexus/peer-postgres/Cargo.toml +++ b/nexus/peer-postgres/Cargo.toml @@ -8,6 +8,7 @@ edition = "2021" [dependencies] anyhow = "1.0" async-trait = "0.1" +rust_decimal = { version = "1.30.0", features = [ "tokio-pg" ] } bytes = "1.0" chrono = { version = "0.4", features = ["serde"] } futures = "0.3" diff --git a/nexus/peer-postgres/src/stream.rs b/nexus/peer-postgres/src/stream.rs index d5bd2ede42..c8500c0ca7 100644 --- a/nexus/peer-postgres/src/stream.rs +++ b/nexus/peer-postgres/src/stream.rs @@ -1,14 +1,14 @@ -use std::{ - pin::Pin, - task::{Context, Poll}, -}; - use bytes::Bytes; use chrono::{DateTime, NaiveDate, NaiveDateTime, NaiveTime, Utc}; use futures::Stream; use peer_cursor::{Record, RecordStream, SchemaRef}; use pgerror::PgError; use pgwire::error::{PgWireError, PgWireResult}; +use rust_decimal::Decimal; +use std::{ + pin::Pin, + task::{Context, Poll}, +}; use tokio_postgres::{types::Type, Row, RowStream}; use uuid::Uuid; use value::{array::ArrayValue, Value}; @@ -151,7 +151,7 @@ fn values_from_row(row: &Row) -> Vec { .unwrap_or(Value::Null) } &Type::NUMERIC => { - let numeric: Option = row.get(i); + let numeric: Option = row.get(i); numeric.map(Value::Numeric).unwrap_or(Value::Null) } &Type::NUMERIC_ARRAY => { diff --git a/nexus/server/src/main.rs b/nexus/server/src/main.rs index 1b87c12aa5..bd8d08b517 100644 --- a/nexus/server/src/main.rs +++ b/nexus/server/src/main.rs @@ -171,6 +171,7 @@ impl NexusBackend { format!("[peer]: invalid configuration: {}", e.to_string()), ))) })?; + self.executors.remove(&peer.name); let catalog = self.catalog.lock().await; catalog.create_peer(peer.as_ref()).await.map_err(|e| { PgWireError::UserError(Box::new(ErrorInfo::new( @@ -552,6 +553,9 @@ impl ExtendedQueryHandler for NexusBackend { Some(Config::MongoConfig(_)) => { panic!("peer type not supported: {:?}", peer) } + Some(Config::EventhubConfig(_)) => { + panic!("peer type not supported: {:?}", peer) + } None => { panic!("peer type not supported: {:?}", peer) } diff --git a/nexus/server/tests/results/expected/bq.sql.out b/nexus/server/tests/results/expected/bq.sql.out index 2cd7aa907c..2c186752a4 100644 --- a/nexus/server/tests/results/expected/bq.sql.out +++ b/nexus/server/tests/results/expected/bq.sql.out @@ -301,3 +301,9 @@ solana solana 2022-08-01 00:00:00.000000 2 +12.33 +12.2 +-1233422.17 +20100000012.2 +12.33 +12.2 diff --git a/nexus/server/tests/results/expected/postgres.sql.out b/nexus/server/tests/results/expected/postgres.sql.out index 616cd46827..527b30dc09 100644 --- a/nexus/server/tests/results/expected/postgres.sql.out +++ b/nexus/server/tests/results/expected/postgres.sql.out @@ -21,6 +21,14 @@ true 1 2.7182818284 4000000000003 +123.45 +123.45 +122.25 +-76.75 +123 +43423424234242342342432432423432432432432432424323.65 +123450.00 +23 Lorem ipsum dolor sit amet, consectetur adipiscing elit, sed do eiusmod tempor incididunt ut labore et dolore magna aliqua. Ut enim ad minim veniam, quis nostrud exercitation ullamco laboris nisi ut aliquip ex ea commodo consequat. Duis aute irure dolor in reprehenderit in voluptate velit esse cillum dolore eu fugiat nulla pariatur. Excepteur sint occaecat cupidatat non proident, sunt in culpa qui officia deserunt mollit anim id est laborum m ipsu 2011-11-11 00:34:56.000000 diff --git a/nexus/server/tests/sql/bq.sql b/nexus/server/tests/sql/bq.sql index d98114151b..007bb79967 100644 --- a/nexus/server/tests/sql/bq.sql +++ b/nexus/server/tests/sql/bq.sql @@ -117,3 +117,9 @@ SELECT NULL FROM bq_test.transactions; SELECT COALESCE(chain,'sai'::text) FROM bq_test.transactions; SELECT date_trunc('month',tx_timestamp),count(*) FROM bq_test.transactions GROUP BY 1; + +-- NUMERIC, BIGNUMERIC columns +SELECT c19,c18 FROM bq_test.test_types LIMIT 1; +SELECT c19 - 1233434.5 FROM bq_test.test_types LIMIT 1; +SELECT c18 + 20100000000 FROM bq_test.test_types LIMIT 1; +SELECT sum(c19), max(c18) FROM bq_test.test_types LIMIT 1; \ No newline at end of file diff --git a/nexus/server/tests/sql/postgres.sql b/nexus/server/tests/sql/postgres.sql index 65c44ccd8b..d5e21f8d05 100644 --- a/nexus/server/tests/sql/postgres.sql +++ b/nexus/server/tests/sql/postgres.sql @@ -30,6 +30,15 @@ SELECT (FLOAT4 - 2)::INT4 FROM pg_test.PUBLIC.ILMETZIK; SELECT FLOAT8 FROM pg_test.PUBLIC.ILMETZIK; SELECT (FLOAT8 + 4000000000000)::INT8 FROM pg_test.PUBLIC.ILMETZIK; +SELECT numeric FROM pg_test.PUBLIC.ILMETZIK; +SELECT SUM(numeric) FROM pg_test.PUBLIC.ILMETZIK; +SELECT (numeric - 1.2)::text FROM pg_test.PUBLIC.ILMETZIK; +SELECT (numeric - 200.2)::float4 FROM pg_test.PUBLIC.ILMETZIK; +SELECT numeric::integer FROM pg_test.public.ilmetzik; +SELECT (numeric + 43423424234242342342432432423432432432432432424200.2)::text FROM pg_test.PUBLIC.ILMETZIK; +SELECT 1000 * numeric FROM pg_test.public.ilmetzik; +SELECT numeric::INT4 - 100 FROM pg_test.public.ilmetzik; + SELECT TEXT FROM pg_test.PUBLIC.ILMETZIK; SELECT SUBSTRING(TEXT, 5, 6) FROM pg_test.PUBLIC.ILMETZIK; diff --git a/nexus/value/Cargo.toml b/nexus/value/Cargo.toml index d91c65a32c..2ff6eb6f64 100644 --- a/nexus/value/Cargo.toml +++ b/nexus/value/Cargo.toml @@ -7,6 +7,7 @@ edition = "2021" [dependencies] base64 = "0.21" +rust_decimal = { version = "1.30.0", features = [ "tokio-pg" ] } bytes = "1.1" serde = { version = "1.0", features = ["derive"] } serde_json = "1.0" diff --git a/nexus/value/src/lib.rs b/nexus/value/src/lib.rs index 88574c77c6..a0512718b8 100644 --- a/nexus/value/src/lib.rs +++ b/nexus/value/src/lib.rs @@ -1,9 +1,10 @@ use array::ArrayValue; use bytes::Bytes; use chrono::{DateTime, NaiveDate, NaiveDateTime, NaiveTime, Utc}; +use rust_decimal::Decimal; use std::collections::HashMap; +use std::str::FromStr; use uuid::Uuid; - pub mod array; #[derive(Debug, PartialEq, Clone)] @@ -17,7 +18,7 @@ pub enum Value { BigInt(i64), Float(f32), Double(f64), - Numeric(String), + Numeric(Decimal), Char(char), VarChar(String), Text(String), @@ -73,7 +74,7 @@ impl Value { Value::Double(value) } - pub fn numeric(value: String) -> Self { + pub fn numeric(value: Decimal) -> Self { Value::Numeric(value) } @@ -164,7 +165,9 @@ impl Value { } else if let Some(f) = n.as_f64() { Value::Double(f) } else { - Value::Numeric(n.to_string()) + let number_str = n.to_string(); + let decimal = Decimal::from_str(&number_str).expect("unsupported number type"); + Value::Numeric(decimal) } } serde_json::Value::String(s) => Value::Text(s.clone()), @@ -224,7 +227,7 @@ impl Value { Value::Double(n) => { serde_json::Value::Number(serde_json::Number::from_f64(*n).unwrap()) } - Value::Numeric(n) => serde_json::Value::String(n.clone()), + Value::Numeric(n) => serde_json::Value::String(n.to_string()), Value::Char(c) => serde_json::Value::String(c.to_string()), Value::VarChar(s) => serde_json::Value::String(s.clone()), Value::Text(s) => serde_json::Value::String(s.clone()), diff --git a/protos/flow.proto b/protos/flow.proto index 58da04867e..ee0a5cfede 100644 --- a/protos/flow.proto +++ b/protos/flow.proto @@ -20,6 +20,11 @@ message FlowConnectionConfigs { map table_name_mapping = 5; map src_table_id_name_mapping = 6; map table_name_schema_mapping = 7; + + // This is an optional peer that will be used to hold metadata in cases where + // the destination isn't ideal for holding metadata. + peerdb.peers.Peer metadata_peer = 8; + uint32 max_batch_size = 9; } message SyncFlowOptions { int32 batch_size = 1; } @@ -77,6 +82,7 @@ message SetupReplicationInput { message CreateRawTableInput { peerdb.peers.Peer peer_connection_config = 1; string flow_job_name = 2; + map table_name_mapping = 3; } message CreateRawTableOutput { string table_identifier = 1; } diff --git a/protos/peers.proto b/protos/peers.proto index e14785ffd5..97f5a3dc37 100644 --- a/protos/peers.proto +++ b/protos/peers.proto @@ -45,11 +45,19 @@ message PostgresConfig { string database = 5; } +message EventHubConfig { + string namespace = 1; + string resource_group = 2; + string location = 3; + PostgresConfig metadata_db = 4; +} + enum DBType { BIGQUERY = 0; SNOWFLAKE = 1; MONGO = 2; POSTGRES = 3; + EVENTHUB = 4; } message Peer { @@ -60,5 +68,6 @@ message Peer { BigqueryConfig bigquery_config = 4; MongoConfig mongo_config = 5; PostgresConfig postgres_config = 6; + EventHubConfig eventhub_config = 7; } } diff --git a/stacks/nexus.Dockerfile b/stacks/nexus.Dockerfile index db2ec69177..c7474782f0 100644 --- a/stacks/nexus.Dockerfile +++ b/stacks/nexus.Dockerfile @@ -21,11 +21,10 @@ WORKDIR /root COPY nexus nexus COPY protos protos WORKDIR /root/nexus -RUN mkdir -p /var/log/peerdb RUN CARGO_REGISTRIES_CRATES_IO_PROTOCOL=sparse cargo build --release --bin peerdb-server -FROM gcr.io/distroless/cc-debian11 +FROM ubuntu:20.04 +RUN apt-get update && apt-get install -y ca-certificates +RUN mkdir -p /var/log/peerdb COPY --from=builder /root/nexus/target/release/peerdb-server . -# distroless doesn't allow mkdir, so we have to copy the log directory -COPY --from=builder /var/log/peerdb /var/log/peerdb CMD ["./peerdb-server"]