From b2fb91fdcec44a40abf72ac39ce82502ac01ffb5 Mon Sep 17 00:00:00 2001 From: Kevin Biju Date: Thu, 7 Sep 2023 10:00:59 +0530 Subject: [PATCH 1/4] split Connector interface to better represent capabilities, Eventhub stores syncBatchID now --- flow/activities/flowable.go | 161 +++++++++++++----------- flow/activities/snapshot_activity.go | 2 +- flow/connectors/bigquery/bigquery.go | 30 +---- flow/connectors/bigquery/qrep.go | 25 ---- flow/connectors/core.go | 164 +++++++++++++++++++------ flow/connectors/eventhub/eventhub.go | 55 +-------- flow/connectors/eventhub/metadata.go | 63 ++++++++-- flow/connectors/eventhub/qrep.go | 33 ----- flow/connectors/postgres/qrep.go | 13 -- flow/connectors/s3/qrep.go | 22 ---- flow/connectors/s3/s3.go | 84 ------------- flow/connectors/snowflake/qrep.go | 12 -- flow/connectors/snowflake/snowflake.go | 23 ---- flow/connectors/sqlserver/qrep.go | 21 ---- flow/connectors/sqlserver/sqlserver.go | 86 ------------- 15 files changed, 281 insertions(+), 513 deletions(-) delete mode 100644 flow/connectors/eventhub/qrep.go diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index 4837eb9d2e..625cad29c9 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -2,6 +2,7 @@ package activities import ( "context" + "errors" "fmt" "sync" "time" @@ -27,7 +28,7 @@ type CheckConnectionResult struct { type SlotSnapshotSignal struct { signal *connpostgres.SlotSignal snapshotName string - connector connectors.Connector + connector connectors.CDCPullConnector } type FlowableActivity struct { @@ -40,14 +41,13 @@ func (a *FlowableActivity) CheckConnection( ctx context.Context, config *protos.Peer, ) (*CheckConnectionResult, error) { - conn, err := connectors.GetConnector(ctx, config) - defer connectors.CloseConnector(conn) - + dstConn, err := connectors.GetCDCSyncConnector(ctx, config) if err != nil { return nil, fmt.Errorf("failed to get connector: %w", err) } + defer connectors.CloseConnector(dstConn) - needsSetup := conn.NeedsSetupMetadataTables() + needsSetup := dstConn.NeedsSetupMetadataTables() return &CheckConnectionResult{ NeedsSetupMetadataTables: needsSetup, @@ -56,14 +56,13 @@ func (a *FlowableActivity) CheckConnection( // SetupMetadataTables implements SetupMetadataTables. func (a *FlowableActivity) SetupMetadataTables(ctx context.Context, config *protos.Peer) error { - conn, err := connectors.GetConnector(ctx, config) - defer connectors.CloseConnector(conn) - + dstConn, err := connectors.GetCDCSyncConnector(ctx, config) if err != nil { return fmt.Errorf("failed to get connector: %w", err) } + defer connectors.CloseConnector(dstConn) - if err := conn.SetupMetadataTables(); err != nil { + if err := dstConn.SetupMetadataTables(); err != nil { return fmt.Errorf("failed to setup metadata tables: %w", err) } @@ -75,14 +74,13 @@ func (a *FlowableActivity) GetLastSyncedID( ctx context.Context, config *protos.GetLastSyncedIDInput, ) (*protos.LastSyncState, error) { - conn, err := connectors.GetConnector(ctx, config.PeerConnectionConfig) - defer connectors.CloseConnector(conn) - + dstConn, err := connectors.GetCDCSyncConnector(ctx, config.PeerConnectionConfig) if err != nil { return nil, fmt.Errorf("failed to get connector: %w", err) } + defer connectors.CloseConnector(dstConn) - return conn.GetLastOffset(config.FlowJobName) + return dstConn.GetLastOffset(config.FlowJobName) } // EnsurePullability implements EnsurePullability. @@ -90,13 +88,13 @@ func (a *FlowableActivity) EnsurePullability( ctx context.Context, config *protos.EnsurePullabilityBatchInput, ) (*protos.EnsurePullabilityBatchOutput, error) { - conn, err := connectors.GetConnector(ctx, config.PeerConnectionConfig) - defer connectors.CloseConnector(conn) - + srcConn, err := connectors.GetCDCPullConnector(ctx, config.PeerConnectionConfig) if err != nil { return nil, fmt.Errorf("failed to get connector: %w", err) } - output, err := conn.EnsurePullability(config) + defer connectors.CloseConnector(srcConn) + + output, err := srcConn.EnsurePullability(config) if err != nil { return nil, fmt.Errorf("failed to ensure pullability: %w", err) } @@ -110,13 +108,13 @@ func (a *FlowableActivity) CreateRawTable( config *protos.CreateRawTableInput, ) (*protos.CreateRawTableOutput, error) { ctx = context.WithValue(ctx, shared.CDCMirrorMonitorKey, a.CatalogMirrorMonitor) - conn, err := connectors.GetConnector(ctx, config.PeerConnectionConfig) + dstConn, err := connectors.GetCDCSyncConnector(ctx, config.PeerConnectionConfig) if err != nil { return nil, fmt.Errorf("failed to get connector: %w", err) } - defer connectors.CloseConnector(conn) + defer connectors.CloseConnector(dstConn) - res, err := conn.CreateRawTable(config) + res, err := dstConn.CreateRawTable(config) if err != nil { return nil, err } @@ -133,14 +131,13 @@ func (a *FlowableActivity) GetTableSchema( ctx context.Context, config *protos.GetTableSchemaBatchInput, ) (*protos.GetTableSchemaBatchOutput, error) { - conn, err := connectors.GetConnector(ctx, config.PeerConnectionConfig) - defer connectors.CloseConnector(conn) - + srcConn, err := connectors.GetCDCPullConnector(ctx, config.PeerConnectionConfig) if err != nil { return nil, fmt.Errorf("failed to get connector: %w", err) } + defer connectors.CloseConnector(srcConn) - return conn.GetTableSchema(config) + return srcConn.GetTableSchema(config) } // CreateNormalizedTable creates a normalized table in the destination flowable. @@ -148,12 +145,11 @@ func (a *FlowableActivity) CreateNormalizedTable( ctx context.Context, config *protos.SetupNormalizedTableBatchInput, ) (*protos.SetupNormalizedTableBatchOutput, error) { - conn, err := connectors.GetConnector(ctx, config.PeerConnectionConfig) - defer connectors.CloseConnector(conn) - + conn, err := connectors.GetCDCSyncConnector(ctx, config.PeerConnectionConfig) if err != nil { return nil, fmt.Errorf("failed to get connector: %w", err) } + defer connectors.CloseConnector(conn) return conn.SetupNormalizedTables(config) } @@ -165,22 +161,22 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, ctx = context.WithValue(ctx, shared.EnableMetricsKey, a.EnableMetrics) ctx = context.WithValue(ctx, shared.CDCMirrorMonitorKey, a.CatalogMirrorMonitor) - src, err := connectors.GetConnector(ctx, conn.Source) - defer connectors.CloseConnector(src) + + srcConn, err := connectors.GetCDCPullConnector(ctx, conn.Source) if err != nil { return nil, fmt.Errorf("failed to get source connector: %w", err) } - - dest, err := connectors.GetConnector(ctx, conn.Destination) - defer connectors.CloseConnector(dest) + defer connectors.CloseConnector(srcConn) + dstConn, err := connectors.GetCDCSyncConnector(ctx, conn.Destination) if err != nil { return nil, fmt.Errorf("failed to get destination connector: %w", err) } + defer connectors.CloseConnector(dstConn) log.WithFields(log.Fields{ "flowName": input.FlowConnectionConfigs.FlowJobName, }).Infof("initializing table schema...") - err = dest.InitializeTableSchema(input.FlowConnectionConfigs.TableNameSchemaMapping) + err = dstConn.InitializeTableSchema(input.FlowConnectionConfigs.TableNameSchemaMapping) if err != nil { return nil, fmt.Errorf("failed to initialize table schema: %w", err) } @@ -190,7 +186,7 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, }).Info("pulling records...") startTime := time.Now() - recordsWithTableSchemaDelta, err := src.PullRecords(&model.PullRecordsRequest{ + recordsWithTableSchemaDelta, err := srcConn.PullRecords(&model.PullRecordsRequest{ FlowJobName: input.FlowConnectionConfigs.FlowJobName, SrcTableIDNameMapping: input.FlowConnectionConfigs.SrcTableIdNameMapping, TableNameMapping: input.FlowConnectionConfigs.TableNameMapping, @@ -207,7 +203,7 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, } recordBatch := recordsWithTableSchemaDelta.RecordBatch if a.CatalogMirrorMonitor.IsActive() && len(recordBatch.Records) > 0 { - syncBatchID, err := dest.GetLastSyncBatchID(input.FlowConnectionConfigs.FlowJobName) + syncBatchID, err := dstConn.GetLastSyncBatchID(input.FlowConnectionConfigs.FlowJobName) if err != nil && conn.Destination.Type != protos.DBType_EVENTHUB { return nil, err } @@ -242,7 +238,7 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, }, nil } - res, err := dest.SyncRecords(&model.SyncRecordsRequest{ + res, err := dstConn.SyncRecords(&model.SyncRecordsRequest{ Records: recordBatch, FlowJobName: input.FlowConnectionConfigs.FlowJobName, SyncMode: input.FlowConnectionConfigs.CdcSyncMode, @@ -271,6 +267,9 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, return nil, err } } + if err != nil { + return nil, err + } res.TableSchemaDelta = recordsWithTableSchemaDelta.TableSchemaDelta res.RelationMessageMapping = recordsWithTableSchemaDelta.RelationMessageMapping activity.RecordHeartbeat(ctx, "pushed records") @@ -285,17 +284,25 @@ func (a *FlowableActivity) StartNormalize( conn := input.FlowConnectionConfigs ctx = context.WithValue(ctx, shared.EnableMetricsKey, a.EnableMetrics) - src, err := connectors.GetConnector(ctx, conn.Source) - defer connectors.CloseConnector(src) - if err != nil { - return nil, fmt.Errorf("failed to get source connector: %w", err) - } + dstConn, err := connectors.GetCDCNormalizeConnector(ctx, conn.Destination) + if errors.Is(err, connectors.ErrUnsupportedFunctionality) { + dstConn, err := connectors.GetCDCSyncConnector(ctx, conn.Destination) + if err != nil { + return nil, fmt.Errorf("failed to get connector: %v", err) + } + defer connectors.CloseConnector(dstConn) - dest, err := connectors.GetConnector(ctx, conn.Destination) - defer connectors.CloseConnector(dest) - if err != nil { - return nil, fmt.Errorf("failed to get destination connector: %w", err) + lastSyncBatchID, err := dstConn.GetLastSyncBatchID(input.FlowConnectionConfigs.FlowJobName) + if err != nil { + return nil, fmt.Errorf("failed to get last sync batch ID: %v", err) + } + + return nil, a.CatalogMirrorMonitor.UpdateEndTimeForCDCBatch(ctx, input.FlowConnectionConfigs.FlowJobName, + lastSyncBatchID) + } else if err != nil { + return nil, err } + defer connectors.CloseConnector(dstConn) shutdown := utils.HeartbeatRoutine(ctx, 2*time.Minute, func() string { return fmt.Sprintf("normalizing records from batch for job - %s", input.FlowConnectionConfigs.FlowJobName) @@ -305,12 +312,12 @@ func (a *FlowableActivity) StartNormalize( }() log.Info("initializing table schema...") - err = dest.InitializeTableSchema(input.FlowConnectionConfigs.TableNameSchemaMapping) + err = dstConn.InitializeTableSchema(input.FlowConnectionConfigs.TableNameSchemaMapping) if err != nil { return nil, fmt.Errorf("failed to initialize table schema: %w", err) } - res, err := dest.NormalizeRecords(&model.NormalizeRecordsRequest{ + res, err := dstConn.NormalizeRecords(&model.NormalizeRecordsRequest{ FlowJobName: input.FlowConnectionConfigs.FlowJobName, SoftDelete: input.FlowConnectionConfigs.SoftDelete, }) @@ -336,18 +343,20 @@ func (a *FlowableActivity) ReplayTableSchemaDelta( ctx context.Context, input *protos.ReplayTableSchemaDeltaInput, ) error { - dest, err := connectors.GetConnector(ctx, input.FlowConnectionConfigs.Destination) - defer connectors.CloseConnector(dest) - if err != nil { - return fmt.Errorf("failed to get destination connector: %w", err) + dest, err := connectors.GetCDCNormalizeConnector(ctx, input.FlowConnectionConfigs.Destination) + if errors.Is(err, connectors.ErrUnsupportedFunctionality) { + return nil + } else if err != nil { + return err } + defer connectors.CloseConnector(dest) return dest.ReplayTableSchemaDelta(input.FlowConnectionConfigs.FlowJobName, input.TableSchemaDelta) } // SetupQRepMetadataTables sets up the metadata tables for QReplication. func (a *FlowableActivity) SetupQRepMetadataTables(ctx context.Context, config *protos.QRepConfig) error { - conn, err := connectors.GetConnector(ctx, config.DestinationPeer) + conn, err := connectors.GetQRepSyncConnector(ctx, config.DestinationPeer) if err != nil { return fmt.Errorf("failed to get connector: %w", err) } @@ -362,11 +371,11 @@ func (a *FlowableActivity) GetQRepPartitions(ctx context.Context, last *protos.QRepPartition, runUUID string, ) (*protos.QRepParitionResult, error) { - conn, err := connectors.GetConnector(ctx, config.SourcePeer) + srcConn, err := connectors.GetQRepPullConnector(ctx, config.SourcePeer) if err != nil { return nil, fmt.Errorf("failed to get connector: %w", err) } - defer connectors.CloseConnector(conn) + defer connectors.CloseConnector(srcConn) shutdown := utils.HeartbeatRoutine(ctx, 2*time.Minute, func() string { return fmt.Sprintf("getting partitions for job - %s", config.FlowJobName) @@ -377,7 +386,7 @@ func (a *FlowableActivity) GetQRepPartitions(ctx context.Context, }() startTime := time.Now() - partitions, err := conn.GetQRepPartitions(config, last) + partitions, err := srcConn.GetQRepPartitions(config, last) if err != nil { return nil, fmt.Errorf("failed to get partitions from source: %w", err) } @@ -423,17 +432,17 @@ func (a *FlowableActivity) replicateQRepPartition(ctx context.Context, runUUID string, ) error { ctx = context.WithValue(ctx, shared.EnableMetricsKey, a.EnableMetrics) - srcConn, err := connectors.GetConnector(ctx, config.SourcePeer) + srcConn, err := connectors.GetQRepPullConnector(ctx, config.SourcePeer) if err != nil { return fmt.Errorf("failed to get source connector: %w", err) } defer connectors.CloseConnector(srcConn) - destConn, err := connectors.GetConnector(ctx, config.DestinationPeer) + dstConn, err := connectors.GetQRepSyncConnector(ctx, config.DestinationPeer) if err != nil { return fmt.Errorf("failed to get destination connector: %w", err) } - defer connectors.CloseConnector(destConn) + defer connectors.CloseConnector(dstConn) log.Printf("replicating partition %s\n", partition.PartitionId) @@ -499,7 +508,7 @@ func (a *FlowableActivity) replicateQRepPartition(ctx context.Context, shutdown <- true }() - res, err := destConn.SyncQRepRecords(config, partition, stream) + res, err := dstConn.SyncQRepRecords(config, partition, stream) if err != nil { return fmt.Errorf("failed to sync records: %w", err) } @@ -529,9 +538,11 @@ func (a *FlowableActivity) replicateQRepPartition(ctx context.Context, func (a *FlowableActivity) ConsolidateQRepPartitions(ctx context.Context, config *protos.QRepConfig, runUUID string) error { ctx = context.WithValue(ctx, shared.EnableMetricsKey, a.EnableMetrics) - dst, err := connectors.GetConnector(ctx, config.DestinationPeer) - if err != nil { - return fmt.Errorf("failed to get destination connector: %w", err) + dstConn, err := connectors.GetQRepConsolidateConnector(ctx, config.DestinationPeer) + if errors.Is(err, connectors.ErrUnsupportedFunctionality) { + return nil + } else if err != nil { + return err } shutdown := utils.HeartbeatRoutine(ctx, 2*time.Minute, func() string { @@ -542,8 +553,10 @@ func (a *FlowableActivity) ConsolidateQRepPartitions(ctx context.Context, config shutdown <- true }() - err = dst.ConsolidateQRepPartitions(config) - if err != nil { + err = dstConn.ConsolidateQRepPartitions(config) + if errors.Is(err, connectors.ErrUnsupportedFunctionality) { + return nil + } else if err != nil { return err } err = a.CatalogMirrorMonitor.UpdateEndTimeForQRepRun(ctx, runUUID) @@ -551,32 +564,34 @@ func (a *FlowableActivity) ConsolidateQRepPartitions(ctx context.Context, config } func (a *FlowableActivity) CleanupQRepFlow(ctx context.Context, config *protos.QRepConfig) error { - dst, err := connectors.GetConnector(ctx, config.DestinationPeer) - if err != nil { - return fmt.Errorf("failed to get destination connector: %w", err) + dst, err := connectors.GetQRepConsolidateConnector(ctx, config.DestinationPeer) + if errors.Is(err, connectors.ErrUnsupportedFunctionality) { + return nil + } else if err != nil { + return err } return dst.CleanupQRepFlow(config) } func (a *FlowableActivity) DropFlow(ctx context.Context, config *protos.ShutdownRequest) error { - src, err := connectors.GetConnector(ctx, config.SourcePeer) - defer connectors.CloseConnector(src) + srcConn, err := connectors.GetCDCPullConnector(ctx, config.SourcePeer) if err != nil { return fmt.Errorf("failed to get source connector: %w", err) } + defer connectors.CloseConnector(srcConn) - dest, err := connectors.GetConnector(ctx, config.DestinationPeer) - defer connectors.CloseConnector(dest) + dstConn, err := connectors.GetCDCSyncConnector(ctx, config.DestinationPeer) if err != nil { return fmt.Errorf("failed to get destination connector: %w", err) } + defer connectors.CloseConnector(dstConn) - err = src.PullFlowCleanup(config.FlowJobName) + err = srcConn.PullFlowCleanup(config.FlowJobName) if err != nil { return fmt.Errorf("failed to cleanup source: %w", err) } - err = dest.SyncFlowCleanup(config.FlowJobName) + err = dstConn.SyncFlowCleanup(config.FlowJobName) if err != nil { return fmt.Errorf("failed to cleanup destination: %w", err) } diff --git a/flow/activities/snapshot_activity.go b/flow/activities/snapshot_activity.go index efd2950b59..f50c235bac 100644 --- a/flow/activities/snapshot_activity.go +++ b/flow/activities/snapshot_activity.go @@ -39,7 +39,7 @@ func (a *SnapshotActivity) SetupReplication( return nil, nil } - conn, err := connectors.GetConnector(ctx, config.PeerConnectionConfig) + conn, err := connectors.GetCDCPullConnector(ctx, config.PeerConnectionConfig) if err != nil { return nil, fmt.Errorf("failed to get connector: %w", err) } diff --git a/flow/connectors/bigquery/bigquery.go b/flow/connectors/bigquery/bigquery.go index 2074f2b62e..8f6df0f5bb 100644 --- a/flow/connectors/bigquery/bigquery.go +++ b/flow/connectors/bigquery/bigquery.go @@ -419,12 +419,6 @@ func (c *BigQueryConnector) getTableNametoUnchangedCols(flowJobName string, sync return resultMap, nil } -// PullRecords pulls records from the source. -func (c *BigQueryConnector) PullRecords(req *model.PullRecordsRequest) ( - *model.RecordsWithTableSchemaDelta, error) { - panic("not implemented") -} - // ValueSaver interface for bqRecord func (r StagingBQRecord) Save() (map[string]bigquery.Value, string, error) { return map[string]bigquery.Value{ @@ -468,13 +462,13 @@ func (c *BigQueryConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S var res *model.SyncResponse if req.SyncMode == protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO { - res, err = c.SyncRecordsViaAvro(req, rawTableName, syncBatchID) + res, err = c.syncRecordsViaAvro(req, rawTableName, syncBatchID) if err != nil { return nil, err } } if req.SyncMode == protos.QRepSyncMode_QREP_SYNC_MODE_MULTI_INSERT { - res, err = c.SyncRecordsViaSQL(req, rawTableName, syncBatchID) + res, err = c.syncRecordsViaSQL(req, rawTableName, syncBatchID) if err != nil { return nil, err } @@ -483,7 +477,7 @@ func (c *BigQueryConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S return res, nil } -func (c *BigQueryConnector) SyncRecordsViaSQL(req *model.SyncRecordsRequest, +func (c *BigQueryConnector) syncRecordsViaSQL(req *model.SyncRecordsRequest, rawTableName string, syncBatchID int64) (*model.SyncResponse, error) { stagingTableName := c.getStagingTableName(req.FlowJobName) stagingTable := c.client.Dataset(c.datasetID).Table(stagingTableName) @@ -660,7 +654,7 @@ func (c *BigQueryConnector) SyncRecordsViaSQL(req *model.SyncRecordsRequest, }, nil } -func (c *BigQueryConnector) SyncRecordsViaAvro(req *model.SyncRecordsRequest, +func (c *BigQueryConnector) syncRecordsViaAvro(req *model.SyncRecordsRequest, rawTableName string, syncBatchID int64) (*model.SyncResponse, error) { tableNameRowsMapping := make(map[string]uint32) first := true @@ -1095,12 +1089,6 @@ func (c *BigQueryConnector) metadataHasJob(jobName string) (bool, error) { return count > 0, nil } -// GetTableSchema returns the schema for a table, implementing the Connector interface. -func (c *BigQueryConnector) GetTableSchema( - req *protos.GetTableSchemaBatchInput) (*protos.GetTableSchemaBatchOutput, error) { - panic("not implemented") -} - // SetupNormalizedTables sets up normalized tables, implementing the Connector interface. // This runs CREATE TABLE IF NOT EXISTS on bigquery, using the schema and table name provided. func (c *BigQueryConnector) SetupNormalizedTables( @@ -1147,16 +1135,6 @@ func (c *BigQueryConnector) SetupNormalizedTables( }, nil } -// EnsurePullability ensures that the given table is pullable, implementing the Connector interface. -func (c *BigQueryConnector) EnsurePullability(*protos.EnsurePullabilityBatchInput) ( - *protos.EnsurePullabilityBatchOutput, error) { - panic("not implemented") -} - -func (c *BigQueryConnector) PullFlowCleanup(jobName string) error { - panic("not implemented") -} - func (c *BigQueryConnector) SyncFlowCleanup(jobName string) error { dataset := c.client.Dataset(c.datasetID) // deleting PeerDB specific tables diff --git a/flow/connectors/bigquery/qrep.go b/flow/connectors/bigquery/qrep.go index e1ee8cf3c5..5966e6fe45 100644 --- a/flow/connectors/bigquery/qrep.go +++ b/flow/connectors/bigquery/qrep.go @@ -13,18 +13,6 @@ import ( "google.golang.org/protobuf/encoding/protojson" ) -func (c *BigQueryConnector) GetQRepPartitions(config *protos.QRepConfig, - last *protos.QRepPartition, -) ([]*protos.QRepPartition, error) { - panic("not implemented") -} - -func (c *BigQueryConnector) PullQRepRecords(config *protos.QRepConfig, - partition *protos.QRepPartition, -) (*model.QRecordBatch, error) { - panic("not implemented") -} - func (c *BigQueryConnector) SyncQRepRecords( config *protos.QRepConfig, partition *protos.QRepPartition, @@ -130,19 +118,6 @@ func (c *BigQueryConnector) SetupQRepMetadataTables(config *protos.QRepConfig) e return nil } -func (c *BigQueryConnector) ConsolidateQRepPartitions(config *protos.QRepConfig) error { - log.Infof("Consolidating partitions for flow job %s", config.FlowJobName) - log.Infof("This is a no-op for BigQuery") - return nil -} - -// CleanupQRepFlow function for bigquery connector -func (c *BigQueryConnector) CleanupQRepFlow(config *protos.QRepConfig) error { - log.Infof("Cleaning up flow job %s", config.FlowJobName) - log.Infof("This is a no-op for BigQuery") - return nil -} - func (c *BigQueryConnector) isPartitionSynced(partitionID string) (bool, error) { queryString := fmt.Sprintf( "SELECT COUNT(*) FROM %s._peerdb_query_replication_metadata WHERE partitionID = '%s';", diff --git a/flow/connectors/core.go b/flow/connectors/core.go index 37d6a0aec2..dbc8171265 100644 --- a/flow/connectors/core.go +++ b/flow/connectors/core.go @@ -2,10 +2,9 @@ package connectors import ( "context" - "fmt" + "errors" 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" conns3 "github.com/PeerDB-io/peer-flow/connectors/s3" connsnowflake "github.com/PeerDB-io/peer-flow/connectors/snowflake" @@ -14,78 +13,124 @@ import ( "github.com/PeerDB-io/peer-flow/model" ) +var ErrUnsupportedFunctionality = errors.New("requested connector does not support functionality") + type Connector interface { Close() error ConnectionActive() bool - NeedsSetupMetadataTables() bool - SetupMetadataTables() error - GetLastOffset(jobName string) (*protos.LastSyncState, error) - GetLastSyncBatchID(jobName string) (int64, error) +} + +type CDCPullConnector interface { + Connector // GetTableSchema returns the schema of a table. GetTableSchema(req *protos.GetTableSchemaBatchInput) (*protos.GetTableSchemaBatchOutput, error) - // SetupNormalizedTables sets up the normalized table on the connector. - SetupNormalizedTables(req *protos.SetupNormalizedTableBatchInput) ( - *protos.SetupNormalizedTableBatchOutput, error) - // EnsurePullability ensures that the connector is pullable. EnsurePullability(req *protos.EnsurePullabilityBatchInput) ( *protos.EnsurePullabilityBatchOutput, error) - // InitializeTableSchema initializes the table schema of all the destination tables for the connector. - InitializeTableSchema(req map[string]*protos.TableSchema) error - // ReplayTableSchemaDelta changes a destination table to match the schema at source - // This could involve adding or dropping multiple columns. - ReplayTableSchemaDelta(flowJobName string, schemaDelta *protos.TableSchemaDelta) error - // Methods related to retrieving and pusing records for this connector as a source and destination. // PullRecords pulls records from the source, and returns a RecordBatch. // This method should be idempotent, and should be able to be called multiple times with the same request. PullRecords(req *model.PullRecordsRequest) (*model.RecordsWithTableSchemaDelta, error) + // PullFlowCleanup drops both the Postgres publication and replication slot, as a part of DROP MIRROR + PullFlowCleanup(jobName string) error +} + +type CDCSyncConnector interface { + Connector + + // NeedsSetupMetadataTables checks if the metadata table [PEERDB_MIRROR_JOBS] needs to be created. + NeedsSetupMetadataTables() bool + + // SetupMetadataTables creates the metadata table [PEERDB_MIRROR_JOBS] if necessary. + SetupMetadataTables() error + + // GetLastOffset gets the last offset from the metadata table on the destination + GetLastOffset(jobName string) (*protos.LastSyncState, error) + + // GetLastSyncBatchID gets the last batch synced to the destination from the metadata table + GetLastSyncBatchID(jobName string) (int64, error) + + // InitializeTableSchema initializes the table schema of all the destination tables for the connector. + InitializeTableSchema(req map[string]*protos.TableSchema) error + + // CreateRawTable creates a raw table for the connector with a given name and a fixed schema. + CreateRawTable(req *protos.CreateRawTableInput) (*protos.CreateRawTableOutput, error) + + // SetupNormalizedTables sets up the normalized table on the connector. + SetupNormalizedTables(req *protos.SetupNormalizedTableBatchInput) ( + *protos.SetupNormalizedTableBatchOutput, error) + // SyncRecords pushes records to the destination peer and stores it in PeerDB specific tables. // This method should be idempotent, and should be able to be called multiple times with the same request. SyncRecords(req *model.SyncRecordsRequest) (*model.SyncResponse, error) + // SyncFlowCleanup drops metadata tables on the destination, as a part of DROP MIRROR. + SyncFlowCleanup(jobName string) error +} + +type CDCNormalizeConnector interface { + Connector + + // InitializeTableSchema initializes the table schema of all the destination tables for the connector. + InitializeTableSchema(req map[string]*protos.TableSchema) error + // NormalizeRecords merges records pushed earlier into the destination table. // This method should be idempotent, and should be able to be called multiple times with the same request. NormalizeRecords(req *model.NormalizeRecordsRequest) (*model.NormalizeResponse, error) - // CreateRawTable creates a raw table for the connector with a given name and a fixed schema. - CreateRawTable(req *protos.CreateRawTableInput) (*protos.CreateRawTableOutput, error) - - ///// QRep methods ///// + // ReplayTableSchemaDelta changes a destination table to match the schema at source + // This could involve adding or dropping multiple columns. + ReplayTableSchemaDelta(flowJobName string, schemaDelta *protos.TableSchemaDelta) error +} - // SetupQRepMetadataTables sets up the metadata tables for QRep. - SetupQRepMetadataTables(config *protos.QRepConfig) error +type QRepPullConnector interface { + Connector // GetQRepPartitions returns the partitions for a given table that haven't been synced yet. GetQRepPartitions(config *protos.QRepConfig, last *protos.QRepPartition) ([]*protos.QRepPartition, error) // GetQRepRecords returns the records for a given partition. PullQRepRecords(config *protos.QRepConfig, partition *protos.QRepPartition) (*model.QRecordBatch, error) +} + +type QRepSyncConnector interface { + Connector + + // SetupQRepMetadataTables sets up the metadata tables for QRep. + SetupQRepMetadataTables(config *protos.QRepConfig) error // SyncQRepRecords syncs the records for a given partition. // returns the number of records synced. - SyncQRepRecords( - config *protos.QRepConfig, - partition *protos.QRepPartition, - stream *model.QRecordStream, - ) (int, error) + SyncQRepRecords(config *protos.QRepConfig, partition *protos.QRepPartition, + stream *model.QRecordStream) (int, error) +} + +type QRepConsolidateConnector interface { + Connector // ConsolidateQRepPartitions consolidates the partitions for a given table. ConsolidateQRepPartitions(config *protos.QRepConfig) error // CleanupQRepFlow cleans up the QRep flow for a given table. CleanupQRepFlow(config *protos.QRepConfig) error +} - PullFlowCleanup(jobName string) error - SyncFlowCleanup(jobName string) error +func GetCDCPullConnector(ctx context.Context, config *protos.Peer) (CDCPullConnector, error) { + inner := config.Config + switch inner.(type) { + case *protos.Peer_PostgresConfig: + return connpostgres.NewPostgresConnector(ctx, config.GetPostgresConfig()) + default: + return nil, ErrUnsupportedFunctionality + } } -func GetConnector(ctx context.Context, config *protos.Peer) (Connector, error) { +func GetCDCSyncConnector(ctx context.Context, config *protos.Peer) (CDCSyncConnector, error) { inner := config.Config switch inner.(type) { case *protos.Peer_PostgresConfig: @@ -94,14 +139,63 @@ 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()) - case *protos.Peer_S3Config: - return conns3.NewS3Connector(ctx, config.GetS3Config()) + default: + return nil, ErrUnsupportedFunctionality + } +} + +func GetCDCNormalizeConnector(ctx context.Context, + config *protos.Peer) (CDCNormalizeConnector, error) { + inner := config.Config + switch inner.(type) { + case *protos.Peer_PostgresConfig: + return connpostgres.NewPostgresConnector(ctx, config.GetPostgresConfig()) + case *protos.Peer_BigqueryConfig: + return connbigquery.NewBigQueryConnector(ctx, config.GetBigqueryConfig()) + case *protos.Peer_SnowflakeConfig: + return connsnowflake.NewSnowflakeConnector(ctx, config.GetSnowflakeConfig()) + default: + return nil, ErrUnsupportedFunctionality + } +} + +func GetQRepPullConnector(ctx context.Context, config *protos.Peer) (QRepPullConnector, error) { + inner := config.Config + switch inner.(type) { + case *protos.Peer_PostgresConfig: + return connpostgres.NewPostgresConnector(ctx, config.GetPostgresConfig()) case *protos.Peer_SqlserverConfig: return connsqlserver.NewSQLServerConnector(ctx, config.GetSqlserverConfig()) default: - return nil, fmt.Errorf("requested connector is not yet implemented") + return nil, ErrUnsupportedFunctionality + } +} + +func GetQRepSyncConnector(ctx context.Context, config *protos.Peer) (QRepSyncConnector, error) { + inner := config.Config + switch inner.(type) { + case *protos.Peer_PostgresConfig: + return connpostgres.NewPostgresConnector(ctx, config.GetPostgresConfig()) + case *protos.Peer_BigqueryConfig: + return connbigquery.NewBigQueryConnector(ctx, config.GetBigqueryConfig()) + case *protos.Peer_SnowflakeConfig: + return connsnowflake.NewSnowflakeConnector(ctx, config.GetSnowflakeConfig()) + case *protos.Peer_S3Config: + return conns3.NewS3Connector(ctx, config.GetS3Config()) + default: + return nil, ErrUnsupportedFunctionality + } +} + +func GetQRepConsolidateConnector(ctx context.Context, + config *protos.Peer) (QRepConsolidateConnector, error) { + inner := config.Config + switch inner.(type) { + case *protos.Peer_SnowflakeConfig: + return connsnowflake.NewSnowflakeConnector(ctx, config.GetSnowflakeConfig()) + + default: + return nil, ErrUnsupportedFunctionality } } diff --git a/flow/connectors/eventhub/eventhub.go b/flow/connectors/eventhub/eventhub.go index 70c1adafa8..6ca63c0251 100644 --- a/flow/connectors/eventhub/eventhub.go +++ b/flow/connectors/eventhub/eventhub.go @@ -89,25 +89,11 @@ func (c *EventHubConnector) ConnectionActive() bool { return true } -func (c *EventHubConnector) EnsurePullability( - req *protos.EnsurePullabilityBatchInput) (*protos.EnsurePullabilityBatchOutput, error) { - panic("ensure pullability not implemented for event hub") -} - func (c *EventHubConnector) InitializeTableSchema(req map[string]*protos.TableSchema) error { c.tableSchemas = req return nil } -func (c *EventHubConnector) ReplayTableSchemaDelta(flowJobName string, schemaDelta *protos.TableSchemaDelta) error { - log.Warnf("ReplayTableSchemaDelta is a no-op for EventHub flow connector") - return nil -} - -func (c *EventHubConnector) PullRecords(req *model.PullRecordsRequest) (*model.RecordsWithTableSchemaDelta, error) { - panic("pull records not implemented for event hub") -} - func (c *EventHubConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.SyncResponse, error) { shutdown := utils.HeartbeatRoutine(c.ctx, 1*time.Minute, func() string { return fmt.Sprintf("syncing records to eventhub with"+ @@ -170,11 +156,16 @@ func (c *EventHubConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S "flowName": req.FlowJobName, }).Infof("[total] successfully sent %d records to event hub", len(batch.Records)) - err := c.UpdateLastOffset(req.FlowJobName, batch.LastCheckPointID) + err := c.updateLastOffset(req.FlowJobName, batch.LastCheckPointID) if err != nil { log.Errorf("failed to update last offset: %v", err) return nil, err } + err = c.incrementSyncBatchID(req.FlowJobName) + if err != nil { + log.Errorf("%v", err) + return nil, err + } return &model.SyncResponse{ FirstSyncedCheckPointID: batch.FirstCheckPointID, @@ -280,11 +271,6 @@ func (c *EventHubConnector) CreateRawTable(req *protos.CreateRawTableInput) (*pr return nil, nil } -func (c *EventHubConnector) GetTableSchema( - req *protos.GetTableSchemaBatchInput) (*protos.GetTableSchemaBatchOutput, error) { - panic("get table schema not implemented for event hub") -} - func (c *EventHubConnector) ensureEventHub(ctx context.Context, name string, flowName string) error { hubClient, err := c.getEventHubMgmtClient() if err != nil { @@ -337,32 +323,3 @@ func (c *EventHubConnector) getEventHubMgmtClient() (*armeventhub.EventHubsClien return hubClient, nil } - -// Normalization - -func (c *EventHubConnector) SetupNormalizedTables( - req *protos.SetupNormalizedTableBatchInput) ( - *protos.SetupNormalizedTableBatchOutput, 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 &model.NormalizeResponse{ - EndBatchID: 0, - StartBatchID: 0, - Done: true, - }, 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 index 07bf30b339..72410bce62 100644 --- a/flow/connectors/eventhub/metadata.go +++ b/flow/connectors/eventhub/metadata.go @@ -2,7 +2,6 @@ package conneventhub import ( "context" - "fmt" "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/generated/protos" @@ -88,7 +87,8 @@ func (c *EventHubConnector) SetupMetadataTables() error { 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() + updated_at TIMESTAMP NOT NULL DEFAULT NOW(), + sync_batch_id BIGINT NOT NULL ) `) if err != nil { @@ -139,14 +139,35 @@ func (c *EventHubConnector) GetLastOffset(jobName string) (*protos.LastSyncState } func (c *EventHubConnector) GetLastSyncBatchID(jobName string) (int64, error) { - log.WithFields(log.Fields{ - "flowName": jobName, - }).Errorf("GetLastSyncBatchID not supported for EventHub") - return 0, fmt.Errorf("GetLastSyncBatchID not supported for EventHub connector") + ms := c.pgMetadata + + rows := ms.pool.QueryRow(c.ctx, ` + SELECT sync_batch_id + FROM `+metadataSchema+`.`+lastSyncStateTableName+` + WHERE job_name = $1 + `, jobName) + + var syncBatchID int64 + err := rows.Scan(&syncBatchID) + if err != nil { + // if the job doesn't exist, return 0 + if err.Error() == "no rows in result set" { + return 0, nil + } + + log.WithFields(log.Fields{ + "flowName": jobName, + }).Errorf("failed to get last offset: %v", err) + return 0, err + } + + log.Infof("got last sync batch ID for job `%s`: %d", jobName, syncBatchID) + + return syncBatchID, nil } // update offset for a job -func (c *EventHubConnector) UpdateLastOffset(jobName string, offset int64) error { +func (c *EventHubConnector) updateLastOffset(jobName string, offset int64) error { ms := c.pgMetadata // start a transaction @@ -161,11 +182,11 @@ func (c *EventHubConnector) UpdateLastOffset(jobName string, offset int64) error "flowName": jobName, }).Infof("updating last offset for job `%s` to `%d`", jobName, offset) _, err = tx.Exec(c.ctx, ` - INSERT INTO `+metadataSchema+`.`+lastSyncStateTableName+` (job_name, last_offset) - VALUES ($1, $2) + INSERT INTO `+metadataSchema+`.`+lastSyncStateTableName+` (job_name, last_offset, sync_batch_id) + VALUES ($1, $2, $3) ON CONFLICT (job_name) DO UPDATE SET last_offset = $2, updated_at = NOW() - `, jobName, offset) + `, jobName, offset, 0) if err != nil { log.WithFields(log.Fields{ @@ -183,3 +204,25 @@ func (c *EventHubConnector) UpdateLastOffset(jobName string, offset int64) error return nil } + +// update offset for a job +func (c *EventHubConnector) incrementSyncBatchID(jobName string) error { + ms := c.pgMetadata + + log.WithFields(log.Fields{ + "flowName": jobName, + }).Infof("incrementing sync batch id for job `%s`", jobName) + _, err := ms.pool.Exec(c.ctx, ` + UPDATE `+metadataSchema+`.`+lastSyncStateTableName+` + SET sync_batch_id=sync_batch_id+1 WHERE job_name=$1 + `, jobName) + + if err != nil { + log.WithFields(log.Fields{ + "flowName": jobName, + }).Errorf("failed to increment sync batch id: %v", err) + return err + } + + return nil +} diff --git a/flow/connectors/eventhub/qrep.go b/flow/connectors/eventhub/qrep.go deleted file mode 100644 index 42a7252d7d..0000000000 --- a/flow/connectors/eventhub/qrep.go +++ /dev/null @@ -1,33 +0,0 @@ -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.QRecordStream) (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/qrep.go b/flow/connectors/postgres/qrep.go index 6d44e25bc9..e92ec8116a 100644 --- a/flow/connectors/postgres/qrep.go +++ b/flow/connectors/postgres/qrep.go @@ -509,19 +509,6 @@ func BuildQuery(query string, flowJobName string) (string, error) { return res, nil } -func (c *PostgresConnector) ConsolidateQRepPartitions(config *protos.QRepConfig) error { - log.Infof("Consolidating partitions for flow job %s", config.FlowJobName) - log.Infof("This is a no-op for Postgres") - return nil -} - -// CleanupQRepFlow function for postgres connector -func (c *PostgresConnector) CleanupQRepFlow(config *protos.QRepConfig) error { - log.Infof("Cleaning up QRep flow for flow job %s", config.FlowJobName) - log.Infof("This is a no-op for Postgres") - return nil -} - // isPartitionSynced checks whether a specific partition is synced func (c *PostgresConnector) isPartitionSynced(partitionID string) (bool, error) { // setup the query string diff --git a/flow/connectors/s3/qrep.go b/flow/connectors/s3/qrep.go index ba051e0c66..e3b30bd4e4 100644 --- a/flow/connectors/s3/qrep.go +++ b/flow/connectors/s3/qrep.go @@ -10,18 +10,6 @@ import ( log "github.com/sirupsen/logrus" ) -func (c *S3Connector) GetQRepPartitions(config *protos.QRepConfig, - last *protos.QRepPartition, -) ([]*protos.QRepPartition, error) { - panic("not implemented for s3") -} - -func (c *S3Connector) PullQRepRecords(config *protos.QRepConfig, - partition *protos.QRepPartition, -) (*model.QRecordBatch, error) { - panic("not implemented for s3") -} - func (c *S3Connector) SyncQRepRecords( config *protos.QRepConfig, partition *protos.QRepPartition, @@ -88,13 +76,3 @@ func (c *S3Connector) SetupQRepMetadataTables(config *protos.QRepConfig) error { log.Infof("QRep metadata setup not needed for S3.") return nil } - -func (c *S3Connector) ConsolidateQRepPartitions(config *protos.QRepConfig) error { - log.Infof("Consolidate partitions not needed for S3.") - return nil -} - -func (c *S3Connector) CleanupQRepFlow(config *protos.QRepConfig) error { - log.Infof("Cleanup QRep Flow not needed for S3.") - return nil -} diff --git a/flow/connectors/s3/s3.go b/flow/connectors/s3/s3.go index 39bf5297ab..17e9fb49ab 100644 --- a/flow/connectors/s3/s3.go +++ b/flow/connectors/s3/s3.go @@ -6,7 +6,6 @@ import ( "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/aws/aws-sdk-go/service/s3" log "github.com/sirupsen/logrus" ) @@ -39,86 +38,3 @@ func (c *S3Connector) ConnectionActive() bool { _, err := c.client.ListBuckets(nil) return err == nil } - -func (c *S3Connector) NeedsSetupMetadataTables() bool { - log.Errorf("NeedsSetupMetadataTables not supported for S3") - return false -} - -func (c *S3Connector) SetupMetadataTables() error { - log.Errorf("SetupMetadataTables not supported for S3") - return fmt.Errorf("cdc based replication is not currently supported for S3 target") -} - -func (c *S3Connector) GetLastOffset(jobName string) (*protos.LastSyncState, error) { - log.Errorf("GetLastOffset not supported for S3") - return nil, fmt.Errorf("cdc based replication is not currently supported for S3 target") -} - -func (c *S3Connector) GetLastSyncBatchID(jobName string) (int64, error) { - log.Errorf("GetLastSyncBatchID not supported for S3") - return 0, fmt.Errorf("cdc based replication is not currently supported for S3 target") -} - -func (c *S3Connector) GetLastNormalizeBatchID() (int64, error) { - log.Errorf("GetLastNormalizeBatchID not supported for S3") - return 0, fmt.Errorf("cdc based replication is not currently supported for S3 target") -} - -func (c *S3Connector) GetTableSchema( - req *protos.GetTableSchemaBatchInput) (*protos.GetTableSchemaBatchOutput, error) { - log.Errorf("GetTableSchema not supported for S3 flow connector") - return nil, fmt.Errorf("cdc based replication is not currently supported for S3 target") -} - -func (c *S3Connector) SetupNormalizedTables(req *protos.SetupNormalizedTableBatchInput) ( - *protos.SetupNormalizedTableBatchOutput, error) { - log.Errorf("SetupNormalizedTable not supported for S3") - return nil, fmt.Errorf("cdc based replication is not currently supported for S3 target") -} - -func (c *S3Connector) InitializeTableSchema(req map[string]*protos.TableSchema) error { - log.Errorf("InitializeTableSchema not supported for S3") - return fmt.Errorf("cdc based replication is not currently supported for S3 target") -} - -func (c *S3Connector) ReplayTableSchemaDelta(flowJobName string, schemaDelta *protos.TableSchemaDelta) error { - log.Warnf("ReplayTableSchemaDelta is a no-op for S3 flow connector") - return nil -} - -func (c *S3Connector) PullRecords(req *model.PullRecordsRequest) (*model.RecordsWithTableSchemaDelta, error) { - log.Errorf("panicking at call to PullRecords for S3 flow connector") - panic("PullRecords is not implemented for the S3 flow connector") -} - -func (c *S3Connector) SyncRecords(req *model.SyncRecordsRequest) (*model.SyncResponse, error) { - log.Errorf("SyncRecords not supported for S3") - return nil, fmt.Errorf("cdc based replication is not currently supported for S3 target") -} - -func (c *S3Connector) NormalizeRecords(req *model.NormalizeRecordsRequest) (*model.NormalizeResponse, error) { - log.Errorf("NormalizeRecords not supported for S3") - return nil, fmt.Errorf("cdc based replication is not currently supported for S3 target") -} - -func (c *S3Connector) CreateRawTable(req *protos.CreateRawTableInput) (*protos.CreateRawTableOutput, error) { - log.Errorf("CreateRawTable not supported for S3") - return nil, fmt.Errorf("cdc based replication is not currently supported for S3 target") -} - -func (c *S3Connector) EnsurePullability(req *protos.EnsurePullabilityBatchInput, -) (*protos.EnsurePullabilityBatchOutput, error) { - log.Errorf("panicking at call to EnsurePullability for S3 flow connector") - panic("EnsurePullability is not implemented for the S3 flow connector") -} - -func (c *S3Connector) PullFlowCleanup(jobName string) error { - log.Errorf("panicking at call to PullFlowCleanup for S3 flow connector") - panic("PullFlowCleanup is not implemented for the S3 flow connector") -} - -func (c *S3Connector) SyncFlowCleanup(jobName string) error { - log.Errorf("SyncFlowCleanup not supported for S3") - return fmt.Errorf("cdc based replication is not currently supported for S3 target") -} diff --git a/flow/connectors/snowflake/qrep.go b/flow/connectors/snowflake/qrep.go index 3f795600ca..d3fe08ab0e 100644 --- a/flow/connectors/snowflake/qrep.go +++ b/flow/connectors/snowflake/qrep.go @@ -18,18 +18,6 @@ import ( const qRepMetadataTableName = "_peerdb_query_replication_metadata" -func (c *SnowflakeConnector) GetQRepPartitions(config *protos.QRepConfig, - last *protos.QRepPartition, -) ([]*protos.QRepPartition, error) { - panic("not implemented") -} - -func (c *SnowflakeConnector) PullQRepRecords(config *protos.QRepConfig, - partition *protos.QRepPartition, -) (*model.QRecordBatch, error) { - panic("not implemented") -} - func (c *SnowflakeConnector) SyncQRepRecords( config *protos.QRepConfig, partition *protos.QRepPartition, diff --git a/flow/connectors/snowflake/snowflake.go b/flow/connectors/snowflake/snowflake.go index b2aa542e75..79b23805be 100644 --- a/flow/connectors/snowflake/snowflake.go +++ b/flow/connectors/snowflake/snowflake.go @@ -322,12 +322,6 @@ func (c *SnowflakeConnector) getTableNametoUnchangedCols(flowJobName string, syn return resultMap, nil } -func (c *SnowflakeConnector) GetTableSchema( - req *protos.GetTableSchemaBatchInput) (*protos.GetTableSchemaBatchOutput, error) { - log.Errorf("panicking at call to GetTableSchema for Snowflake flow connector") - panic("GetTableSchema is not implemented for the Snowflake flow connector") -} - func (c *SnowflakeConnector) SetupNormalizedTables( req *protos.SetupNormalizedTableBatchInput) (*protos.SetupNormalizedTableBatchOutput, error) { tableExistsMapping := make(map[string]bool) @@ -421,11 +415,6 @@ func (c *SnowflakeConnector) ReplayTableSchemaDelta(flowJobName string, schemaDe return nil } -func (c *SnowflakeConnector) PullRecords(req *model.PullRecordsRequest) (*model.RecordsWithTableSchemaDelta, error) { - log.Errorf("panicking at call to PullRecords for Snowflake flow connector") - panic("PullRecords is not implemented for the Snowflake flow connector") -} - func (c *SnowflakeConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.SyncResponse, error) { if len(req.Records.Records) == 0 { return &model.SyncResponse{ @@ -932,18 +921,6 @@ func (c *SnowflakeConnector) CreateRawTable(req *protos.CreateRawTableInput) (*p }, nil } -// EnsurePullability ensures that the table is pullable, implementing the Connector interface. -func (c *SnowflakeConnector) EnsurePullability(req *protos.EnsurePullabilityBatchInput, -) (*protos.EnsurePullabilityBatchOutput, error) { - log.Errorf("panicking at call to EnsurePullability for Snowflake flow connector") - panic("EnsurePullability is not implemented for the Snowflake flow connector") -} - -func (c *SnowflakeConnector) PullFlowCleanup(jobName string) error { - log.Errorf("panicking at call to PullFlowCleanup for Snowflake flow connector") - panic("PullFlowCleanup is not implemented for the Snowflake flow connector") -} - func (c *SnowflakeConnector) SyncFlowCleanup(jobName string) error { syncFlowCleanupTx, err := c.database.BeginTx(c.ctx, nil) if err != nil { diff --git a/flow/connectors/sqlserver/qrep.go b/flow/connectors/sqlserver/qrep.go index fa2f49ceb6..522648fb1c 100644 --- a/flow/connectors/sqlserver/qrep.go +++ b/flow/connectors/sqlserver/qrep.go @@ -13,11 +13,6 @@ import ( log "github.com/sirupsen/logrus" ) -func (c *SQLServerConnector) SetupQRepMetadataTables(config *protos.QRepConfig) error { - log.Infof("Setting up metadata tables for query replication on sql server is a no-op") - return nil -} - func (c *SQLServerConnector) GetQRepPartitions( config *protos.QRepConfig, last *protos.QRepPartition) ([]*protos.QRepPartition, error) { if config.WatermarkTable == "" { @@ -214,19 +209,3 @@ func BuildQuery(query string) (string, error) { log.Infof("templated query: %s", res) return res, nil } - -func (c *SQLServerConnector) SyncQRepRecords( - config *protos.QRepConfig, - partition *protos.QRepPartition, - stream *model.QRecordStream, -) (int, error) { - panic("not implemented") -} - -func (c *SQLServerConnector) ConsolidateQRepPartitions(config *protos.QRepConfig) error { - panic("not implemented") -} - -func (c *SQLServerConnector) CleanupQRepFlow(config *protos.QRepConfig) error { - panic("not implemented") -} diff --git a/flow/connectors/sqlserver/sqlserver.go b/flow/connectors/sqlserver/sqlserver.go index feb15e8a47..5a529c9408 100644 --- a/flow/connectors/sqlserver/sqlserver.go +++ b/flow/connectors/sqlserver/sqlserver.go @@ -6,10 +6,8 @@ import ( peersql "github.com/PeerDB-io/peer-flow/connectors/sql" "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/model" "github.com/jmoiron/sqlx" _ "github.com/microsoft/go-mssqldb" - log "github.com/sirupsen/logrus" ) type SQLServerConnector struct { @@ -61,87 +59,3 @@ func (c *SQLServerConnector) ConnectionActive() bool { } return true } - -func (c *SQLServerConnector) NeedsSetupMetadataTables() bool { - log.Errorf("NeedsSetupMetadataTables not supported for SQLServer") - return false -} - -func (c *SQLServerConnector) SetupMetadataTables() error { - log.Errorf("SetupMetadataTables not supported for SQLServer") - return fmt.Errorf("cdc based replication is not currently supported for SQLServer target") -} - -func (c *SQLServerConnector) GetLastOffset(jobName string) (*protos.LastSyncState, error) { - log.Errorf("GetLastOffset not supported for SQLServer") - return nil, fmt.Errorf("cdc based replication is not currently supported for SQLServer target") -} - -func (c *SQLServerConnector) GetLastSyncBatchID(jobName string) (int64, error) { - log.Errorf("GetLastSyncBatchID not supported for SQLServer") - return 0, fmt.Errorf("cdc based replication is not currently supported for SQLServer target") -} - -func (c *SQLServerConnector) GetLastNormalizeBatchID() (int64, error) { - log.Errorf("GetLastNormalizeBatchID not supported for SQLServer") - return 0, fmt.Errorf("cdc based replication is not currently supported for SQLServer target") -} - -func (c *SQLServerConnector) GetTableSchema( - req *protos.GetTableSchemaBatchInput) (*protos.GetTableSchemaBatchOutput, error) { - log.Errorf("GetTableSchema not supported for SQLServer flow connector") - return nil, fmt.Errorf("cdc based replication is not currently supported for SQLServer target") -} - -func (c *SQLServerConnector) SetupNormalizedTables( - req *protos.SetupNormalizedTableBatchInput) ( - *protos.SetupNormalizedTableBatchOutput, error) { - log.Errorf("SetupNormalizedTable not supported for SQLServer") - return nil, fmt.Errorf("cdc based replication is not currently supported for SQLServer target") -} - -func (c *SQLServerConnector) InitializeTableSchema(req map[string]*protos.TableSchema) error { - log.Errorf("InitializeTableSchema not supported for SQLServer") - return fmt.Errorf("cdc based replication is not currently supported for SQLServer target") -} - -func (c *SQLServerConnector) ReplayTableSchemaDelta(flowJobName string, schemaDelta *protos.TableSchemaDelta) error { - log.Warnf("ReplayTableSchemaDelta is a no-op for SQLServer flow connector") - return nil -} - -func (c *SQLServerConnector) PullRecords(req *model.PullRecordsRequest) (*model.RecordsWithTableSchemaDelta, error) { - log.Errorf("panicking at call to PullRecords for SQLServer flow connector") - panic("PullRecords is not implemented for the SQLServer flow connector") -} - -func (c *SQLServerConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.SyncResponse, error) { - log.Errorf("SyncRecords not supported for SQLServer") - return nil, fmt.Errorf("cdc based replication is not currently supported for SQLServer target") -} - -func (c *SQLServerConnector) NormalizeRecords(req *model.NormalizeRecordsRequest) (*model.NormalizeResponse, error) { - log.Errorf("NormalizeRecords not supported for SQLServer") - return nil, fmt.Errorf("cdc based replication is not currently supported for SQLServer target") -} - -func (c *SQLServerConnector) CreateRawTable(req *protos.CreateRawTableInput) (*protos.CreateRawTableOutput, error) { - log.Errorf("CreateRawTable not supported for SQLServer") - return nil, fmt.Errorf("cdc based replication is not currently supported for SQLServer target") -} - -func (c *SQLServerConnector) EnsurePullability(req *protos.EnsurePullabilityBatchInput, -) (*protos.EnsurePullabilityBatchOutput, error) { - log.Errorf("panicking at call to EnsurePullability for SQLServer flow connector") - panic("EnsurePullability is not implemented for the SQLServer flow connector") -} - -func (c *SQLServerConnector) PullFlowCleanup(jobName string) error { - log.Errorf("panicking at call to PullFlowCleanup for SQLServer flow connector") - panic("PullFlowCleanup is not implemented for the SQLServer flow connector") -} - -func (c *SQLServerConnector) SyncFlowCleanup(jobName string) error { - log.Errorf("SyncFlowCleanup not supported for SQLServer") - return fmt.Errorf("cdc based replication is not currently supported for SQLServer target") -} From ffa556d5abd0da8124371239c9c21b186c29fd69 Mon Sep 17 00:00:00 2001 From: Kevin Biju Date: Thu, 7 Sep 2023 10:07:49 +0530 Subject: [PATCH 2/4] fixing Eventhub fpr CDC sync --- flow/connectors/core.go | 3 +++ flow/connectors/eventhub/eventhub.go | 12 ++++++++++++ 2 files changed, 15 insertions(+) diff --git a/flow/connectors/core.go b/flow/connectors/core.go index dbc8171265..9810fe6672 100644 --- a/flow/connectors/core.go +++ b/flow/connectors/core.go @@ -5,6 +5,7 @@ import ( "errors" 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" conns3 "github.com/PeerDB-io/peer-flow/connectors/s3" connsnowflake "github.com/PeerDB-io/peer-flow/connectors/snowflake" @@ -139,6 +140,8 @@ func GetCDCSyncConnector(ctx context.Context, config *protos.Peer) (CDCSyncConne 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, ErrUnsupportedFunctionality } diff --git a/flow/connectors/eventhub/eventhub.go b/flow/connectors/eventhub/eventhub.go index 6ca63c0251..4376036cba 100644 --- a/flow/connectors/eventhub/eventhub.go +++ b/flow/connectors/eventhub/eventhub.go @@ -323,3 +323,15 @@ func (c *EventHubConnector) getEventHubMgmtClient() (*armeventhub.EventHubsClien return hubClient, nil } + +func (c *EventHubConnector) SetupNormalizedTables( + req *protos.SetupNormalizedTableBatchInput) ( + *protos.SetupNormalizedTableBatchOutput, error) { + log.Infof("setting up tables for Eventhub is a no-op") + return nil, nil +} + +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") +} From 2c40f2ab6f4c688ac0ca5adf35609f27c244bc38 Mon Sep 17 00:00:00 2001 From: Kevin Biju Date: Tue, 12 Sep 2023 20:17:41 +0530 Subject: [PATCH 3/4] refactored PeerFlow -> CDCFlow --- flow/activities/fetch_config.go | 121 --- flow/cmd/handler.go | 24 +- flow/cmd/worker.go | 4 +- flow/e2e/bigquery/peer_flow_bq_test.go | 78 +- flow/e2e/eventhub/peer_flow_eh_test.go | 6 +- flow/e2e/postgres/peer_flow_pg_test.go | 12 +- flow/e2e/postgres/timescale.sql | 53 ++ flow/e2e/snowflake/peer_flow_sf_test.go | 66 +- .../e2e/sqlserver/qrep_flow_sqlserver_test.go | 2 +- flow/e2e/test_utils.go | 14 +- flow/generated/protos/route.pb.go | 195 +++-- flow/generated/protos/route_grpc.pb.go | 30 +- flow/shared/constants.go | 6 +- flow/workflows/activities.go | 5 +- flow/workflows/{peer_flow.go => cdc_flow.go} | 151 +--- flow/workflows/setup_flow.go | 28 +- flow/workflows/sync_flow.go | 22 +- nexus/pt/src/peerdb_route.rs | 4 +- nexus/pt/src/peerdb_route.serde.rs | 32 +- nexus/pt/src/peerdb_route.tonic.rs | 32 +- protos/route.proto | 6 +- ui/grpc_generated/flow.ts | 766 +++++++++++++++++- ui/grpc_generated/route.ts | 80 +- 23 files changed, 1167 insertions(+), 570 deletions(-) delete mode 100644 flow/activities/fetch_config.go create mode 100644 flow/e2e/postgres/timescale.sql rename flow/workflows/{peer_flow.go => cdc_flow.go} (69%) diff --git a/flow/activities/fetch_config.go b/flow/activities/fetch_config.go deleted file mode 100644 index 93e74ec1dd..0000000000 --- a/flow/activities/fetch_config.go +++ /dev/null @@ -1,121 +0,0 @@ -package activities - -import ( - "context" - "fmt" - - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/jackc/pgx/v5/pgxpool" - log "github.com/sirupsen/logrus" - "google.golang.org/protobuf/proto" -) - -// FetchConfigActivityInput is the input for the FetchConfigActivity. -type FetchConfigActivityInput struct { - // The JDBC URL for the catalog database. - CatalogJdbcURL string - // The name of the peer flow to fetch the config for. - PeerFlowName string -} - -// FetchConfigActivity is an activity that fetches the config for the specified peer flow. -// This activity is invoked by the PeerFlowWorkflow. -type FetchConfigActivity struct{} - -// FetchConfig retrieves the source and destination config. -func (a *FetchConfigActivity) FetchConfig( - ctx context.Context, - input *FetchConfigActivityInput, -) (*protos.FlowConnectionConfigs, error) { - pool, err := pgxpool.New(ctx, input.CatalogJdbcURL) - if err != nil { - return nil, fmt.Errorf("failed to create connection pool: %w", err) - } - - sourceConnectionConfig, err := FetchPeerConfig(ctx, pool, input.PeerFlowName, "source_peer") - if err != nil { - return nil, fmt.Errorf("failed to unmarshal source connection config: %w", err) - } - - destinationConnectionConfig, err := FetchPeerConfig(ctx, pool, input.PeerFlowName, "destination_peer") - if err != nil { - return nil, fmt.Errorf("failed to unmarshal destination connection config: %w", err) - } - - query := `SELECT source_table_identifier, destination_table_identifier FROM flows WHERE name = $1` - rows, err := pool.Query(ctx, query, input.PeerFlowName) - if err != nil { - return nil, fmt.Errorf("failed to fetch table identifiers: %w", err) - } - defer rows.Close() - // Create a map to store the mapping of source table to destination table - tableNameMapping := make(map[string]string) - var srcTableIdentifier, dstTableIdentifier string - - // Iterate over all the result rows - for rows.Next() { - err = rows.Scan(&srcTableIdentifier, &dstTableIdentifier) - if err != nil { - return nil, fmt.Errorf("error scanning row %w", err) - } - - // Store the tableNameMapping in the map - tableNameMapping[srcTableIdentifier] = dstTableIdentifier - } - - log.Printf("successfully fetched config for peer flow - %s", input.PeerFlowName) - - return &protos.FlowConnectionConfigs{ - Source: sourceConnectionConfig, - Destination: destinationConnectionConfig, - TableNameMapping: tableNameMapping, - }, nil -} - -// fetchPeerConfig retrieves the config for a given peer by join label. -func FetchPeerConfig(ctx context.Context, pool *pgxpool.Pool, flowName string, label string) (*protos.Peer, error) { - var name string - var dbtype int32 - var opts []byte - - query := fmt.Sprintf( - "SELECT e.name, e.type, e.options FROM flows f JOIN peers e ON f.%s = e.id WHERE f.name = $1", - label) - err := pool.QueryRow(ctx, query, flowName).Scan(&name, &dbtype, &opts) - if err != nil { - return nil, fmt.Errorf("failed to fetch config for %s: %w", label, err) - } - - res := &protos.Peer{ - Name: name, - Type: protos.DBType(dbtype), - } - - switch protos.DBType(dbtype) { - case protos.DBType_BIGQUERY: - var peerConfig protos.BigqueryConfig - err = proto.Unmarshal(opts, &peerConfig) - if err != nil { - return nil, fmt.Errorf("failed to unmarshal bigquery config: %w", err) - } - res.Config = &protos.Peer_BigqueryConfig{BigqueryConfig: &peerConfig} - case protos.DBType_POSTGRES: - var peerConfig protos.PostgresConfig - err = proto.Unmarshal(opts, &peerConfig) - if err != nil { - return nil, fmt.Errorf("failed to unmarshal postgres config: %w", err) - } - res.Config = &protos.Peer_PostgresConfig{PostgresConfig: &peerConfig} - case protos.DBType_SNOWFLAKE: - var peerConfig protos.SnowflakeConfig - err = proto.Unmarshal(opts, &peerConfig) - if err != nil { - return nil, fmt.Errorf("failed to unmarshal snowflake config: %w", err) - } - res.Config = &protos.Peer_SnowflakeConfig{SnowflakeConfig: &peerConfig} - default: - return nil, fmt.Errorf("unsupported database type: %d", dbtype) - } - - return res, nil -} diff --git a/flow/cmd/handler.go b/flow/cmd/handler.go index 2f16cb808c..0365c76a85 100644 --- a/flow/cmd/handler.go +++ b/flow/cmd/handler.go @@ -35,8 +35,8 @@ func (h *FlowRequestHandler) Close() { } } -func (h *FlowRequestHandler) CreatePeerFlow( - ctx context.Context, req *protos.CreatePeerFlowRequest) (*protos.CreatePeerFlowResponse, error) { +func (h *FlowRequestHandler) CreateCDCFlow( + ctx context.Context, req *protos.CreateCDCFlowRequest) (*protos.CreateCDCFlowResponse, error) { cfg := req.ConnectionConfigs workflowID := fmt.Sprintf("%s-peerflow-%s", cfg.FlowJobName, uuid.New()) workflowOptions := client.StartWorkflowOptions{ @@ -50,26 +50,26 @@ func (h *FlowRequestHandler) CreatePeerFlow( cfg.MaxBatchSize = uint32(maxBatchSize) } - limits := &peerflow.PeerFlowLimits{ + limits := &peerflow.CDCFlowLimits{ TotalSyncFlows: 0, TotalNormalizeFlows: 0, MaxBatchSize: maxBatchSize, } - state := peerflow.NewStartedPeerFlowState() + state := peerflow.NewCDCFlowState() _, err := h.temporalClient.ExecuteWorkflow( - ctx, // context - workflowOptions, // workflow start options - peerflow.PeerFlowWorkflowWithConfig, // workflow function - cfg, // workflow input - limits, // workflow limits - state, // workflow state + ctx, // context + workflowOptions, // workflow start options + peerflow.CDCFlowWorkflowWithConfig, // workflow function + cfg, // workflow input + limits, // workflow limits + state, // workflow state ) if err != nil { return nil, fmt.Errorf("unable to start PeerFlow workflow: %w", err) } - return &protos.CreatePeerFlowResponse{ + return &protos.CreateCDCFlowResponse{ WorflowId: workflowID, }, nil } @@ -112,7 +112,7 @@ func (h *FlowRequestHandler) ShutdownFlow( ctx, req.WorkflowId, "", - shared.PeerFlowSignalName, + shared.CDCFlowSignalName, shared.ShutdownSignal, ) if err != nil { diff --git a/flow/cmd/worker.go b/flow/cmd/worker.go index a6528fbd32..b1631e68d4 100644 --- a/flow/cmd/worker.go +++ b/flow/cmd/worker.go @@ -128,15 +128,13 @@ func WorkerMain(opts *WorkerOptions) error { defer c.Close() w := worker.New(c, shared.PeerFlowTaskQueue, worker.Options{}) - w.RegisterWorkflow(peerflow.PeerFlowWorkflow) - w.RegisterWorkflow(peerflow.PeerFlowWorkflowWithConfig) + w.RegisterWorkflow(peerflow.CDCFlowWorkflowWithConfig) w.RegisterWorkflow(peerflow.SyncFlowWorkflow) w.RegisterWorkflow(peerflow.SetupFlowWorkflow) w.RegisterWorkflow(peerflow.NormalizeFlowWorkflow) w.RegisterWorkflow(peerflow.QRepFlowWorkflow) w.RegisterWorkflow(peerflow.QRepPartitionWorkflow) w.RegisterWorkflow(peerflow.DropFlowWorkflow) - w.RegisterActivity(&activities.FetchConfigActivity{}) w.RegisterActivity(&activities.FlowableActivity{ EnableMetrics: opts.EnableMetrics, CatalogMirrorMonitor: &catalogMirrorMonitor, diff --git a/flow/e2e/bigquery/peer_flow_bq_test.go b/flow/e2e/bigquery/peer_flow_bq_test.go index 72acb12230..649f59ede9 100644 --- a/flow/e2e/bigquery/peer_flow_bq_test.go +++ b/flow/e2e/bigquery/peer_flow_bq_test.go @@ -95,12 +95,12 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Invalid_Connection_Config() { e2e.RegisterWorkflowsAndActivities(env) // TODO (kaushikiska): ensure flow name can only be alpha numeric and underscores. - limits := peerflow.PeerFlowLimits{ + limits := peerflow.CDCFlowLimits{ TotalSyncFlows: 1, MaxBatchSize: 1, } - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, nil, &limits, nil) + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, nil, &limits, nil) // Verify workflow completes s.True(env.IsWorkflowCompleted()) @@ -139,12 +139,12 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Complete_Flow_No_Data() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - limits := peerflow.PeerFlowLimits{ + limits := peerflow.CDCFlowLimits{ TotalSyncFlows: 1, MaxBatchSize: 1, } - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -183,12 +183,12 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Char_ColType_Error() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - limits := peerflow.PeerFlowLimits{ + limits := peerflow.CDCFlowLimits{ TotalSyncFlows: 1, MaxBatchSize: 1, } - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -230,7 +230,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Complete_Simple_Flow_BQ() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - limits := peerflow.PeerFlowLimits{ + limits := peerflow.CDCFlowLimits{ TotalSyncFlows: 2, MaxBatchSize: 100, } @@ -238,7 +238,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Complete_Simple_Flow_BQ() { // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup // and then insert 10 rows into the source table go func() { - e2e.SetupPeerFlowStatusQuery(env, connectionGen) + e2e.SetupCDCFlowStatusQuery(env, connectionGen) // insert 10 rows into the source table for i := 0; i < 10; i++ { testKey := fmt.Sprintf("test_key_%d", i) @@ -251,7 +251,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Complete_Simple_Flow_BQ() { fmt.Println("Inserted 10 rows into the source table") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -301,7 +301,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Toast_BQ() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - limits := peerflow.PeerFlowLimits{ + limits := peerflow.CDCFlowLimits{ TotalSyncFlows: 1, MaxBatchSize: 100, } @@ -309,7 +309,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Toast_BQ() { // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup // and execute a transaction touching toast columns go func() { - e2e.SetupPeerFlowStatusQuery(env, connectionGen) + e2e.SetupCDCFlowStatusQuery(env, connectionGen) /* Executing a transaction which 1. changes both toast column @@ -328,7 +328,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Toast_BQ() { fmt.Println("Executed a transaction touching toast columns") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -372,7 +372,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Toast_Nochanges_BQ() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - limits := peerflow.PeerFlowLimits{ + limits := peerflow.CDCFlowLimits{ TotalSyncFlows: 1, MaxBatchSize: 100, } @@ -380,7 +380,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Toast_Nochanges_BQ() { // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup // and execute a transaction touching toast columns go func() { - e2e.SetupPeerFlowStatusQuery(env, connectionGen) + e2e.SetupCDCFlowStatusQuery(env, connectionGen) /* transaction updating no rows */ _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` BEGIN; @@ -392,7 +392,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Toast_Nochanges_BQ() { fmt.Println("Executed a transaction touching toast columns") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -436,7 +436,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_1_BQ() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - limits := peerflow.PeerFlowLimits{ + limits := peerflow.CDCFlowLimits{ TotalSyncFlows: 1, MaxBatchSize: 100, } @@ -444,7 +444,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_1_BQ() { // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup // and execute a transaction touching toast columns go func() { - e2e.SetupPeerFlowStatusQuery(env, connectionGen) + e2e.SetupCDCFlowStatusQuery(env, connectionGen) //complex transaction with random DMLs on a table with toast columns _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` BEGIN; @@ -469,7 +469,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_1_BQ() { fmt.Println("Executed a transaction touching toast columns") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -512,7 +512,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_2_BQ() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - limits := peerflow.PeerFlowLimits{ + limits := peerflow.CDCFlowLimits{ TotalSyncFlows: 1, MaxBatchSize: 100, } @@ -520,7 +520,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_2_BQ() { // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup // and execute a transaction touching toast columns go func() { - e2e.SetupPeerFlowStatusQuery(env, connectionGen) + e2e.SetupCDCFlowStatusQuery(env, connectionGen) //complex transaction with random DMLs on a table with toast columns _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` BEGIN; @@ -539,7 +539,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_2_BQ() { fmt.Println("Executed a transaction touching toast columns") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -583,7 +583,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_3_BQ() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - limits := peerflow.PeerFlowLimits{ + limits := peerflow.CDCFlowLimits{ TotalSyncFlows: 1, MaxBatchSize: 100, } @@ -591,7 +591,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_3_BQ() { // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup // and execute a transaction touching toast columns go func() { - e2e.SetupPeerFlowStatusQuery(env, connectionGen) + e2e.SetupCDCFlowStatusQuery(env, connectionGen) /* transaction updating a single row multiple times with changed/unchanged toast columns @@ -609,7 +609,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_3_BQ() { fmt.Println("Executed a transaction touching toast columns") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -658,7 +658,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Types_BQ() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - limits := peerflow.PeerFlowLimits{ + limits := peerflow.CDCFlowLimits{ TotalSyncFlows: 1, MaxBatchSize: 100, @@ -667,7 +667,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Types_BQ() { // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup // and execute a transaction touching toast columns go func() { - e2e.SetupPeerFlowStatusQuery(env, connectionGen) + e2e.SetupCDCFlowStatusQuery(env, connectionGen) /* test inserting various types*/ _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s SELECT 2,2,b'1',b'101', @@ -687,7 +687,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Types_BQ() { fmt.Println("Executed an insert with all types") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -747,7 +747,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Types_Avro_BQ() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - limits := peerflow.PeerFlowLimits{ + limits := peerflow.CDCFlowLimits{ TotalSyncFlows: 1, MaxBatchSize: 100, @@ -756,7 +756,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Types_Avro_BQ() { // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup // and execute a transaction touching toast columns go func() { - e2e.SetupPeerFlowStatusQuery(env, connectionGen) + e2e.SetupCDCFlowStatusQuery(env, connectionGen) /* test inserting various types*/ _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s SELECT 2,2,b'1',b'101', @@ -776,7 +776,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Types_Avro_BQ() { fmt.Println("Executed an insert with all types") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -826,13 +826,13 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Simple_Flow_BQ_Avro_CDC() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - limits := peerflow.PeerFlowLimits{ + limits := peerflow.CDCFlowLimits{ TotalSyncFlows: 2, MaxBatchSize: 100, } go func() { - e2e.SetupPeerFlowStatusQuery(env, connectionGen) + e2e.SetupCDCFlowStatusQuery(env, connectionGen) for i := 0; i < 10; i++ { testKey := fmt.Sprintf("test_key_%d", i) testValue := fmt.Sprintf("test_value_%d", i) @@ -844,7 +844,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Simple_Flow_BQ_Avro_CDC() { fmt.Println("Inserted 10 rows into the source table") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -889,7 +889,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Multi_Table_BQ() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - limits := peerflow.PeerFlowLimits{ + limits := peerflow.CDCFlowLimits{ TotalSyncFlows: 1, MaxBatchSize: 100, } @@ -897,7 +897,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Multi_Table_BQ() { // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup // and execute a transaction touching toast columns go func() { - e2e.SetupPeerFlowStatusQuery(env, connectionGen) + e2e.SetupCDCFlowStatusQuery(env, connectionGen) /* inserting across multiple tables*/ _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s (c1,c2) VALUES (1,'dummy_1'); @@ -907,7 +907,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Multi_Table_BQ() { fmt.Println("Executed an insert on two tables") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error require.True(s.T(), env.IsWorkflowCompleted()) @@ -949,7 +949,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Simple_Schema_Changes_BQ() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - limits := peerflow.PeerFlowLimits{ + limits := peerflow.CDCFlowLimits{ TotalSyncFlows: 10, MaxBatchSize: 100, } @@ -958,7 +958,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Simple_Schema_Changes_BQ() { // and then insert and mutate schema repeatedly. go func() { // insert first row. - e2e.SetupPeerFlowStatusQuery(env, connectionGen) + e2e.SetupCDCFlowStatusQuery(env, connectionGen) _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c1) VALUES ($1)`, srcTableName), 1) s.NoError(err) @@ -1015,7 +1015,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Simple_Schema_Changes_BQ() { s.compareTableContentsBQ("test_simple_schema_changes", "id,c1") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) diff --git a/flow/e2e/eventhub/peer_flow_eh_test.go b/flow/e2e/eventhub/peer_flow_eh_test.go index 687f17face..5462975508 100644 --- a/flow/e2e/eventhub/peer_flow_eh_test.go +++ b/flow/e2e/eventhub/peer_flow_eh_test.go @@ -117,7 +117,7 @@ func (s *PeerFlowE2ETestSuiteEH) Test_Complete_Simple_Flow_EH() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - peerFlowInput := peerflow.PeerFlowLimits{ + peerFlowInput := peerflow.CDCFlowLimits{ TotalSyncFlows: 2, MaxBatchSize: 100, } @@ -125,7 +125,7 @@ func (s *PeerFlowE2ETestSuiteEH) Test_Complete_Simple_Flow_EH() { // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup // and then insert 10 rows into the source table go func() { - e2e.SetupPeerFlowStatusQuery(env, connectionGen) + e2e.SetupCDCFlowStatusQuery(env, connectionGen) // insert 10 rows into the source table for i := 0; i < 10; i++ { testKey := fmt.Sprintf("test_key_%d", i) @@ -138,7 +138,7 @@ func (s *PeerFlowE2ETestSuiteEH) Test_Complete_Simple_Flow_EH() { fmt.Println("Inserted 10 rows into the source table") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &peerFlowInput, nil) + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &peerFlowInput, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) diff --git a/flow/e2e/postgres/peer_flow_pg_test.go b/flow/e2e/postgres/peer_flow_pg_test.go index ec120a1e8c..8ee321ee95 100644 --- a/flow/e2e/postgres/peer_flow_pg_test.go +++ b/flow/e2e/postgres/peer_flow_pg_test.go @@ -42,7 +42,7 @@ func (s *PeerFlowE2ETestSuitePG) Test_Simple_Flow_PG() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - limits := peerflow.PeerFlowLimits{ + limits := peerflow.CDCFlowLimits{ TotalSyncFlows: 2, MaxBatchSize: 100, } @@ -50,7 +50,7 @@ func (s *PeerFlowE2ETestSuitePG) Test_Simple_Flow_PG() { // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup // and then insert 10 rows into the source table go func() { - e2e.SetupPeerFlowStatusQuery(env, connectionGen) + e2e.SetupCDCFlowStatusQuery(env, connectionGen) // insert 10 rows into the source table for i := 0; i < 10; i++ { testKey := fmt.Sprintf("test_key_%d", i) @@ -63,7 +63,7 @@ func (s *PeerFlowE2ETestSuitePG) Test_Simple_Flow_PG() { fmt.Println("Inserted 10 rows into the source table") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -104,7 +104,7 @@ func (s *PeerFlowE2ETestSuitePG) Test_Simple_Schema_Changes_PG() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - limits := peerflow.PeerFlowLimits{ + limits := peerflow.CDCFlowLimits{ TotalSyncFlows: 10, MaxBatchSize: 100, } @@ -113,7 +113,7 @@ func (s *PeerFlowE2ETestSuitePG) Test_Simple_Schema_Changes_PG() { // and then insert and mutate schema repeatedly. go func() { // insert first row. - e2e.SetupPeerFlowStatusQuery(env, connectionGen) + e2e.SetupCDCFlowStatusQuery(env, connectionGen) _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c1) VALUES ($1)`, srcTableName), 1) s.NoError(err) @@ -170,7 +170,7 @@ func (s *PeerFlowE2ETestSuitePG) Test_Simple_Schema_Changes_PG() { s.NoError(err) }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) diff --git a/flow/e2e/postgres/timescale.sql b/flow/e2e/postgres/timescale.sql new file mode 100644 index 0000000000..e635cba839 --- /dev/null +++ b/flow/e2e/postgres/timescale.sql @@ -0,0 +1,53 @@ +CREATE PEER source_pg_2 FROM POSTGRES WITH +( + host = 'kevin-test-cluster.ctwiqpycdrx0.us-east-2.rds.amazonaws.com', + port = '5432', + user = 'postgres', + password = 'SUMM3RN!GHTS', + database = 'ts2' +); + +CREATE PEER target_ts_2 FROM POSTGRES WITH +( + host = '3.19.228.194', + port = '5432', + user = 'postgres', + password = 'T1mesc@l3', + database = 'dst2' +); + +CREATE TABLE public.diagnostics ( + id bigint, + "time" timestamp with time zone, + tags_id integer, + fuel_state double precision, + current_load double precision, + status double precision, + additional_tags jsonb, + primary key(id, "time") +); + +SELECT create_hypertable('diagnostics', 'time', chunk_time_interval => INTERVAL '12 hours'); + +CREATE TABLE public.readings ( + id bigint, + "time" timestamp with time zone, + tags_id integer, + latitude double precision, + longitude double precision, + elevation double precision, + velocity double precision, + heading double precision, + grade double precision, + fuel_consumption double precision, + additional_tags jsonb, + primary key(id, "time") +); + +SELECT create_hypertable('readings', 'time', chunk_time_interval => INTERVAL '12 hours'); + +CREATE MIRROR tstsv4 FROM source_pg_2 TO target_ts_2 WITH TABLE MAPPING(public.diagnostics:public.diagnostics,public.readings:public.readings); + +flow_worker1 | time="2023-08-30T06:47:18Z" level=info msg="RelationMessage => RelationID: 16747, Namespace: public, RelationName: fss1, Columns: [0x400175e360 0x400175e380]" +flow_worker1 | time="2023-08-30T06:47:18Z" level=info msg="23 1 id -1\n" +flow_worker1 | time="2023-08-30T06:47:18Z" level=info msg="20 0 c1 -1\n" diff --git a/flow/e2e/snowflake/peer_flow_sf_test.go b/flow/e2e/snowflake/peer_flow_sf_test.go index b6090a9df0..796402d033 100644 --- a/flow/e2e/snowflake/peer_flow_sf_test.go +++ b/flow/e2e/snowflake/peer_flow_sf_test.go @@ -111,7 +111,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Complete_Simple_Flow_SF() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - limits := peerflow.PeerFlowLimits{ + limits := peerflow.CDCFlowLimits{ TotalSyncFlows: 2, MaxBatchSize: 100, } @@ -119,7 +119,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Complete_Simple_Flow_SF() { // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup // and then insert 10 rows into the source table go func() { - e2e.SetupPeerFlowStatusQuery(env, connectionGen) + e2e.SetupCDCFlowStatusQuery(env, connectionGen) // insert 10 rows into the source table for i := 0; i < 10; i++ { testKey := fmt.Sprintf("test_key_%d", i) @@ -132,7 +132,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Complete_Simple_Flow_SF() { fmt.Println("Inserted 10 rows into the source table") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -179,7 +179,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Complete_Simple_Flow_SF_Avro_CDC() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - limits := peerflow.PeerFlowLimits{ + limits := peerflow.CDCFlowLimits{ TotalSyncFlows: 2, MaxBatchSize: 100, } @@ -187,7 +187,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Complete_Simple_Flow_SF_Avro_CDC() { // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup // and then insert 10 rows into the source table go func() { - e2e.SetupPeerFlowStatusQuery(env, connectionGen) + e2e.SetupCDCFlowStatusQuery(env, connectionGen) // insert 10 rows into the source table for i := 0; i < 10; i++ { testKey := fmt.Sprintf("test_key_%d", i) @@ -200,7 +200,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Complete_Simple_Flow_SF_Avro_CDC() { fmt.Println("Inserted 10 rows into the source table") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -250,7 +250,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Toast_SF() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - limits := peerflow.PeerFlowLimits{ + limits := peerflow.CDCFlowLimits{ TotalSyncFlows: 1, MaxBatchSize: 100, } @@ -258,7 +258,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Toast_SF() { // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup // and execute a transaction touching toast columns go func() { - e2e.SetupPeerFlowStatusQuery(env, connectionGen) + e2e.SetupCDCFlowStatusQuery(env, connectionGen) /* Executing a transaction which 1. changes both toast column @@ -277,7 +277,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Toast_SF() { fmt.Println("Executed a transaction touching toast columns") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -321,7 +321,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Toast_Nochanges_SF() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - limits := peerflow.PeerFlowLimits{ + limits := peerflow.CDCFlowLimits{ TotalSyncFlows: 1, MaxBatchSize: 100, } @@ -329,7 +329,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Toast_Nochanges_SF() { // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup // and execute a transaction touching toast columns go func() { - e2e.SetupPeerFlowStatusQuery(env, connectionGen) + e2e.SetupCDCFlowStatusQuery(env, connectionGen) /* transaction updating no rows */ _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` BEGIN; @@ -341,7 +341,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Toast_Nochanges_SF() { fmt.Println("Executed a transaction touching toast columns") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -385,7 +385,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Toast_Advance_1_SF() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - limits := peerflow.PeerFlowLimits{ + limits := peerflow.CDCFlowLimits{ TotalSyncFlows: 2, MaxBatchSize: 100, } @@ -393,7 +393,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Toast_Advance_1_SF() { // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup // and execute a transaction touching toast columns go func() { - e2e.SetupPeerFlowStatusQuery(env, connectionGen) + e2e.SetupCDCFlowStatusQuery(env, connectionGen) //complex transaction with random DMLs on a table with toast columns _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` BEGIN; @@ -418,7 +418,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Toast_Advance_1_SF() { fmt.Println("Executed a transaction touching toast columns") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -461,7 +461,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Toast_Advance_2_SF() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - limits := peerflow.PeerFlowLimits{ + limits := peerflow.CDCFlowLimits{ TotalSyncFlows: 1, MaxBatchSize: 100, } @@ -469,7 +469,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Toast_Advance_2_SF() { // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup // and execute a transaction touching toast columns go func() { - e2e.SetupPeerFlowStatusQuery(env, connectionGen) + e2e.SetupCDCFlowStatusQuery(env, connectionGen) //complex transaction with random DMLs on a table with toast columns _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` BEGIN; @@ -488,7 +488,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Toast_Advance_2_SF() { fmt.Println("Executed a transaction touching toast columns") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -532,7 +532,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Toast_Advance_3_SF() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - limits := peerflow.PeerFlowLimits{ + limits := peerflow.CDCFlowLimits{ TotalSyncFlows: 1, MaxBatchSize: 100, } @@ -540,7 +540,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Toast_Advance_3_SF() { // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup // and execute a transaction touching toast columns go func() { - e2e.SetupPeerFlowStatusQuery(env, connectionGen) + e2e.SetupCDCFlowStatusQuery(env, connectionGen) /* transaction updating a single row multiple times with changed/unchanged toast columns @@ -558,7 +558,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Toast_Advance_3_SF() { fmt.Println("Executed a transaction touching toast columns") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -607,7 +607,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Types_SF() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - limits := peerflow.PeerFlowLimits{ + limits := peerflow.CDCFlowLimits{ TotalSyncFlows: 1, MaxBatchSize: 100, } @@ -615,7 +615,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Types_SF() { // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup // and execute a transaction touching toast columns go func() { - e2e.SetupPeerFlowStatusQuery(env, connectionGen) + e2e.SetupCDCFlowStatusQuery(env, connectionGen) /* test inserting various types*/ _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s SELECT 2,2,b'1',b'101', @@ -632,7 +632,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Types_SF() { fmt.Println("Executed an insert with all types") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -691,7 +691,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Types_SF_Avro_CDC() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - limits := peerflow.PeerFlowLimits{ + limits := peerflow.CDCFlowLimits{ TotalSyncFlows: 1, MaxBatchSize: 100, } @@ -699,7 +699,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Types_SF_Avro_CDC() { // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup // and execute a transaction touching toast columns go func() { - e2e.SetupPeerFlowStatusQuery(env, connectionGen) + e2e.SetupCDCFlowStatusQuery(env, connectionGen) /* test inserting various types*/ _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s SELECT 2,2,b'1',b'101', @@ -716,7 +716,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Types_SF_Avro_CDC() { fmt.Println("Executed an insert with all types") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -764,7 +764,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Multi_Table_SF() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - limits := peerflow.PeerFlowLimits{ + limits := peerflow.CDCFlowLimits{ TotalSyncFlows: 1, MaxBatchSize: 100, } @@ -772,7 +772,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Multi_Table_SF() { // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup // and execute a transaction touching toast columns go func() { - e2e.SetupPeerFlowStatusQuery(env, connectionGen) + e2e.SetupCDCFlowStatusQuery(env, connectionGen) /* inserting across multiple tables*/ _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s (c1,c2) VALUES (1,'dummy_1'); @@ -782,7 +782,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Multi_Table_SF() { fmt.Println("Executed an insert with all types") }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) @@ -824,7 +824,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Simple_Schema_Changes_SF() { flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() s.NoError(err) - limits := peerflow.PeerFlowLimits{ + limits := peerflow.CDCFlowLimits{ TotalSyncFlows: 10, MaxBatchSize: 100, } @@ -833,7 +833,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Simple_Schema_Changes_SF() { // and then insert and mutate schema repeatedly. go func() { // insert first row. - e2e.SetupPeerFlowStatusQuery(env, connectionGen) + e2e.SetupCDCFlowStatusQuery(env, connectionGen) _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c1) VALUES ($1)`, srcTableName), 1) s.NoError(err) @@ -890,7 +890,7 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Simple_Schema_Changes_SF() { s.compareTableContentsSF("test_simple_schema_changes", "id,c1", false) }() - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) diff --git a/flow/e2e/sqlserver/qrep_flow_sqlserver_test.go b/flow/e2e/sqlserver/qrep_flow_sqlserver_test.go index 4a072f2011..aad017c01a 100644 --- a/flow/e2e/sqlserver/qrep_flow_sqlserver_test.go +++ b/flow/e2e/sqlserver/qrep_flow_sqlserver_test.go @@ -29,7 +29,7 @@ type PeerFlowE2ETestSuiteSQLServer struct { sqlsHelper *SQLServerHelper } -func TestPeerFlowE2ETestSuiteSQLServer(t *testing.T) { +func TestCDCFlowE2ETestSuiteSQLServer(t *testing.T) { suite.Run(t, new(PeerFlowE2ETestSuiteSQLServer)) } diff --git a/flow/e2e/test_utils.go b/flow/e2e/test_utils.go index 2d4b005dff..f3b3b00fa3 100644 --- a/flow/e2e/test_utils.go +++ b/flow/e2e/test_utils.go @@ -43,31 +43,29 @@ func RegisterWorkflowsAndActivities(env *testsuite.TestWorkflowEnvironment) { // set a 300 second timeout for the workflow to execute a few runs. env.SetTestTimeout(300 * time.Second) - env.RegisterWorkflow(peerflow.PeerFlowWorkflow) - env.RegisterWorkflow(peerflow.PeerFlowWorkflowWithConfig) + env.RegisterWorkflow(peerflow.CDCFlowWorkflowWithConfig) env.RegisterWorkflow(peerflow.SyncFlowWorkflow) env.RegisterWorkflow(peerflow.SetupFlowWorkflow) env.RegisterWorkflow(peerflow.SnapshotFlowWorkflow) env.RegisterWorkflow(peerflow.NormalizeFlowWorkflow) env.RegisterWorkflow(peerflow.QRepFlowWorkflow) env.RegisterWorkflow(peerflow.QRepPartitionWorkflow) - env.RegisterActivity(&activities.FetchConfigActivity{}) env.RegisterActivity(&activities.FlowableActivity{}) env.RegisterActivity(&activities.SnapshotActivity{}) } -func SetupPeerFlowStatusQuery(env *testsuite.TestWorkflowEnvironment, +func SetupCDCFlowStatusQuery(env *testsuite.TestWorkflowEnvironment, connectionGen FlowConnectionGenerationConfig) { // wait for PeerFlowStatusQuery to finish setup // sleep for 5 second to allow the workflow to start time.Sleep(5 * time.Second) for { response, err := env.QueryWorkflow( - peerflow.PeerFlowStatusQuery, + peerflow.CDCFlowStatusQuery, connectionGen.FlowJobName, ) if err == nil { - var state peerflow.PeerFlowState + var state peerflow.CDCFlowState err = response.Get(&state) if err != nil { log.Errorln(err) @@ -93,11 +91,11 @@ func NormalizeFlowCountQuery(env *testsuite.TestWorkflowEnvironment, time.Sleep(5 * time.Second) for { response, err := env.QueryWorkflow( - peerflow.PeerFlowStatusQuery, + peerflow.CDCFlowStatusQuery, connectionGen.FlowJobName, ) if err == nil { - var state peerflow.PeerFlowState + var state peerflow.CDCFlowState err = response.Get(&state) if err != nil { log.Errorln(err) diff --git a/flow/generated/protos/route.pb.go b/flow/generated/protos/route.pb.go index 850d871d1e..d11e4f6e87 100644 --- a/flow/generated/protos/route.pb.go +++ b/flow/generated/protos/route.pb.go @@ -21,7 +21,7 @@ const ( _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) ) -type CreatePeerFlowRequest struct { +type CreateCDCFlowRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields @@ -29,8 +29,8 @@ type CreatePeerFlowRequest struct { ConnectionConfigs *FlowConnectionConfigs `protobuf:"bytes,1,opt,name=connection_configs,json=connectionConfigs,proto3" json:"connection_configs,omitempty"` } -func (x *CreatePeerFlowRequest) Reset() { - *x = CreatePeerFlowRequest{} +func (x *CreateCDCFlowRequest) Reset() { + *x = CreateCDCFlowRequest{} if protoimpl.UnsafeEnabled { mi := &file_route_proto_msgTypes[0] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -38,13 +38,13 @@ func (x *CreatePeerFlowRequest) Reset() { } } -func (x *CreatePeerFlowRequest) String() string { +func (x *CreateCDCFlowRequest) String() string { return protoimpl.X.MessageStringOf(x) } -func (*CreatePeerFlowRequest) ProtoMessage() {} +func (*CreateCDCFlowRequest) ProtoMessage() {} -func (x *CreatePeerFlowRequest) ProtoReflect() protoreflect.Message { +func (x *CreateCDCFlowRequest) ProtoReflect() protoreflect.Message { mi := &file_route_proto_msgTypes[0] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -56,19 +56,19 @@ func (x *CreatePeerFlowRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use CreatePeerFlowRequest.ProtoReflect.Descriptor instead. -func (*CreatePeerFlowRequest) Descriptor() ([]byte, []int) { +// Deprecated: Use CreateCDCFlowRequest.ProtoReflect.Descriptor instead. +func (*CreateCDCFlowRequest) Descriptor() ([]byte, []int) { return file_route_proto_rawDescGZIP(), []int{0} } -func (x *CreatePeerFlowRequest) GetConnectionConfigs() *FlowConnectionConfigs { +func (x *CreateCDCFlowRequest) GetConnectionConfigs() *FlowConnectionConfigs { if x != nil { return x.ConnectionConfigs } return nil } -type CreatePeerFlowResponse struct { +type CreateCDCFlowResponse struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields @@ -76,8 +76,8 @@ type CreatePeerFlowResponse struct { WorflowId string `protobuf:"bytes,1,opt,name=worflow_id,json=worflowId,proto3" json:"worflow_id,omitempty"` } -func (x *CreatePeerFlowResponse) Reset() { - *x = CreatePeerFlowResponse{} +func (x *CreateCDCFlowResponse) Reset() { + *x = CreateCDCFlowResponse{} if protoimpl.UnsafeEnabled { mi := &file_route_proto_msgTypes[1] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -85,13 +85,13 @@ func (x *CreatePeerFlowResponse) Reset() { } } -func (x *CreatePeerFlowResponse) String() string { +func (x *CreateCDCFlowResponse) String() string { return protoimpl.X.MessageStringOf(x) } -func (*CreatePeerFlowResponse) ProtoMessage() {} +func (*CreateCDCFlowResponse) ProtoMessage() {} -func (x *CreatePeerFlowResponse) ProtoReflect() protoreflect.Message { +func (x *CreateCDCFlowResponse) ProtoReflect() protoreflect.Message { mi := &file_route_proto_msgTypes[1] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -103,12 +103,12 @@ func (x *CreatePeerFlowResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use CreatePeerFlowResponse.ProtoReflect.Descriptor instead. -func (*CreatePeerFlowResponse) Descriptor() ([]byte, []int) { +// Deprecated: Use CreateCDCFlowResponse.ProtoReflect.Descriptor instead. +func (*CreateCDCFlowResponse) Descriptor() ([]byte, []int) { return file_route_proto_rawDescGZIP(), []int{1} } -func (x *CreatePeerFlowResponse) GetWorflowId() string { +func (x *CreateCDCFlowResponse) GetWorflowId() string { if x != nil { return x.WorflowId } @@ -428,80 +428,79 @@ var file_route_proto_rawDesc = []byte{ 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x0b, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x0a, 0x66, 0x6c, 0x6f, 0x77, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x6a, 0x0a, 0x15, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, - 0x65, 0x65, 0x72, 0x46, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x51, - 0x0a, 0x12, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, - 0x66, 0x69, 0x67, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x65, 0x65, - 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, - 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x52, 0x11, - 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, - 0x73, 0x22, 0x37, 0x0a, 0x16, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x65, 0x65, 0x72, 0x46, - 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x77, - 0x6f, 0x72, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x09, 0x77, 0x6f, 0x72, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x22, 0x51, 0x0a, 0x15, 0x43, 0x72, - 0x65, 0x61, 0x74, 0x65, 0x51, 0x52, 0x65, 0x70, 0x46, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0b, 0x71, 0x72, 0x65, 0x70, 0x5f, 0x63, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x52, 0x0a, 0x71, 0x72, 0x65, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x22, 0x37, 0x0a, - 0x16, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x51, 0x52, 0x65, 0x70, 0x46, 0x6c, 0x6f, 0x77, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x77, 0x6f, 0x72, 0x66, 0x6c, - 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x77, 0x6f, 0x72, - 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x22, 0xca, 0x01, 0x0a, 0x0f, 0x53, 0x68, 0x75, 0x74, 0x64, - 0x6f, 0x77, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1f, 0x0a, 0x0b, 0x77, 0x6f, - 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0a, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 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, - 0x33, 0x0a, 0x0b, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, - 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0a, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, - 0x50, 0x65, 0x65, 0x72, 0x12, 0x3d, 0x0a, 0x10, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, - 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, - 0x65, 0x72, 0x52, 0x0f, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, - 0x65, 0x65, 0x72, 0x22, 0x47, 0x0a, 0x10, 0x53, 0x68, 0x75, 0x74, 0x64, 0x6f, 0x77, 0x6e, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x6f, 0x6b, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x02, 0x6f, 0x6b, 0x12, 0x23, 0x0a, 0x0d, 0x65, 0x72, 0x72, 0x6f, 0x72, - 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, - 0x65, 0x72, 0x72, 0x6f, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x12, 0x0a, 0x10, - 0x4c, 0x69, 0x73, 0x74, 0x50, 0x65, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x22, 0x3d, 0x0a, 0x11, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x65, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x28, 0x0a, 0x05, 0x70, 0x65, 0x65, 0x72, 0x73, 0x18, 0x01, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, - 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x05, 0x70, 0x65, 0x65, 0x72, 0x73, 0x32, - 0xec, 0x02, 0x0a, 0x0b, 0x46, 0x6c, 0x6f, 0x77, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, - 0x4e, 0x0a, 0x09, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x65, 0x65, 0x72, 0x73, 0x12, 0x1e, 0x2e, 0x70, - 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x4c, 0x69, 0x73, 0x74, - 0x50, 0x65, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x70, - 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x4c, 0x69, 0x73, 0x74, - 0x50, 0x65, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, - 0x5d, 0x0a, 0x0e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x65, 0x65, 0x72, 0x46, 0x6c, 0x6f, - 0x77, 0x12, 0x23, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, - 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x65, 0x65, 0x72, 0x46, 0x6c, 0x6f, 0x77, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, - 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x65, 0x65, 0x72, - 0x46, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5d, - 0x0a, 0x0e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x51, 0x52, 0x65, 0x70, 0x46, 0x6c, 0x6f, 0x77, - 0x12, 0x23, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x69, 0x0a, 0x14, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, + 0x44, 0x43, 0x46, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x51, 0x0a, + 0x12, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x65, 0x65, 0x72, + 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x52, 0x11, 0x63, + 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, + 0x22, 0x36, 0x0a, 0x15, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x44, 0x43, 0x46, 0x6c, 0x6f, + 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x77, 0x6f, 0x72, + 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x77, + 0x6f, 0x72, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x22, 0x51, 0x0a, 0x15, 0x43, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x51, 0x52, 0x65, 0x70, 0x46, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x38, 0x0a, 0x0b, 0x71, 0x72, 0x65, 0x70, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, + 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, + 0x0a, 0x71, 0x72, 0x65, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x22, 0x37, 0x0a, 0x16, 0x43, + 0x72, 0x65, 0x61, 0x74, 0x65, 0x51, 0x52, 0x65, 0x70, 0x46, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x77, 0x6f, 0x72, 0x66, 0x6c, 0x6f, 0x77, + 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x77, 0x6f, 0x72, 0x66, 0x6c, + 0x6f, 0x77, 0x49, 0x64, 0x22, 0xca, 0x01, 0x0a, 0x0f, 0x53, 0x68, 0x75, 0x74, 0x64, 0x6f, 0x77, + 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1f, 0x0a, 0x0b, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 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, 0x33, 0x0a, + 0x0b, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, + 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0a, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x65, + 0x65, 0x72, 0x12, 0x3d, 0x0a, 0x10, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, + 0x52, 0x0f, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x65, 0x65, + 0x72, 0x22, 0x47, 0x0a, 0x10, 0x53, 0x68, 0x75, 0x74, 0x64, 0x6f, 0x77, 0x6e, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x6f, 0x6b, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x02, 0x6f, 0x6b, 0x12, 0x23, 0x0a, 0x0d, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x6d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x72, + 0x72, 0x6f, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x12, 0x0a, 0x10, 0x4c, 0x69, + 0x73, 0x74, 0x50, 0x65, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x3d, + 0x0a, 0x11, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x65, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x28, 0x0a, 0x05, 0x70, 0x65, 0x65, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, + 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x05, 0x70, 0x65, 0x65, 0x72, 0x73, 0x32, 0xe9, 0x02, + 0x0a, 0x0b, 0x46, 0x6c, 0x6f, 0x77, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x4e, 0x0a, + 0x09, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x65, 0x65, 0x72, 0x73, 0x12, 0x1e, 0x2e, 0x70, 0x65, 0x65, + 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x65, + 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x70, 0x65, 0x65, + 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x65, + 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5a, 0x0a, + 0x0d, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x44, 0x43, 0x46, 0x6c, 0x6f, 0x77, 0x12, 0x22, + 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x43, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x43, 0x44, 0x43, 0x46, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, + 0x65, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x44, 0x43, 0x46, 0x6c, 0x6f, 0x77, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5d, 0x0a, 0x0e, 0x43, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x51, 0x52, 0x65, 0x70, 0x46, 0x6c, 0x6f, 0x77, 0x12, 0x23, 0x2e, 0x70, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x51, 0x52, 0x65, 0x70, 0x46, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x24, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x51, 0x52, 0x65, 0x70, 0x46, 0x6c, 0x6f, 0x77, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, - 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x51, 0x52, 0x65, 0x70, 0x46, - 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4f, 0x0a, - 0x0c, 0x53, 0x68, 0x75, 0x74, 0x64, 0x6f, 0x77, 0x6e, 0x46, 0x6c, 0x6f, 0x77, 0x12, 0x1d, 0x2e, - 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x53, 0x68, 0x75, - 0x74, 0x64, 0x6f, 0x77, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x70, - 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x53, 0x68, 0x75, 0x74, - 0x64, 0x6f, 0x77, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x7c, - 0x0a, 0x10, 0x63, 0x6f, 0x6d, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, - 0x74, 0x65, 0x42, 0x0a, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, - 0x5a, 0x10, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x64, 0x2f, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x73, 0xa2, 0x02, 0x03, 0x50, 0x58, 0x58, 0xaa, 0x02, 0x0b, 0x50, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x52, 0x6f, 0x75, 0x74, 0x65, 0xca, 0x02, 0x0b, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x52, - 0x6f, 0x75, 0x74, 0x65, 0xe2, 0x02, 0x17, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x52, 0x6f, 0x75, - 0x74, 0x65, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, - 0x0b, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x62, 0x06, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x33, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4f, 0x0a, 0x0c, 0x53, 0x68, 0x75, 0x74, + 0x64, 0x6f, 0x77, 0x6e, 0x46, 0x6c, 0x6f, 0x77, 0x12, 0x1d, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, + 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x53, 0x68, 0x75, 0x74, 0x64, 0x6f, 0x77, 0x6e, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, + 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x53, 0x68, 0x75, 0x74, 0x64, 0x6f, 0x77, 0x6e, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x7c, 0x0a, 0x10, 0x63, 0x6f, 0x6d, + 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x65, 0x42, 0x0a, 0x52, + 0x6f, 0x75, 0x74, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x10, 0x67, 0x65, 0x6e, + 0x65, 0x72, 0x61, 0x74, 0x65, 0x64, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x73, 0xa2, 0x02, 0x03, + 0x50, 0x58, 0x58, 0xaa, 0x02, 0x0b, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x52, 0x6f, 0x75, 0x74, + 0x65, 0xca, 0x02, 0x0b, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x52, 0x6f, 0x75, 0x74, 0x65, 0xe2, + 0x02, 0x17, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x5c, 0x47, 0x50, + 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x0b, 0x50, 0x65, 0x65, 0x72, + 0x64, 0x62, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -518,8 +517,8 @@ func file_route_proto_rawDescGZIP() []byte { var file_route_proto_msgTypes = make([]protoimpl.MessageInfo, 8) var file_route_proto_goTypes = []interface{}{ - (*CreatePeerFlowRequest)(nil), // 0: peerdb_route.CreatePeerFlowRequest - (*CreatePeerFlowResponse)(nil), // 1: peerdb_route.CreatePeerFlowResponse + (*CreateCDCFlowRequest)(nil), // 0: peerdb_route.CreateCDCFlowRequest + (*CreateCDCFlowResponse)(nil), // 1: peerdb_route.CreateCDCFlowResponse (*CreateQRepFlowRequest)(nil), // 2: peerdb_route.CreateQRepFlowRequest (*CreateQRepFlowResponse)(nil), // 3: peerdb_route.CreateQRepFlowResponse (*ShutdownRequest)(nil), // 4: peerdb_route.ShutdownRequest @@ -531,17 +530,17 @@ var file_route_proto_goTypes = []interface{}{ (*Peer)(nil), // 10: peerdb_peers.Peer } var file_route_proto_depIdxs = []int32{ - 8, // 0: peerdb_route.CreatePeerFlowRequest.connection_configs:type_name -> peerdb_flow.FlowConnectionConfigs + 8, // 0: peerdb_route.CreateCDCFlowRequest.connection_configs:type_name -> peerdb_flow.FlowConnectionConfigs 9, // 1: peerdb_route.CreateQRepFlowRequest.qrep_config:type_name -> peerdb_flow.QRepConfig 10, // 2: peerdb_route.ShutdownRequest.source_peer:type_name -> peerdb_peers.Peer 10, // 3: peerdb_route.ShutdownRequest.destination_peer:type_name -> peerdb_peers.Peer 10, // 4: peerdb_route.ListPeersResponse.peers:type_name -> peerdb_peers.Peer 6, // 5: peerdb_route.FlowService.ListPeers:input_type -> peerdb_route.ListPeersRequest - 0, // 6: peerdb_route.FlowService.CreatePeerFlow:input_type -> peerdb_route.CreatePeerFlowRequest + 0, // 6: peerdb_route.FlowService.CreateCDCFlow:input_type -> peerdb_route.CreateCDCFlowRequest 2, // 7: peerdb_route.FlowService.CreateQRepFlow:input_type -> peerdb_route.CreateQRepFlowRequest 4, // 8: peerdb_route.FlowService.ShutdownFlow:input_type -> peerdb_route.ShutdownRequest 7, // 9: peerdb_route.FlowService.ListPeers:output_type -> peerdb_route.ListPeersResponse - 1, // 10: peerdb_route.FlowService.CreatePeerFlow:output_type -> peerdb_route.CreatePeerFlowResponse + 1, // 10: peerdb_route.FlowService.CreateCDCFlow:output_type -> peerdb_route.CreateCDCFlowResponse 3, // 11: peerdb_route.FlowService.CreateQRepFlow:output_type -> peerdb_route.CreateQRepFlowResponse 5, // 12: peerdb_route.FlowService.ShutdownFlow:output_type -> peerdb_route.ShutdownResponse 9, // [9:13] is the sub-list for method output_type @@ -560,7 +559,7 @@ func file_route_proto_init() { file_flow_proto_init() if !protoimpl.UnsafeEnabled { file_route_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CreatePeerFlowRequest); i { + switch v := v.(*CreateCDCFlowRequest); i { case 0: return &v.state case 1: @@ -572,7 +571,7 @@ func file_route_proto_init() { } } file_route_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CreatePeerFlowResponse); i { + switch v := v.(*CreateCDCFlowResponse); i { case 0: return &v.state case 1: diff --git a/flow/generated/protos/route_grpc.pb.go b/flow/generated/protos/route_grpc.pb.go index d688dc3337..9efd24b251 100644 --- a/flow/generated/protos/route_grpc.pb.go +++ b/flow/generated/protos/route_grpc.pb.go @@ -20,7 +20,7 @@ const _ = grpc.SupportPackageIsVersion7 const ( FlowService_ListPeers_FullMethodName = "/peerdb_route.FlowService/ListPeers" - FlowService_CreatePeerFlow_FullMethodName = "/peerdb_route.FlowService/CreatePeerFlow" + FlowService_CreateCDCFlow_FullMethodName = "/peerdb_route.FlowService/CreateCDCFlow" FlowService_CreateQRepFlow_FullMethodName = "/peerdb_route.FlowService/CreateQRepFlow" FlowService_ShutdownFlow_FullMethodName = "/peerdb_route.FlowService/ShutdownFlow" ) @@ -30,7 +30,7 @@ const ( // For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. type FlowServiceClient interface { ListPeers(ctx context.Context, in *ListPeersRequest, opts ...grpc.CallOption) (*ListPeersResponse, error) - CreatePeerFlow(ctx context.Context, in *CreatePeerFlowRequest, opts ...grpc.CallOption) (*CreatePeerFlowResponse, error) + CreateCDCFlow(ctx context.Context, in *CreateCDCFlowRequest, opts ...grpc.CallOption) (*CreateCDCFlowResponse, error) CreateQRepFlow(ctx context.Context, in *CreateQRepFlowRequest, opts ...grpc.CallOption) (*CreateQRepFlowResponse, error) ShutdownFlow(ctx context.Context, in *ShutdownRequest, opts ...grpc.CallOption) (*ShutdownResponse, error) } @@ -52,9 +52,9 @@ func (c *flowServiceClient) ListPeers(ctx context.Context, in *ListPeersRequest, return out, nil } -func (c *flowServiceClient) CreatePeerFlow(ctx context.Context, in *CreatePeerFlowRequest, opts ...grpc.CallOption) (*CreatePeerFlowResponse, error) { - out := new(CreatePeerFlowResponse) - err := c.cc.Invoke(ctx, FlowService_CreatePeerFlow_FullMethodName, in, out, opts...) +func (c *flowServiceClient) CreateCDCFlow(ctx context.Context, in *CreateCDCFlowRequest, opts ...grpc.CallOption) (*CreateCDCFlowResponse, error) { + out := new(CreateCDCFlowResponse) + err := c.cc.Invoke(ctx, FlowService_CreateCDCFlow_FullMethodName, in, out, opts...) if err != nil { return nil, err } @@ -84,7 +84,7 @@ func (c *flowServiceClient) ShutdownFlow(ctx context.Context, in *ShutdownReques // for forward compatibility type FlowServiceServer interface { ListPeers(context.Context, *ListPeersRequest) (*ListPeersResponse, error) - CreatePeerFlow(context.Context, *CreatePeerFlowRequest) (*CreatePeerFlowResponse, error) + CreateCDCFlow(context.Context, *CreateCDCFlowRequest) (*CreateCDCFlowResponse, error) CreateQRepFlow(context.Context, *CreateQRepFlowRequest) (*CreateQRepFlowResponse, error) ShutdownFlow(context.Context, *ShutdownRequest) (*ShutdownResponse, error) mustEmbedUnimplementedFlowServiceServer() @@ -97,8 +97,8 @@ type UnimplementedFlowServiceServer struct { func (UnimplementedFlowServiceServer) ListPeers(context.Context, *ListPeersRequest) (*ListPeersResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method ListPeers not implemented") } -func (UnimplementedFlowServiceServer) CreatePeerFlow(context.Context, *CreatePeerFlowRequest) (*CreatePeerFlowResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method CreatePeerFlow not implemented") +func (UnimplementedFlowServiceServer) CreateCDCFlow(context.Context, *CreateCDCFlowRequest) (*CreateCDCFlowResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method CreateCDCFlow not implemented") } func (UnimplementedFlowServiceServer) CreateQRepFlow(context.Context, *CreateQRepFlowRequest) (*CreateQRepFlowResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method CreateQRepFlow not implemented") @@ -137,20 +137,20 @@ func _FlowService_ListPeers_Handler(srv interface{}, ctx context.Context, dec fu return interceptor(ctx, in, info, handler) } -func _FlowService_CreatePeerFlow_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(CreatePeerFlowRequest) +func _FlowService_CreateCDCFlow_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(CreateCDCFlowRequest) if err := dec(in); err != nil { return nil, err } if interceptor == nil { - return srv.(FlowServiceServer).CreatePeerFlow(ctx, in) + return srv.(FlowServiceServer).CreateCDCFlow(ctx, in) } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: FlowService_CreatePeerFlow_FullMethodName, + FullMethod: FlowService_CreateCDCFlow_FullMethodName, } handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(FlowServiceServer).CreatePeerFlow(ctx, req.(*CreatePeerFlowRequest)) + return srv.(FlowServiceServer).CreateCDCFlow(ctx, req.(*CreateCDCFlowRequest)) } return interceptor(ctx, in, info, handler) } @@ -203,8 +203,8 @@ var FlowService_ServiceDesc = grpc.ServiceDesc{ Handler: _FlowService_ListPeers_Handler, }, { - MethodName: "CreatePeerFlow", - Handler: _FlowService_CreatePeerFlow_Handler, + MethodName: "CreateCDCFlow", + Handler: _FlowService_CreateCDCFlow_Handler, }, { MethodName: "CreateQRepFlow", diff --git a/flow/shared/constants.go b/flow/shared/constants.go index 5334243e59..17efcfc452 100644 --- a/flow/shared/constants.go +++ b/flow/shared/constants.go @@ -3,14 +3,14 @@ package shared const ( PeerFlowTaskQueue = "peer-flow-task-queue" SnapshotFlowTaskQueue = "snapshot-flow-task-queue" - PeerFlowSignalName = "peer-flow-signal" + CDCFlowSignalName = "peer-flow-signal" ) -type PeerFlowSignal int64 +type CDCFlowSignal int64 type ContextKey string const ( - NoopSignal PeerFlowSignal = iota + NoopSignal CDCFlowSignal = iota ShutdownSignal EnableMetricsKey ContextKey = "enableMetrics" CDCMirrorMonitorKey ContextKey = "cdcMirrorMonitor" diff --git a/flow/workflows/activities.go b/flow/workflows/activities.go index 5b318031af..0b23d10dd1 100644 --- a/flow/workflows/activities.go +++ b/flow/workflows/activities.go @@ -3,7 +3,6 @@ package peerflow import "github.com/PeerDB-io/peer-flow/activities" var ( - fetchConfig *activities.FetchConfigActivity - flowable *activities.FlowableActivity - snapshot *activities.SnapshotActivity + flowable *activities.FlowableActivity + snapshot *activities.SnapshotActivity ) diff --git a/flow/workflows/peer_flow.go b/flow/workflows/cdc_flow.go similarity index 69% rename from flow/workflows/peer_flow.go rename to flow/workflows/cdc_flow.go index 9c33d5464d..46d44ad51b 100644 --- a/flow/workflows/peer_flow.go +++ b/flow/workflows/cdc_flow.go @@ -4,7 +4,6 @@ import ( "fmt" "time" - "github.com/PeerDB-io/peer-flow/activities" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/shared" @@ -17,11 +16,11 @@ import ( ) const ( - PeerFlowStatusQuery = "q-peer-flow-status" - maxSyncFlowsPerPeerFlow = 32 + CDCFlowStatusQuery = "q-cdc-flow-status" + maxSyncFlowsPerCDCFlow = 32 ) -type PeerFlowLimits struct { +type CDCFlowLimits 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. @@ -34,8 +33,8 @@ type PeerFlowLimits struct { MaxBatchSize int } -type PeerFlowWorkflowInput struct { - PeerFlowLimits +type CDCFlowWorkflowInput struct { + CDCFlowLimits // The JDBC URL for the catalog database. CatalogJdbcURL string // The name of the peer flow to execute. @@ -52,7 +51,7 @@ type PeerFlowWorkflowInput struct { MaxBatchSize int } -type PeerFlowState struct { +type CDCFlowState struct { // Progress events for the peer flow. Progress []string // Accumulates status for sync flows spawned. @@ -60,7 +59,7 @@ type PeerFlowState struct { // Accumulates status for sync flows spawned. NormalizeFlowStatuses []*model.NormalizeResponse // Current signalled state of the peer flow. - ActiveSignal shared.PeerFlowSignal + ActiveSignal shared.CDCFlowSignal // SetupComplete indicates whether the peer flow setup has completed. SetupComplete bool // Errors encountered during child sync flow executions. @@ -73,8 +72,8 @@ type PeerFlowState struct { } // returns a new empty PeerFlowState -func NewStartedPeerFlowState() *PeerFlowState { - return &PeerFlowState{ +func NewCDCFlowState() *CDCFlowState { + return &CDCFlowState{ Progress: []string{"started"}, SyncFlowStatuses: nil, NormalizeFlowStatuses: nil, @@ -93,7 +92,7 @@ func NewStartedPeerFlowState() *PeerFlowState { } // truncate the progress and other arrays to a max of 10 elements -func (s *PeerFlowState) TruncateProgress(log log.Logger) { +func (s *CDCFlowState) TruncateProgress() { if len(s.Progress) > 10 { s.Progress = s.Progress[len(s.Progress)-10:] } @@ -103,68 +102,22 @@ func (s *PeerFlowState) TruncateProgress(log log.Logger) { if len(s.NormalizeFlowStatuses) > 10 { s.NormalizeFlowStatuses = s.NormalizeFlowStatuses[len(s.NormalizeFlowStatuses)-10:] } - - if s.SyncFlowErrors != nil { - // log and clear the error - log.Error("sync flow error: ", s.SyncFlowErrors) - s.SyncFlowErrors = nil - } - - if s.NormalizeFlowErrors != nil { - // log and clear the error - log.Error("normalize flow error: ", s.NormalizeFlowErrors) - s.NormalizeFlowErrors = nil - } } -// PeerFlowWorkflowExecution represents the state for execution of a peer flow. -type PeerFlowWorkflowExecution struct { +// CDCFlowWorkflowExecution represents the state for execution of a peer flow. +type CDCFlowWorkflowExecution struct { flowExecutionID string logger log.Logger } -// NewPeerFlowWorkflowExecution creates a new instance of PeerFlowWorkflowExecution. -func NewPeerFlowWorkflowExecution(ctx workflow.Context) *PeerFlowWorkflowExecution { - return &PeerFlowWorkflowExecution{ +// NewCDCFlowWorkflowExecution creates a new instance of PeerFlowWorkflowExecution. +func NewCDCFlowWorkflowExecution(ctx workflow.Context) *CDCFlowWorkflowExecution { + return &CDCFlowWorkflowExecution{ flowExecutionID: workflow.GetInfo(ctx).WorkflowExecution.ID, logger: workflow.GetLogger(ctx), } } -// fetchConnectionConfigs fetches the connection configs for source and destination peers. -func fetchConnectionConfigs( - ctx workflow.Context, - logger log.Logger, - input *PeerFlowWorkflowInput, -) (*protos.FlowConnectionConfigs, error) { - logger.Info("fetching connection configs for peer flow - ", input.PeerFlowName) - - ctx = workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ - StartToCloseTimeout: 1 * time.Minute, - }) - - fetchConfigActivityInput := &activities.FetchConfigActivityInput{ - CatalogJdbcURL: input.CatalogJdbcURL, - PeerFlowName: input.PeerFlowName, - } - - configsFuture := workflow.ExecuteActivity(ctx, fetchConfig.FetchConfig, fetchConfigActivityInput) - - flowConnectionConfigs := &protos.FlowConnectionConfigs{} - if err := configsFuture.Get(ctx, &flowConnectionConfigs); err != nil { - return nil, fmt.Errorf("failed to fetch connection configs: %w", err) - } - - if flowConnectionConfigs == nil || - flowConnectionConfigs.Source == nil || - flowConnectionConfigs.Destination == nil { - return nil, fmt.Errorf("invalid connection configs") - } - - logger.Info("fetched connection configs for peer flow - ", input.PeerFlowName) - return flowConnectionConfigs, nil -} - func GetChildWorkflowID( ctx workflow.Context, prefix string, @@ -182,81 +135,39 @@ func GetChildWorkflowID( return childWorkflowID, nil } -// PeerFlowWorkflowResult is the result of the PeerFlowWorkflow. -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) { - 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: 20, - }, - } - - 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 -} +// CDCFlowWorkflowResult is the result of the PeerFlowWorkflow. +type CDCFlowWorkflowResult = CDCFlowState -func PeerFlowWorkflowWithConfig( +func CDCFlowWorkflowWithConfig( ctx workflow.Context, cfg *protos.FlowConnectionConfigs, - limits *PeerFlowLimits, - state *PeerFlowState, -) (*PeerFlowWorkflowResult, error) { + limits *CDCFlowLimits, + state *CDCFlowState, +) (*CDCFlowWorkflowResult, error) { if state == nil { - state = NewStartedPeerFlowState() + state = NewCDCFlowState() } if cfg == nil { return nil, fmt.Errorf("invalid connection configs") } - w := NewPeerFlowWorkflowExecution(ctx) + w := NewCDCFlowWorkflowExecution(ctx) if limits.TotalSyncFlows == 0 { - limits.TotalSyncFlows = maxSyncFlowsPerPeerFlow + limits.TotalSyncFlows = maxSyncFlowsPerCDCFlow } // Support a Query for the current state of the peer flow. - err := workflow.SetQueryHandler(ctx, PeerFlowStatusQuery, func(jobName string) (PeerFlowState, error) { + err := workflow.SetQueryHandler(ctx, CDCFlowStatusQuery, func(jobName string) (CDCFlowState, error) { return *state, nil }) if err != nil { - return state, fmt.Errorf("failed to set `%s` query handler: %w", PeerFlowStatusQuery, err) + return state, fmt.Errorf("failed to set `%s` query handler: %w", CDCFlowStatusQuery, err) } - signalChan := workflow.GetSignalChannel(ctx, shared.PeerFlowSignalName) - signalHandler := func(_ workflow.Context, v shared.PeerFlowSignal) { + signalChan := workflow.GetSignalChannel(ctx, shared.CDCFlowSignalName) + signalHandler := func(_ workflow.Context, v shared.CDCFlowSignal) { w.logger.Info("received signal - ", v) state.ActiveSignal = v } @@ -264,7 +175,7 @@ func PeerFlowWorkflowWithConfig( // 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 + var signalVal shared.CDCFlowSignal c.Receive(ctx, &signalVal) signalHandler(ctx, signalVal) }) @@ -425,6 +336,6 @@ func PeerFlowWorkflowWithConfig( } } - state.TruncateProgress(w.logger) - return nil, workflow.NewContinueAsNewError(ctx, PeerFlowWorkflowWithConfig, cfg, limits, state) + state.TruncateProgress() + return nil, workflow.NewContinueAsNewError(ctx, CDCFlowWorkflowWithConfig, cfg, limits, state) } diff --git a/flow/workflows/setup_flow.go b/flow/workflows/setup_flow.go index c5a47d6d38..cf5d395698 100644 --- a/flow/workflows/setup_flow.go +++ b/flow/workflows/setup_flow.go @@ -30,8 +30,8 @@ import ( // - creating the normalized table on the destination peer type SetupFlowState struct { - PeerFlowName string - Progress []string + CDCFlowName string + Progress []string } type SetupFlowExecution struct { @@ -55,7 +55,7 @@ func (s *SetupFlowExecution) checkConnectionsAndSetupMetadataTables( ctx workflow.Context, config *protos.FlowConnectionConfigs, ) error { - s.logger.Info("checking connections for peer flow - ", s.PeerFlowName) + s.logger.Info("checking connections for CDC flow - ", s.CDCFlowName) ctx = workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ StartToCloseTimeout: 2 * time.Minute, @@ -75,7 +75,7 @@ func (s *SetupFlowExecution) checkConnectionsAndSetupMetadataTables( return fmt.Errorf("failed to check destination peer connection: %w", err) } - s.logger.Info("ensuring metadata table exists - ", s.PeerFlowName) + s.logger.Info("ensuring metadata table exists - ", s.CDCFlowName) // then setup the destination peer metadata tables if destConnStatus.NeedsSetupMetadataTables { @@ -95,7 +95,7 @@ func (s *SetupFlowExecution) ensurePullability( ctx workflow.Context, config *protos.FlowConnectionConfigs, ) error { - s.logger.Info("ensuring pullability for peer flow - ", s.PeerFlowName) + s.logger.Info("ensuring pullability for peer flow - ", s.CDCFlowName) ctx = workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ StartToCloseTimeout: 15 * time.Minute, @@ -108,7 +108,7 @@ func (s *SetupFlowExecution) ensurePullability( // create EnsurePullabilityInput for the srcTableName ensurePullabilityInput := &protos.EnsurePullabilityBatchInput{ PeerConnectionConfig: config.Source, - FlowJobName: s.PeerFlowName, + FlowJobName: s.CDCFlowName, SourceTableIdentifiers: srcTblIdentifiers, } @@ -139,7 +139,7 @@ func (s *SetupFlowExecution) createRawTable( ctx workflow.Context, config *protos.FlowConnectionConfigs, ) error { - s.logger.Info("creating raw table on destination - ", s.PeerFlowName) + s.logger.Info("creating raw table on destination - ", s.CDCFlowName) ctx = workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ StartToCloseTimeout: 5 * time.Minute, }) @@ -147,7 +147,7 @@ func (s *SetupFlowExecution) createRawTable( // attempt to create the tables. createRawTblInput := &protos.CreateRawTableInput{ PeerConnectionConfig: config.Destination, - FlowJobName: s.PeerFlowName, + FlowJobName: s.CDCFlowName, TableNameMapping: config.TableNameMapping, CdcSyncMode: config.CdcSyncMode, } @@ -164,7 +164,7 @@ func (s *SetupFlowExecution) createRawTable( // sets up the normalized tables on the destination peer. func (s *SetupFlowExecution) fetchTableSchemaAndSetupNormalizedTables( ctx workflow.Context, flowConnectionConfigs *protos.FlowConnectionConfigs) (map[string]*protos.TableSchema, error) { - s.logger.Info("fetching table schema for peer flow - ", s.PeerFlowName) + s.logger.Info("fetching table schema for peer flow - ", s.CDCFlowName) ctx = workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ StartToCloseTimeout: 1 * time.Hour, @@ -191,7 +191,7 @@ func (s *SetupFlowExecution) fetchTableSchemaAndSetupNormalizedTables( sortedSourceTables := maps.Keys(tableNameSchemaMapping) sort.Strings(sortedSourceTables) - s.logger.Info("setting up normalized tables for peer flow - ", s.PeerFlowName) + s.logger.Info("setting up normalized tables for peer flow - ", s.CDCFlowName) normalizedTableMapping := make(map[string]*protos.TableSchema) for _, srcTableName := range sortedSourceTables { tableSchema := tableNameSchemaMapping[srcTableName] @@ -213,7 +213,7 @@ func (s *SetupFlowExecution) fetchTableSchemaAndSetupNormalizedTables( return nil, fmt.Errorf("failed to create normalized tables: %w", err) } - s.logger.Info("finished setting up normalized tables for peer flow - ", s.PeerFlowName) + s.logger.Info("finished setting up normalized tables for peer flow - ", s.CDCFlowName) return normalizedTableMapping, nil } @@ -222,7 +222,7 @@ func (s *SetupFlowExecution) executeSetupFlow( ctx workflow.Context, config *protos.FlowConnectionConfigs, ) (map[string]*protos.TableSchema, error) { - s.logger.Info("executing setup flow - ", s.PeerFlowName) + s.logger.Info("executing setup flow - ", s.CDCFlowName) // first check the connectionsAndSetupMetadataTables if err := s.checkConnectionsAndSetupMetadataTables(ctx, config); err != nil { @@ -252,8 +252,8 @@ func (s *SetupFlowExecution) executeSetupFlow( func SetupFlowWorkflow(ctx workflow.Context, config *protos.FlowConnectionConfigs) (*protos.FlowConnectionConfigs, error) { setupFlowState := &SetupFlowState{ - PeerFlowName: config.FlowJobName, - Progress: []string{}, + CDCFlowName: config.FlowJobName, + Progress: []string{}, } // create the setup flow execution diff --git a/flow/workflows/sync_flow.go b/flow/workflows/sync_flow.go index a2be62af9e..6661d6ecc4 100644 --- a/flow/workflows/sync_flow.go +++ b/flow/workflows/sync_flow.go @@ -11,8 +11,8 @@ import ( ) type SyncFlowState struct { - PeerFlowName string - Progress []string + CDCFlowName string + Progress []string } type SyncFlowExecution struct { @@ -22,8 +22,8 @@ type SyncFlowExecution struct { } type NormalizeFlowState struct { - PeerFlowName string - Progress []string + CDCFlowName string + Progress []string } type NormalizeFlowExecution struct { @@ -56,7 +56,7 @@ func (s *SyncFlowExecution) executeSyncFlow( opts *protos.SyncFlowOptions, relationMessageMapping model.RelationMessageMapping, ) (*model.SyncResponse, error) { - s.logger.Info("executing sync flow - ", s.PeerFlowName) + s.logger.Info("executing sync flow - ", s.CDCFlowName) syncMetaCtx := workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ StartToCloseTimeout: 1 * time.Minute, @@ -65,7 +65,7 @@ func (s *SyncFlowExecution) executeSyncFlow( // execute GetLastSyncedID on destination peer lastSyncInput := &protos.GetLastSyncedIDInput{ PeerConnectionConfig: config.Destination, - FlowJobName: s.PeerFlowName, + FlowJobName: s.CDCFlowName, } lastSyncFuture := workflow.ExecuteActivity(syncMetaCtx, flowable.GetLastSyncedID, lastSyncInput) @@ -113,8 +113,8 @@ func SyncFlowWorkflow(ctx workflow.Context, options *protos.SyncFlowOptions, ) (*model.SyncResponse, error) { s := NewSyncFlowExecution(ctx, &SyncFlowState{ - PeerFlowName: config.FlowJobName, - Progress: []string{}, + CDCFlowName: config.FlowJobName, + Progress: []string{}, }) return s.executeSyncFlow(ctx, config, options, options.RelationMessageMapping) @@ -125,8 +125,8 @@ func NormalizeFlowWorkflow(ctx workflow.Context, tableSchemaDelta *protos.TableSchemaDelta, ) (*model.NormalizeResponse, error) { s := NewNormalizeFlowExecution(ctx, &NormalizeFlowState{ - PeerFlowName: config.FlowJobName, - Progress: []string{}, + CDCFlowName: config.FlowJobName, + Progress: []string{}, }) return s.executeNormalizeFlow(ctx, config, tableSchemaDelta) @@ -137,7 +137,7 @@ func (s *NormalizeFlowExecution) executeNormalizeFlow( config *protos.FlowConnectionConfigs, tableSchemaDelta *protos.TableSchemaDelta, ) (*model.NormalizeResponse, error) { - s.logger.Info("executing normalize flow - ", s.PeerFlowName) + s.logger.Info("executing normalize flow - ", s.CDCFlowName) normalizeFlowCtx := workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ StartToCloseTimeout: 7 * 24 * time.Hour, diff --git a/nexus/pt/src/peerdb_route.rs b/nexus/pt/src/peerdb_route.rs index 5a9a17e5d9..fb51949ea6 100644 --- a/nexus/pt/src/peerdb_route.rs +++ b/nexus/pt/src/peerdb_route.rs @@ -1,13 +1,13 @@ // @generated #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct CreatePeerFlowRequest { +pub struct CreateCdcFlowRequest { #[prost(message, optional, tag="1")] pub connection_configs: ::core::option::Option, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct CreatePeerFlowResponse { +pub struct CreateCdcFlowResponse { #[prost(string, tag="1")] pub worflow_id: ::prost::alloc::string::String, } diff --git a/nexus/pt/src/peerdb_route.serde.rs b/nexus/pt/src/peerdb_route.serde.rs index c9311a96a5..e636634484 100644 --- a/nexus/pt/src/peerdb_route.serde.rs +++ b/nexus/pt/src/peerdb_route.serde.rs @@ -1,5 +1,5 @@ // @generated -impl serde::Serialize for CreatePeerFlowRequest { +impl serde::Serialize for CreateCdcFlowRequest { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result where @@ -10,14 +10,14 @@ impl serde::Serialize for CreatePeerFlowRequest { if self.connection_configs.is_some() { len += 1; } - let mut struct_ser = serializer.serialize_struct("peerdb_route.CreatePeerFlowRequest", len)?; + let mut struct_ser = serializer.serialize_struct("peerdb_route.CreateCDCFlowRequest", len)?; if let Some(v) = self.connection_configs.as_ref() { struct_ser.serialize_field("connectionConfigs", v)?; } struct_ser.end() } } -impl<'de> serde::Deserialize<'de> for CreatePeerFlowRequest { +impl<'de> serde::Deserialize<'de> for CreateCdcFlowRequest { #[allow(deprecated)] fn deserialize(deserializer: D) -> std::result::Result where @@ -63,13 +63,13 @@ impl<'de> serde::Deserialize<'de> for CreatePeerFlowRequest { } struct GeneratedVisitor; impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { - type Value = CreatePeerFlowRequest; + type Value = CreateCdcFlowRequest; fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - formatter.write_str("struct peerdb_route.CreatePeerFlowRequest") + formatter.write_str("struct peerdb_route.CreateCDCFlowRequest") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -87,15 +87,15 @@ impl<'de> serde::Deserialize<'de> for CreatePeerFlowRequest { } } } - Ok(CreatePeerFlowRequest { + Ok(CreateCdcFlowRequest { connection_configs: connection_configs__, }) } } - deserializer.deserialize_struct("peerdb_route.CreatePeerFlowRequest", FIELDS, GeneratedVisitor) + deserializer.deserialize_struct("peerdb_route.CreateCDCFlowRequest", FIELDS, GeneratedVisitor) } } -impl serde::Serialize for CreatePeerFlowResponse { +impl serde::Serialize for CreateCdcFlowResponse { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result where @@ -106,14 +106,14 @@ impl serde::Serialize for CreatePeerFlowResponse { if !self.worflow_id.is_empty() { len += 1; } - let mut struct_ser = serializer.serialize_struct("peerdb_route.CreatePeerFlowResponse", len)?; + let mut struct_ser = serializer.serialize_struct("peerdb_route.CreateCDCFlowResponse", len)?; if !self.worflow_id.is_empty() { struct_ser.serialize_field("worflowId", &self.worflow_id)?; } struct_ser.end() } } -impl<'de> serde::Deserialize<'de> for CreatePeerFlowResponse { +impl<'de> serde::Deserialize<'de> for CreateCdcFlowResponse { #[allow(deprecated)] fn deserialize(deserializer: D) -> std::result::Result where @@ -159,13 +159,13 @@ impl<'de> serde::Deserialize<'de> for CreatePeerFlowResponse { } struct GeneratedVisitor; impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { - type Value = CreatePeerFlowResponse; + type Value = CreateCdcFlowResponse; fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - formatter.write_str("struct peerdb_route.CreatePeerFlowResponse") + formatter.write_str("struct peerdb_route.CreateCDCFlowResponse") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -183,12 +183,12 @@ impl<'de> serde::Deserialize<'de> for CreatePeerFlowResponse { } } } - Ok(CreatePeerFlowResponse { + Ok(CreateCdcFlowResponse { worflow_id: worflow_id__.unwrap_or_default(), }) } } - deserializer.deserialize_struct("peerdb_route.CreatePeerFlowResponse", FIELDS, GeneratedVisitor) + deserializer.deserialize_struct("peerdb_route.CreateCDCFlowResponse", FIELDS, GeneratedVisitor) } } impl serde::Serialize for CreateQRepFlowRequest { diff --git a/nexus/pt/src/peerdb_route.tonic.rs b/nexus/pt/src/peerdb_route.tonic.rs index 62c5b9c58b..1f75c48893 100644 --- a/nexus/pt/src/peerdb_route.tonic.rs +++ b/nexus/pt/src/peerdb_route.tonic.rs @@ -112,11 +112,11 @@ pub mod flow_service_client { self.inner.unary(req, path, codec).await } /// - pub async fn create_peer_flow( + pub async fn create_cdc_flow( &mut self, - request: impl tonic::IntoRequest, + request: impl tonic::IntoRequest, ) -> std::result::Result< - tonic::Response, + tonic::Response, tonic::Status, > { self.inner @@ -130,11 +130,11 @@ pub mod flow_service_client { })?; let codec = tonic::codec::ProstCodec::default(); let path = http::uri::PathAndQuery::from_static( - "/peerdb_route.FlowService/CreatePeerFlow", + "/peerdb_route.FlowService/CreateCDCFlow", ); let mut req = request.into_request(); req.extensions_mut() - .insert(GrpcMethod::new("peerdb_route.FlowService", "CreatePeerFlow")); + .insert(GrpcMethod::new("peerdb_route.FlowService", "CreateCDCFlow")); self.inner.unary(req, path, codec).await } /// @@ -207,11 +207,11 @@ pub mod flow_service_server { tonic::Status, >; /// - async fn create_peer_flow( + async fn create_cdc_flow( &self, - request: tonic::Request, + request: tonic::Request, ) -> std::result::Result< - tonic::Response, + tonic::Response, tonic::Status, >; /// @@ -355,25 +355,25 @@ pub mod flow_service_server { }; Box::pin(fut) } - "/peerdb_route.FlowService/CreatePeerFlow" => { + "/peerdb_route.FlowService/CreateCDCFlow" => { #[allow(non_camel_case_types)] - struct CreatePeerFlowSvc(pub Arc); + struct CreateCDCFlowSvc(pub Arc); impl< T: FlowService, - > tonic::server::UnaryService - for CreatePeerFlowSvc { - type Response = super::CreatePeerFlowResponse; + > tonic::server::UnaryService + for CreateCDCFlowSvc { + type Response = super::CreateCdcFlowResponse; type Future = BoxFuture< tonic::Response, tonic::Status, >; fn call( &mut self, - request: tonic::Request, + request: tonic::Request, ) -> Self::Future { let inner = Arc::clone(&self.0); let fut = async move { - (*inner).create_peer_flow(request).await + (*inner).create_cdc_flow(request).await }; Box::pin(fut) } @@ -385,7 +385,7 @@ pub mod flow_service_server { let inner = self.inner.clone(); let fut = async move { let inner = inner.0; - let method = CreatePeerFlowSvc(inner); + let method = CreateCDCFlowSvc(inner); let codec = tonic::codec::ProstCodec::default(); let mut grpc = tonic::server::Grpc::new(codec) .apply_compression_config( diff --git a/protos/route.proto b/protos/route.proto index 98c7d5e0e3..d7cd10f08e 100644 --- a/protos/route.proto +++ b/protos/route.proto @@ -6,11 +6,11 @@ import "flow.proto"; package peerdb_route; -message CreatePeerFlowRequest { +message CreateCDCFlowRequest { peerdb_flow.FlowConnectionConfigs connection_configs = 1; } -message CreatePeerFlowResponse { +message CreateCDCFlowResponse { string worflow_id = 1; } @@ -43,7 +43,7 @@ message ListPeersResponse { service FlowService { rpc ListPeers(ListPeersRequest) returns (ListPeersResponse) {} - rpc CreatePeerFlow(CreatePeerFlowRequest) returns (CreatePeerFlowResponse) {} + rpc CreateCDCFlow(CreateCDCFlowRequest) returns (CreateCDCFlowResponse) {} rpc CreateQRepFlow(CreateQRepFlowRequest) returns (CreateQRepFlowResponse) {} rpc ShutdownFlow(ShutdownRequest) returns (ShutdownResponse) {} } diff --git a/ui/grpc_generated/flow.ts b/ui/grpc_generated/flow.ts index c129730d5b..a7201b2e4f 100644 --- a/ui/grpc_generated/flow.ts +++ b/ui/grpc_generated/flow.ts @@ -78,6 +78,18 @@ export interface TableNameMapping { destinationTableName: string; } +export interface RelationMessageColumn { + flags: number; + name: string; + dataType: number; +} + +export interface RelationMessage { + relationId: number; + relationName: string; + columns: RelationMessageColumn[]; +} + export interface FlowConnectionConfigs { source: Peer | undefined; destination: Peer | undefined; @@ -105,6 +117,9 @@ export interface FlowConnectionConfigs { /** currently only works for snowflake */ softDelete: boolean; replicationSlotName: string; + /** the below two are for eventhub only */ + pushBatchSize: number; + pushParallelism: number; } export interface FlowConnectionConfigs_TableNameMappingEntry { @@ -124,6 +139,12 @@ export interface FlowConnectionConfigs_TableNameSchemaMappingEntry { export interface SyncFlowOptions { batchSize: number; + relationMessageMapping: { [key: number]: RelationMessage }; +} + +export interface SyncFlowOptions_RelationMessageMappingEntry { + key: number; + value: RelationMessage | undefined; } export interface NormalizeFlowOptions { @@ -139,6 +160,12 @@ export interface StartFlowInput { lastSyncState: LastSyncState | undefined; flowConnectionConfigs: FlowConnectionConfigs | undefined; syncFlowOptions: SyncFlowOptions | undefined; + relationMessageMapping: { [key: number]: RelationMessage }; +} + +export interface StartFlowInput_RelationMessageMappingEntry { + key: number; + value: RelationMessage | undefined; } export interface StartNormalizeInput { @@ -365,6 +392,23 @@ export interface DropFlowInput { flowName: string; } +export interface DeltaAddedColumn { + columnName: string; + columnType: string; +} + +export interface TableSchemaDelta { + srcTableName: string; + dstTableName: string; + addedColumns: DeltaAddedColumn[]; + droppedColumns: string[]; +} + +export interface ReplayTableSchemaDeltaInput { + flowConnectionConfigs: FlowConnectionConfigs | undefined; + tableSchemaDelta: TableSchemaDelta | undefined; +} + function createBaseTableNameMapping(): TableNameMapping { return { sourceTableName: "", destinationTableName: "" }; } @@ -439,6 +483,184 @@ export const TableNameMapping = { }, }; +function createBaseRelationMessageColumn(): RelationMessageColumn { + return { flags: 0, name: "", dataType: 0 }; +} + +export const RelationMessageColumn = { + encode(message: RelationMessageColumn, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.flags !== 0) { + writer.uint32(8).uint32(message.flags); + } + if (message.name !== "") { + writer.uint32(18).string(message.name); + } + if (message.dataType !== 0) { + writer.uint32(24).uint32(message.dataType); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): RelationMessageColumn { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseRelationMessageColumn(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 8) { + break; + } + + message.flags = reader.uint32(); + continue; + case 2: + if (tag !== 18) { + break; + } + + message.name = reader.string(); + continue; + case 3: + if (tag !== 24) { + break; + } + + message.dataType = reader.uint32(); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): RelationMessageColumn { + return { + flags: isSet(object.flags) ? Number(object.flags) : 0, + name: isSet(object.name) ? String(object.name) : "", + dataType: isSet(object.dataType) ? Number(object.dataType) : 0, + }; + }, + + toJSON(message: RelationMessageColumn): unknown { + const obj: any = {}; + if (message.flags !== 0) { + obj.flags = Math.round(message.flags); + } + if (message.name !== "") { + obj.name = message.name; + } + if (message.dataType !== 0) { + obj.dataType = Math.round(message.dataType); + } + return obj; + }, + + create, I>>(base?: I): RelationMessageColumn { + return RelationMessageColumn.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): RelationMessageColumn { + const message = createBaseRelationMessageColumn(); + message.flags = object.flags ?? 0; + message.name = object.name ?? ""; + message.dataType = object.dataType ?? 0; + return message; + }, +}; + +function createBaseRelationMessage(): RelationMessage { + return { relationId: 0, relationName: "", columns: [] }; +} + +export const RelationMessage = { + encode(message: RelationMessage, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.relationId !== 0) { + writer.uint32(8).uint32(message.relationId); + } + if (message.relationName !== "") { + writer.uint32(18).string(message.relationName); + } + for (const v of message.columns) { + RelationMessageColumn.encode(v!, writer.uint32(26).fork()).ldelim(); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): RelationMessage { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseRelationMessage(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 8) { + break; + } + + message.relationId = reader.uint32(); + continue; + case 2: + if (tag !== 18) { + break; + } + + message.relationName = reader.string(); + continue; + case 3: + if (tag !== 26) { + break; + } + + message.columns.push(RelationMessageColumn.decode(reader, reader.uint32())); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): RelationMessage { + return { + relationId: isSet(object.relationId) ? Number(object.relationId) : 0, + relationName: isSet(object.relationName) ? String(object.relationName) : "", + columns: Array.isArray(object?.columns) ? object.columns.map((e: any) => RelationMessageColumn.fromJSON(e)) : [], + }; + }, + + toJSON(message: RelationMessage): unknown { + const obj: any = {}; + if (message.relationId !== 0) { + obj.relationId = Math.round(message.relationId); + } + if (message.relationName !== "") { + obj.relationName = message.relationName; + } + if (message.columns?.length) { + obj.columns = message.columns.map((e) => RelationMessageColumn.toJSON(e)); + } + return obj; + }, + + create, I>>(base?: I): RelationMessage { + return RelationMessage.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): RelationMessage { + const message = createBaseRelationMessage(); + message.relationId = object.relationId ?? 0; + message.relationName = object.relationName ?? ""; + message.columns = object.columns?.map((e) => RelationMessageColumn.fromPartial(e)) || []; + return message; + }, +}; + function createBaseFlowConnectionConfigs(): FlowConnectionConfigs { return { source: undefined, @@ -461,6 +683,8 @@ function createBaseFlowConnectionConfigs(): FlowConnectionConfigs { cdcStagingPath: "", softDelete: false, replicationSlotName: "", + pushBatchSize: 0, + pushParallelism: 0, }; } @@ -528,6 +752,12 @@ export const FlowConnectionConfigs = { if (message.replicationSlotName !== "") { writer.uint32(162).string(message.replicationSlotName); } + if (message.pushBatchSize !== 0) { + writer.uint32(168).int64(message.pushBatchSize); + } + if (message.pushParallelism !== 0) { + writer.uint32(176).int64(message.pushParallelism); + } return writer; }, @@ -687,6 +917,20 @@ export const FlowConnectionConfigs = { message.replicationSlotName = reader.string(); continue; + case 21: + if (tag !== 168) { + break; + } + + message.pushBatchSize = longToNumber(reader.int64() as Long); + continue; + case 22: + if (tag !== 176) { + break; + } + + message.pushParallelism = longToNumber(reader.int64() as Long); + continue; } if ((tag & 7) === 4 || tag === 0) { break; @@ -739,6 +983,8 @@ export const FlowConnectionConfigs = { cdcStagingPath: isSet(object.cdcStagingPath) ? String(object.cdcStagingPath) : "", softDelete: isSet(object.softDelete) ? Boolean(object.softDelete) : false, replicationSlotName: isSet(object.replicationSlotName) ? String(object.replicationSlotName) : "", + pushBatchSize: isSet(object.pushBatchSize) ? Number(object.pushBatchSize) : 0, + pushParallelism: isSet(object.pushParallelism) ? Number(object.pushParallelism) : 0, }; }, @@ -822,6 +1068,12 @@ export const FlowConnectionConfigs = { if (message.replicationSlotName !== "") { obj.replicationSlotName = message.replicationSlotName; } + if (message.pushBatchSize !== 0) { + obj.pushBatchSize = Math.round(message.pushBatchSize); + } + if (message.pushParallelism !== 0) { + obj.pushParallelism = Math.round(message.pushParallelism); + } return obj; }, @@ -880,6 +1132,8 @@ export const FlowConnectionConfigs = { message.cdcStagingPath = object.cdcStagingPath ?? ""; message.softDelete = object.softDelete ?? false; message.replicationSlotName = object.replicationSlotName ?? ""; + message.pushBatchSize = object.pushBatchSize ?? 0; + message.pushParallelism = object.pushParallelism ?? 0; return message; }, }; @@ -1121,7 +1375,7 @@ export const FlowConnectionConfigs_TableNameSchemaMappingEntry = { }; function createBaseSyncFlowOptions(): SyncFlowOptions { - return { batchSize: 0 }; + return { batchSize: 0, relationMessageMapping: {} }; } export const SyncFlowOptions = { @@ -1129,6 +1383,9 @@ export const SyncFlowOptions = { if (message.batchSize !== 0) { writer.uint32(8).int32(message.batchSize); } + Object.entries(message.relationMessageMapping).forEach(([key, value]) => { + SyncFlowOptions_RelationMessageMappingEntry.encode({ key: key as any, value }, writer.uint32(18).fork()).ldelim(); + }); return writer; }, @@ -1146,6 +1403,16 @@ export const SyncFlowOptions = { message.batchSize = reader.int32(); continue; + case 2: + if (tag !== 18) { + break; + } + + const entry2 = SyncFlowOptions_RelationMessageMappingEntry.decode(reader, reader.uint32()); + if (entry2.value !== undefined) { + message.relationMessageMapping[entry2.key] = entry2.value; + } + continue; } if ((tag & 7) === 4 || tag === 0) { break; @@ -1156,7 +1423,18 @@ export const SyncFlowOptions = { }, fromJSON(object: any): SyncFlowOptions { - return { batchSize: isSet(object.batchSize) ? Number(object.batchSize) : 0 }; + return { + batchSize: isSet(object.batchSize) ? Number(object.batchSize) : 0, + relationMessageMapping: isObject(object.relationMessageMapping) + ? Object.entries(object.relationMessageMapping).reduce<{ [key: number]: RelationMessage }>( + (acc, [key, value]) => { + acc[Number(key)] = RelationMessage.fromJSON(value); + return acc; + }, + {}, + ) + : {}, + }; }, toJSON(message: SyncFlowOptions): unknown { @@ -1164,6 +1442,15 @@ export const SyncFlowOptions = { if (message.batchSize !== 0) { obj.batchSize = Math.round(message.batchSize); } + if (message.relationMessageMapping) { + const entries = Object.entries(message.relationMessageMapping); + if (entries.length > 0) { + obj.relationMessageMapping = {}; + entries.forEach(([k, v]) => { + obj.relationMessageMapping[k] = RelationMessage.toJSON(v); + }); + } + } return obj; }, @@ -1173,6 +1460,94 @@ export const SyncFlowOptions = { fromPartial, I>>(object: I): SyncFlowOptions { const message = createBaseSyncFlowOptions(); message.batchSize = object.batchSize ?? 0; + message.relationMessageMapping = Object.entries(object.relationMessageMapping ?? {}).reduce< + { [key: number]: RelationMessage } + >((acc, [key, value]) => { + if (value !== undefined) { + acc[Number(key)] = RelationMessage.fromPartial(value); + } + return acc; + }, {}); + return message; + }, +}; + +function createBaseSyncFlowOptions_RelationMessageMappingEntry(): SyncFlowOptions_RelationMessageMappingEntry { + return { key: 0, value: undefined }; +} + +export const SyncFlowOptions_RelationMessageMappingEntry = { + encode(message: SyncFlowOptions_RelationMessageMappingEntry, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.key !== 0) { + writer.uint32(8).uint32(message.key); + } + if (message.value !== undefined) { + RelationMessage.encode(message.value, writer.uint32(18).fork()).ldelim(); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): SyncFlowOptions_RelationMessageMappingEntry { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseSyncFlowOptions_RelationMessageMappingEntry(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 8) { + break; + } + + message.key = reader.uint32(); + continue; + case 2: + if (tag !== 18) { + break; + } + + message.value = RelationMessage.decode(reader, reader.uint32()); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): SyncFlowOptions_RelationMessageMappingEntry { + return { + key: isSet(object.key) ? Number(object.key) : 0, + value: isSet(object.value) ? RelationMessage.fromJSON(object.value) : undefined, + }; + }, + + toJSON(message: SyncFlowOptions_RelationMessageMappingEntry): unknown { + const obj: any = {}; + if (message.key !== 0) { + obj.key = Math.round(message.key); + } + if (message.value !== undefined) { + obj.value = RelationMessage.toJSON(message.value); + } + return obj; + }, + + create, I>>( + base?: I, + ): SyncFlowOptions_RelationMessageMappingEntry { + return SyncFlowOptions_RelationMessageMappingEntry.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>( + object: I, + ): SyncFlowOptions_RelationMessageMappingEntry { + const message = createBaseSyncFlowOptions_RelationMessageMappingEntry(); + message.key = object.key ?? 0; + message.value = (object.value !== undefined && object.value !== null) + ? RelationMessage.fromPartial(object.value) + : undefined; return message; }, }; @@ -1309,7 +1684,12 @@ export const LastSyncState = { }; function createBaseStartFlowInput(): StartFlowInput { - return { lastSyncState: undefined, flowConnectionConfigs: undefined, syncFlowOptions: undefined }; + return { + lastSyncState: undefined, + flowConnectionConfigs: undefined, + syncFlowOptions: undefined, + relationMessageMapping: {}, + }; } export const StartFlowInput = { @@ -1323,6 +1703,9 @@ export const StartFlowInput = { if (message.syncFlowOptions !== undefined) { SyncFlowOptions.encode(message.syncFlowOptions, writer.uint32(26).fork()).ldelim(); } + Object.entries(message.relationMessageMapping).forEach(([key, value]) => { + StartFlowInput_RelationMessageMappingEntry.encode({ key: key as any, value }, writer.uint32(34).fork()).ldelim(); + }); return writer; }, @@ -1354,6 +1737,16 @@ export const StartFlowInput = { message.syncFlowOptions = SyncFlowOptions.decode(reader, reader.uint32()); continue; + case 4: + if (tag !== 34) { + break; + } + + const entry4 = StartFlowInput_RelationMessageMappingEntry.decode(reader, reader.uint32()); + if (entry4.value !== undefined) { + message.relationMessageMapping[entry4.key] = entry4.value; + } + continue; } if ((tag & 7) === 4 || tag === 0) { break; @@ -1370,6 +1763,15 @@ export const StartFlowInput = { ? FlowConnectionConfigs.fromJSON(object.flowConnectionConfigs) : undefined, syncFlowOptions: isSet(object.syncFlowOptions) ? SyncFlowOptions.fromJSON(object.syncFlowOptions) : undefined, + relationMessageMapping: isObject(object.relationMessageMapping) + ? Object.entries(object.relationMessageMapping).reduce<{ [key: number]: RelationMessage }>( + (acc, [key, value]) => { + acc[Number(key)] = RelationMessage.fromJSON(value); + return acc; + }, + {}, + ) + : {}, }; }, @@ -1384,6 +1786,15 @@ export const StartFlowInput = { if (message.syncFlowOptions !== undefined) { obj.syncFlowOptions = SyncFlowOptions.toJSON(message.syncFlowOptions); } + if (message.relationMessageMapping) { + const entries = Object.entries(message.relationMessageMapping); + if (entries.length > 0) { + obj.relationMessageMapping = {}; + entries.forEach(([k, v]) => { + obj.relationMessageMapping[k] = RelationMessage.toJSON(v); + }); + } + } return obj; }, @@ -1402,6 +1813,94 @@ export const StartFlowInput = { message.syncFlowOptions = (object.syncFlowOptions !== undefined && object.syncFlowOptions !== null) ? SyncFlowOptions.fromPartial(object.syncFlowOptions) : undefined; + message.relationMessageMapping = Object.entries(object.relationMessageMapping ?? {}).reduce< + { [key: number]: RelationMessage } + >((acc, [key, value]) => { + if (value !== undefined) { + acc[Number(key)] = RelationMessage.fromPartial(value); + } + return acc; + }, {}); + return message; + }, +}; + +function createBaseStartFlowInput_RelationMessageMappingEntry(): StartFlowInput_RelationMessageMappingEntry { + return { key: 0, value: undefined }; +} + +export const StartFlowInput_RelationMessageMappingEntry = { + encode(message: StartFlowInput_RelationMessageMappingEntry, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.key !== 0) { + writer.uint32(8).uint32(message.key); + } + if (message.value !== undefined) { + RelationMessage.encode(message.value, writer.uint32(18).fork()).ldelim(); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): StartFlowInput_RelationMessageMappingEntry { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseStartFlowInput_RelationMessageMappingEntry(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 8) { + break; + } + + message.key = reader.uint32(); + continue; + case 2: + if (tag !== 18) { + break; + } + + message.value = RelationMessage.decode(reader, reader.uint32()); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): StartFlowInput_RelationMessageMappingEntry { + return { + key: isSet(object.key) ? Number(object.key) : 0, + value: isSet(object.value) ? RelationMessage.fromJSON(object.value) : undefined, + }; + }, + + toJSON(message: StartFlowInput_RelationMessageMappingEntry): unknown { + const obj: any = {}; + if (message.key !== 0) { + obj.key = Math.round(message.key); + } + if (message.value !== undefined) { + obj.value = RelationMessage.toJSON(message.value); + } + return obj; + }, + + create, I>>( + base?: I, + ): StartFlowInput_RelationMessageMappingEntry { + return StartFlowInput_RelationMessageMappingEntry.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>( + object: I, + ): StartFlowInput_RelationMessageMappingEntry { + const message = createBaseStartFlowInput_RelationMessageMappingEntry(); + message.key = object.key ?? 0; + message.value = (object.value !== undefined && object.value !== null) + ? RelationMessage.fromPartial(object.value) + : undefined; return message; }, }; @@ -4671,6 +5170,267 @@ export const DropFlowInput = { }, }; +function createBaseDeltaAddedColumn(): DeltaAddedColumn { + return { columnName: "", columnType: "" }; +} + +export const DeltaAddedColumn = { + encode(message: DeltaAddedColumn, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.columnName !== "") { + writer.uint32(10).string(message.columnName); + } + if (message.columnType !== "") { + writer.uint32(18).string(message.columnType); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): DeltaAddedColumn { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseDeltaAddedColumn(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.columnName = reader.string(); + continue; + case 2: + if (tag !== 18) { + break; + } + + message.columnType = reader.string(); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): DeltaAddedColumn { + return { + columnName: isSet(object.columnName) ? String(object.columnName) : "", + columnType: isSet(object.columnType) ? String(object.columnType) : "", + }; + }, + + toJSON(message: DeltaAddedColumn): unknown { + const obj: any = {}; + if (message.columnName !== "") { + obj.columnName = message.columnName; + } + if (message.columnType !== "") { + obj.columnType = message.columnType; + } + return obj; + }, + + create, I>>(base?: I): DeltaAddedColumn { + return DeltaAddedColumn.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): DeltaAddedColumn { + const message = createBaseDeltaAddedColumn(); + message.columnName = object.columnName ?? ""; + message.columnType = object.columnType ?? ""; + return message; + }, +}; + +function createBaseTableSchemaDelta(): TableSchemaDelta { + return { srcTableName: "", dstTableName: "", addedColumns: [], droppedColumns: [] }; +} + +export const TableSchemaDelta = { + encode(message: TableSchemaDelta, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.srcTableName !== "") { + writer.uint32(10).string(message.srcTableName); + } + if (message.dstTableName !== "") { + writer.uint32(18).string(message.dstTableName); + } + for (const v of message.addedColumns) { + DeltaAddedColumn.encode(v!, writer.uint32(26).fork()).ldelim(); + } + for (const v of message.droppedColumns) { + writer.uint32(34).string(v!); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): TableSchemaDelta { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseTableSchemaDelta(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.srcTableName = reader.string(); + continue; + case 2: + if (tag !== 18) { + break; + } + + message.dstTableName = reader.string(); + continue; + case 3: + if (tag !== 26) { + break; + } + + message.addedColumns.push(DeltaAddedColumn.decode(reader, reader.uint32())); + continue; + case 4: + if (tag !== 34) { + break; + } + + message.droppedColumns.push(reader.string()); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): TableSchemaDelta { + return { + srcTableName: isSet(object.srcTableName) ? String(object.srcTableName) : "", + dstTableName: isSet(object.dstTableName) ? String(object.dstTableName) : "", + addedColumns: Array.isArray(object?.addedColumns) + ? object.addedColumns.map((e: any) => DeltaAddedColumn.fromJSON(e)) + : [], + droppedColumns: Array.isArray(object?.droppedColumns) ? object.droppedColumns.map((e: any) => String(e)) : [], + }; + }, + + toJSON(message: TableSchemaDelta): unknown { + const obj: any = {}; + if (message.srcTableName !== "") { + obj.srcTableName = message.srcTableName; + } + if (message.dstTableName !== "") { + obj.dstTableName = message.dstTableName; + } + if (message.addedColumns?.length) { + obj.addedColumns = message.addedColumns.map((e) => DeltaAddedColumn.toJSON(e)); + } + if (message.droppedColumns?.length) { + obj.droppedColumns = message.droppedColumns; + } + return obj; + }, + + create, I>>(base?: I): TableSchemaDelta { + return TableSchemaDelta.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): TableSchemaDelta { + const message = createBaseTableSchemaDelta(); + message.srcTableName = object.srcTableName ?? ""; + message.dstTableName = object.dstTableName ?? ""; + message.addedColumns = object.addedColumns?.map((e) => DeltaAddedColumn.fromPartial(e)) || []; + message.droppedColumns = object.droppedColumns?.map((e) => e) || []; + return message; + }, +}; + +function createBaseReplayTableSchemaDeltaInput(): ReplayTableSchemaDeltaInput { + return { flowConnectionConfigs: undefined, tableSchemaDelta: undefined }; +} + +export const ReplayTableSchemaDeltaInput = { + encode(message: ReplayTableSchemaDeltaInput, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.flowConnectionConfigs !== undefined) { + FlowConnectionConfigs.encode(message.flowConnectionConfigs, writer.uint32(10).fork()).ldelim(); + } + if (message.tableSchemaDelta !== undefined) { + TableSchemaDelta.encode(message.tableSchemaDelta, writer.uint32(18).fork()).ldelim(); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): ReplayTableSchemaDeltaInput { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseReplayTableSchemaDeltaInput(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.flowConnectionConfigs = FlowConnectionConfigs.decode(reader, reader.uint32()); + continue; + case 2: + if (tag !== 18) { + break; + } + + message.tableSchemaDelta = TableSchemaDelta.decode(reader, reader.uint32()); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): ReplayTableSchemaDeltaInput { + return { + flowConnectionConfigs: isSet(object.flowConnectionConfigs) + ? FlowConnectionConfigs.fromJSON(object.flowConnectionConfigs) + : undefined, + tableSchemaDelta: isSet(object.tableSchemaDelta) ? TableSchemaDelta.fromJSON(object.tableSchemaDelta) : undefined, + }; + }, + + toJSON(message: ReplayTableSchemaDeltaInput): unknown { + const obj: any = {}; + if (message.flowConnectionConfigs !== undefined) { + obj.flowConnectionConfigs = FlowConnectionConfigs.toJSON(message.flowConnectionConfigs); + } + if (message.tableSchemaDelta !== undefined) { + obj.tableSchemaDelta = TableSchemaDelta.toJSON(message.tableSchemaDelta); + } + return obj; + }, + + create, I>>(base?: I): ReplayTableSchemaDeltaInput { + return ReplayTableSchemaDeltaInput.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): ReplayTableSchemaDeltaInput { + const message = createBaseReplayTableSchemaDeltaInput(); + message.flowConnectionConfigs = + (object.flowConnectionConfigs !== undefined && object.flowConnectionConfigs !== null) + ? FlowConnectionConfigs.fromPartial(object.flowConnectionConfigs) + : undefined; + message.tableSchemaDelta = (object.tableSchemaDelta !== undefined && object.tableSchemaDelta !== null) + ? TableSchemaDelta.fromPartial(object.tableSchemaDelta) + : undefined; + return message; + }, +}; + declare const self: any | undefined; declare const window: any | undefined; declare const global: any | undefined; diff --git a/ui/grpc_generated/route.ts b/ui/grpc_generated/route.ts index 3abcee4687..387a418b0c 100644 --- a/ui/grpc_generated/route.ts +++ b/ui/grpc_generated/route.ts @@ -17,11 +17,11 @@ import { Peer } from "./peers"; export const protobufPackage = "peerdb_route"; -export interface CreatePeerFlowRequest { +export interface CreateCDCFlowRequest { connectionConfigs: FlowConnectionConfigs | undefined; } -export interface CreatePeerFlowResponse { +export interface CreateCDCFlowResponse { worflowId: string; } @@ -52,22 +52,22 @@ export interface ListPeersResponse { peers: Peer[]; } -function createBaseCreatePeerFlowRequest(): CreatePeerFlowRequest { +function createBaseCreateCDCFlowRequest(): CreateCDCFlowRequest { return { connectionConfigs: undefined }; } -export const CreatePeerFlowRequest = { - encode(message: CreatePeerFlowRequest, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { +export const CreateCDCFlowRequest = { + encode(message: CreateCDCFlowRequest, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { if (message.connectionConfigs !== undefined) { FlowConnectionConfigs.encode(message.connectionConfigs, writer.uint32(10).fork()).ldelim(); } return writer; }, - decode(input: _m0.Reader | Uint8Array, length?: number): CreatePeerFlowRequest { + decode(input: _m0.Reader | Uint8Array, length?: number): CreateCDCFlowRequest { const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); let end = length === undefined ? reader.len : reader.pos + length; - const message = createBaseCreatePeerFlowRequest(); + const message = createBaseCreateCDCFlowRequest(); while (reader.pos < end) { const tag = reader.uint32(); switch (tag >>> 3) { @@ -87,7 +87,7 @@ export const CreatePeerFlowRequest = { return message; }, - fromJSON(object: any): CreatePeerFlowRequest { + fromJSON(object: any): CreateCDCFlowRequest { return { connectionConfigs: isSet(object.connectionConfigs) ? FlowConnectionConfigs.fromJSON(object.connectionConfigs) @@ -95,7 +95,7 @@ export const CreatePeerFlowRequest = { }; }, - toJSON(message: CreatePeerFlowRequest): unknown { + toJSON(message: CreateCDCFlowRequest): unknown { const obj: any = {}; if (message.connectionConfigs !== undefined) { obj.connectionConfigs = FlowConnectionConfigs.toJSON(message.connectionConfigs); @@ -103,11 +103,11 @@ export const CreatePeerFlowRequest = { return obj; }, - create, I>>(base?: I): CreatePeerFlowRequest { - return CreatePeerFlowRequest.fromPartial(base ?? ({} as any)); + create, I>>(base?: I): CreateCDCFlowRequest { + return CreateCDCFlowRequest.fromPartial(base ?? ({} as any)); }, - fromPartial, I>>(object: I): CreatePeerFlowRequest { - const message = createBaseCreatePeerFlowRequest(); + fromPartial, I>>(object: I): CreateCDCFlowRequest { + const message = createBaseCreateCDCFlowRequest(); message.connectionConfigs = (object.connectionConfigs !== undefined && object.connectionConfigs !== null) ? FlowConnectionConfigs.fromPartial(object.connectionConfigs) : undefined; @@ -115,22 +115,22 @@ export const CreatePeerFlowRequest = { }, }; -function createBaseCreatePeerFlowResponse(): CreatePeerFlowResponse { +function createBaseCreateCDCFlowResponse(): CreateCDCFlowResponse { return { worflowId: "" }; } -export const CreatePeerFlowResponse = { - encode(message: CreatePeerFlowResponse, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { +export const CreateCDCFlowResponse = { + encode(message: CreateCDCFlowResponse, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { if (message.worflowId !== "") { writer.uint32(10).string(message.worflowId); } return writer; }, - decode(input: _m0.Reader | Uint8Array, length?: number): CreatePeerFlowResponse { + decode(input: _m0.Reader | Uint8Array, length?: number): CreateCDCFlowResponse { const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); let end = length === undefined ? reader.len : reader.pos + length; - const message = createBaseCreatePeerFlowResponse(); + const message = createBaseCreateCDCFlowResponse(); while (reader.pos < end) { const tag = reader.uint32(); switch (tag >>> 3) { @@ -150,11 +150,11 @@ export const CreatePeerFlowResponse = { return message; }, - fromJSON(object: any): CreatePeerFlowResponse { + fromJSON(object: any): CreateCDCFlowResponse { return { worflowId: isSet(object.worflowId) ? String(object.worflowId) : "" }; }, - toJSON(message: CreatePeerFlowResponse): unknown { + toJSON(message: CreateCDCFlowResponse): unknown { const obj: any = {}; if (message.worflowId !== "") { obj.worflowId = message.worflowId; @@ -162,11 +162,11 @@ export const CreatePeerFlowResponse = { return obj; }, - create, I>>(base?: I): CreatePeerFlowResponse { - return CreatePeerFlowResponse.fromPartial(base ?? ({} as any)); + create, I>>(base?: I): CreateCDCFlowResponse { + return CreateCDCFlowResponse.fromPartial(base ?? ({} as any)); }, - fromPartial, I>>(object: I): CreatePeerFlowResponse { - const message = createBaseCreatePeerFlowResponse(); + fromPartial, I>>(object: I): CreateCDCFlowResponse { + const message = createBaseCreateCDCFlowResponse(); message.worflowId = object.worflowId ?? ""; return message; }, @@ -581,14 +581,14 @@ export const FlowServiceService = { responseSerialize: (value: ListPeersResponse) => Buffer.from(ListPeersResponse.encode(value).finish()), responseDeserialize: (value: Buffer) => ListPeersResponse.decode(value), }, - createPeerFlow: { - path: "/peerdb_route.FlowService/CreatePeerFlow", + createCdcFlow: { + path: "/peerdb_route.FlowService/CreateCDCFlow", requestStream: false, responseStream: false, - requestSerialize: (value: CreatePeerFlowRequest) => Buffer.from(CreatePeerFlowRequest.encode(value).finish()), - requestDeserialize: (value: Buffer) => CreatePeerFlowRequest.decode(value), - responseSerialize: (value: CreatePeerFlowResponse) => Buffer.from(CreatePeerFlowResponse.encode(value).finish()), - responseDeserialize: (value: Buffer) => CreatePeerFlowResponse.decode(value), + requestSerialize: (value: CreateCDCFlowRequest) => Buffer.from(CreateCDCFlowRequest.encode(value).finish()), + requestDeserialize: (value: Buffer) => CreateCDCFlowRequest.decode(value), + responseSerialize: (value: CreateCDCFlowResponse) => Buffer.from(CreateCDCFlowResponse.encode(value).finish()), + responseDeserialize: (value: Buffer) => CreateCDCFlowResponse.decode(value), }, createQRepFlow: { path: "/peerdb_route.FlowService/CreateQRepFlow", @@ -612,7 +612,7 @@ export const FlowServiceService = { export interface FlowServiceServer extends UntypedServiceImplementation { listPeers: handleUnaryCall; - createPeerFlow: handleUnaryCall; + createCdcFlow: handleUnaryCall; createQRepFlow: handleUnaryCall; shutdownFlow: handleUnaryCall; } @@ -633,20 +633,20 @@ export interface FlowServiceClient extends Client { options: Partial, callback: (error: ServiceError | null, response: ListPeersResponse) => void, ): ClientUnaryCall; - createPeerFlow( - request: CreatePeerFlowRequest, - callback: (error: ServiceError | null, response: CreatePeerFlowResponse) => void, + createCdcFlow( + request: CreateCDCFlowRequest, + callback: (error: ServiceError | null, response: CreateCDCFlowResponse) => void, ): ClientUnaryCall; - createPeerFlow( - request: CreatePeerFlowRequest, + createCdcFlow( + request: CreateCDCFlowRequest, metadata: Metadata, - callback: (error: ServiceError | null, response: CreatePeerFlowResponse) => void, + callback: (error: ServiceError | null, response: CreateCDCFlowResponse) => void, ): ClientUnaryCall; - createPeerFlow( - request: CreatePeerFlowRequest, + createCdcFlow( + request: CreateCDCFlowRequest, metadata: Metadata, options: Partial, - callback: (error: ServiceError | null, response: CreatePeerFlowResponse) => void, + callback: (error: ServiceError | null, response: CreateCDCFlowResponse) => void, ): ClientUnaryCall; createQRepFlow( request: CreateQRepFlowRequest, From 903af3e181d11d53e6daf4ed0020964dd73733e3 Mon Sep 17 00:00:00 2001 From: Kevin Biju Date: Wed, 13 Sep 2023 00:08:56 +0530 Subject: [PATCH 4/4] fixed Rust refactors --- nexus/flow-rs/src/grpc.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/nexus/flow-rs/src/grpc.rs b/nexus/flow-rs/src/grpc.rs index c06d7aa71e..7b21fb46c1 100644 --- a/nexus/flow-rs/src/grpc.rs +++ b/nexus/flow-rs/src/grpc.rs @@ -86,10 +86,10 @@ impl FlowGrpcClient { &mut self, peer_flow_config: pt::peerdb_flow::FlowConnectionConfigs, ) -> anyhow::Result { - let create_peer_flow_req = pt::peerdb_route::CreatePeerFlowRequest { + let create_peer_flow_req = pt::peerdb_route::CreateCdcFlowRequest { connection_configs: Some(peer_flow_config), }; - let response = self.client.create_peer_flow(create_peer_flow_req).await?; + let response = self.client.create_cdc_flow(create_peer_flow_req).await?; let workflow_id = response.into_inner().worflow_id; Ok(workflow_id) }