From 783869223769899bdbc13b585bd415dd2f9fcb12 Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Mon, 28 Aug 2023 12:14:53 -0700 Subject: [PATCH 01/12] simplify bound selector always wait for the first future (#360) --- flow/concurrency/bound_selector.go | 65 ++++++++++++++---------------- 1 file changed, 30 insertions(+), 35 deletions(-) diff --git a/flow/concurrency/bound_selector.go b/flow/concurrency/bound_selector.go index 56f5b42891..29735c09d6 100644 --- a/flow/concurrency/bound_selector.go +++ b/flow/concurrency/bound_selector.go @@ -7,54 +7,49 @@ import ( ) type BoundSelector struct { - ctx workflow.Context - limit workflow.Channel - statusCh workflow.Channel - numFutures int + ctx workflow.Context + limit int + futures []workflow.Future + ferrors []error } func NewBoundSelector(limit int, total int, ctx workflow.Context) *BoundSelector { return &BoundSelector{ - ctx: ctx, - limit: workflow.NewBufferedChannel(ctx, limit), - statusCh: workflow.NewBufferedChannel(ctx, total), - numFutures: 0, + ctx: ctx, + limit: limit, } } func (s *BoundSelector) SpawnChild(chCtx workflow.Context, w interface{}, args ...interface{}) { - s.numFutures++ - workflow.Go(s.ctx, func(ctx workflow.Context) { - s.limit.Send(ctx, struct{}{}) - future := workflow.ExecuteChildWorkflow(chCtx, w, args...) - err := future.Get(ctx, nil) - s.statusCh.Send(ctx, err) - s.limit.Receive(ctx, nil) - }) + if len(s.futures) >= s.limit { + s.waitOne() + } + + future := workflow.ExecuteChildWorkflow(chCtx, w, args...) + s.futures = append(s.futures, future) +} + +func (s *BoundSelector) waitOne() { + if len(s.futures) == 0 { + return + } + + f := s.futures[0] + s.futures = s.futures[1:] + + err := f.Get(s.ctx, nil) + if err != nil { + s.ferrors = append(s.ferrors, err) + } } func (s *BoundSelector) Wait() error { - defer s.statusCh.Close() - defer s.limit.Close() - - ferrors := make([]error, 0) - doneCount := 0 - - for doneCount < s.numFutures { - selector := workflow.NewSelector(s.ctx) - selector.AddReceive(s.statusCh, func(c workflow.ReceiveChannel, more bool) { - var err error - c.Receive(s.ctx, &err) - if err != nil { - ferrors = append(ferrors, err) - } - doneCount++ - }) - selector.Select(s.ctx) + for len(s.futures) > 0 { + s.waitOne() } - if len(ferrors) > 0 { - return errors.Join(ferrors...) + if len(s.ferrors) > 0 { + return errors.Join(s.ferrors...) } return nil From 68f4ae8eec105babeec795302146c1091a53a8c9 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Tue, 29 Aug 2023 21:26:09 +0530 Subject: [PATCH 02/12] Relevant fields to logs for more info (#347) Adds fields such as flow job name, partition ID and other relevant fields in function logs. Fixes #337 --- flow/activities/flowable.go | 36 +++++-- flow/activities/snapshot_activity.go | 12 ++- flow/connectors/bigquery/qrep.go | 10 +- flow/connectors/bigquery/qrep_avro_sync.go | 34 ++++-- flow/connectors/bigquery/qrep_sync_method.go | 10 +- flow/connectors/eventhub/eventhub.go | 5 +- flow/connectors/eventhub/metadata.go | 8 +- flow/connectors/postgres/cdc.go | 12 ++- flow/connectors/postgres/client.go | 4 +- flow/connectors/postgres/postgres.go | 54 +++++++--- flow/connectors/postgres/qrep.go | 69 +++++++++--- flow/connectors/postgres/qrep_bench_test.go | 2 +- .../postgres/qrep_query_build_test.go | 2 +- .../postgres/qrep_query_executor.go | 101 +++++++++++++----- .../postgres/qrep_query_executor_test.go | 6 +- flow/connectors/postgres/qrep_sync_method.go | 39 +++++-- flow/connectors/s3/qrep.go | 5 +- flow/connectors/snowflake/qrep.go | 54 +++++++--- flow/connectors/snowflake/qrep_avro_sync.go | 89 ++++++++++++--- flow/connectors/snowflake/snowflake.go | 15 ++- flow/e2e/qrep_flow_test.go | 7 +- 21 files changed, 447 insertions(+), 127 deletions(-) diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index 776b43c9fd..5893b15859 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -176,13 +176,17 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, input *protos.StartFlo return nil, fmt.Errorf("failed to get destination connector: %w", err) } - log.Info("initializing table schema...") + log.WithFields(log.Fields{ + "flowName": input.FlowConnectionConfigs.FlowJobName, + }).Infof("initializing table schema...") err = dest.InitializeTableSchema(input.FlowConnectionConfigs.TableNameSchemaMapping) if err != nil { return nil, fmt.Errorf("failed to initialize table schema: %w", err) } - log.Info("pulling records...") + log.WithFields(log.Fields{ + "flowName": input.FlowConnectionConfigs.FlowJobName, + }).Info("pulling records...") startTime := time.Now() records, err := src.PullRecords(&model.PullRecordsRequest{ @@ -220,11 +224,15 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, input *protos.StartFlo // log the number of records numRecords := len(records.Records) - log.Printf("pulled %d records", numRecords) + log.WithFields(log.Fields{ + "flowName": input.FlowConnectionConfigs.FlowJobName, + }).Printf("pulled %d records", numRecords) activity.RecordHeartbeat(ctx, fmt.Sprintf("pulled %d records", numRecords)) if numRecords == 0 { - log.Info("no records to push") + log.WithFields(log.Fields{ + "flowName": input.FlowConnectionConfigs.FlowJobName, + }).Info("no records to push") return nil, nil } @@ -238,7 +246,9 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, input *protos.StartFlo log.Warnf("failed to push records: %v", err) return nil, fmt.Errorf("failed to push records: %w", err) } - log.Info("pushed records") + log.WithFields(log.Fields{ + "flowName": input.FlowConnectionConfigs.FlowJobName, + }).Infof("pushed %d records", res.NumRecordsSynced) err = a.CatalogMirrorMonitor. UpdateLatestLSNAtTargetForCDCFlow(ctx, input.FlowConnectionConfigs.FlowJobName, @@ -402,7 +412,9 @@ func (a *FlowableActivity) ReplicateQRepPartition(ctx context.Context, tmp, err := pgConn.PullQRepRecordStream(config, partition, stream) numRecords = int64(tmp) if err != nil { - log.Errorf("failed to pull records: %v", err) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Errorf("failed to pull records: %v", err) goroutineErr = err } err = a.CatalogMirrorMonitor.UpdatePullEndTimeAndRowsForPartition(ctx, runUUID, partition, numRecords) @@ -420,7 +432,9 @@ func (a *FlowableActivity) ReplicateQRepPartition(ctx context.Context, return fmt.Errorf("failed to pull records: %w", err) } numRecords = int64(recordBatch.NumRecords) - log.Printf("pulled %d records\n", len(recordBatch.Records)) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Printf("pulled %d records\n", len(recordBatch.Records)) err = a.CatalogMirrorMonitor.UpdatePullEndTimeAndRowsForPartition(ctx, runUUID, partition, numRecords) if err != nil { @@ -447,7 +461,9 @@ func (a *FlowableActivity) ReplicateQRepPartition(ctx context.Context, } if res == 0 { - log.Printf("no records to push for partition %s\n", partition.PartitionId) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Printf("no records to push for partition %s\n", partition.PartitionId) return nil } @@ -455,7 +471,9 @@ func (a *FlowableActivity) ReplicateQRepPartition(ctx context.Context, if goroutineErr != nil { return goroutineErr } - log.Printf("pushed %d records\n", res) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Printf("pushed %d records\n", res) err = a.CatalogMirrorMonitor.UpdateEndTimeForPartition(ctx, runUUID, partition) if err != nil { return err diff --git a/flow/activities/snapshot_activity.go b/flow/activities/snapshot_activity.go index 30d74f1d01..efd2950b59 100644 --- a/flow/activities/snapshot_activity.go +++ b/flow/activities/snapshot_activity.go @@ -54,17 +54,23 @@ func (a *SnapshotActivity) SetupReplication( pgConn := conn.(*connpostgres.PostgresConnector) err = pgConn.SetupReplication(slotSignal, config) if err != nil { - log.Errorf("failed to setup replication: %v", err) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Errorf("failed to setup replication: %v", err) replicationErr <- err return } }() - log.Info("waiting for slot to be created...") + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Info("waiting for slot to be created...") var slotInfo *connpostgres.SlotCreationResult select { case slotInfo = <-slotSignal.SlotCreated: - log.Infof("slot '%s' created", slotInfo.SlotName) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Infof("slot '%s' created", slotInfo.SlotName) case err := <-replicationErr: return nil, fmt.Errorf("failed to setup replication: %w", err) } diff --git a/flow/connectors/bigquery/qrep.go b/flow/connectors/bigquery/qrep.go index 9ae060a1fd..e1ee8cf3c5 100644 --- a/flow/connectors/bigquery/qrep.go +++ b/flow/connectors/bigquery/qrep.go @@ -44,9 +44,17 @@ func (c *BigQueryConnector) SyncQRepRecords( } if done { - log.Infof("Partition %s has already been synced", partition.PartitionId) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + "partitionID": partition.PartitionId, + }).Infof("Partition %s has already been synced", partition.PartitionId) return 0, nil } + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Infof("QRep sync function called and partition existence checked for"+ + " partition %s of destination table %s", + partition.PartitionId, destTable) syncMode := config.SyncMode switch syncMode { diff --git a/flow/connectors/bigquery/qrep_avro_sync.go b/flow/connectors/bigquery/qrep_avro_sync.go index ce90de6c14..a4cd92a82c 100644 --- a/flow/connectors/bigquery/qrep_avro_sync.go +++ b/flow/connectors/bigquery/qrep_avro_sync.go @@ -85,7 +85,11 @@ func (s *QRepAvroSyncMethod) SyncRecords( // drop the staging table if err := bqClient.Dataset(datasetID).Table(stagingTable).Delete(s.connector.ctx); err != nil { // just log the error this isn't fatal. - log.Errorf("failed to delete staging table %s: %v", stagingTable, err) + log.WithFields(log.Fields{ + "flowName": flowJobName, + "syncBatchID": syncBatchID, + "destinationTable": dstTableName, + }).Errorf("failed to delete staging table %s: %v", stagingTable, err) } log.Printf("loaded stage into %s.%s", @@ -108,7 +112,10 @@ func (s *QRepAvroSyncMethod) SyncQRepRecords( if err != nil { return 0, fmt.Errorf("failed to define Avro schema: %w", err) } - + log.WithFields(log.Fields{ + "flowName": flowJobName, + }).Infof("Obtained Avro schema for destination table %s and partition ID %s", + dstTableName, partition.PartitionId) fmt.Printf("Avro schema: %s\n", avroSchema) // create a staging table name with partitionID replace hyphens with underscores stagingTable := fmt.Sprintf("%s_%s_staging", dstTableName, strings.ReplaceAll(partition.PartitionId, "-", "_")) @@ -136,6 +143,10 @@ func (s *QRepAvroSyncMethod) SyncQRepRecords( if err != nil { return -1, fmt.Errorf("failed to create metadata insert statement: %v", err) } + log.WithFields(log.Fields{ + "flowName": flowJobName, + }).Infof("Performing transaction inside QRep sync function for partition ID %s", + partition.PartitionId) stmts = append(stmts, insertMetadataStmt) stmts = append(stmts, "COMMIT TRANSACTION;") // Execute the statements in a transaction @@ -150,10 +161,17 @@ func (s *QRepAvroSyncMethod) SyncQRepRecords( // drop the staging table if err := bqClient.Dataset(datasetID).Table(stagingTable).Delete(s.connector.ctx); err != nil { // just log the error this isn't fatal. - log.Errorf("failed to delete staging table %s: %v", stagingTable, err) + log.WithFields(log.Fields{ + "flowName": flowJobName, + "partitionID": partition.PartitionId, + "destinationTable": dstTableName, + }).Errorf("failed to delete staging table %s: %v", stagingTable, err) } - log.Printf("loaded stage into %s.%s", + log.WithFields(log.Fields{ + "flowName": flowJobName, + "partitionID": partition.PartitionId, + }).Infof("loaded stage into %s.%s", datasetID, dstTableName) return numRecords, nil } @@ -326,7 +344,9 @@ func (s *QRepAvroSyncMethod) writeToStage( schema, err := stream.Schema() if err != nil { - log.Errorf("failed to get schema from stream: %v", err) + log.WithFields(log.Fields{ + "partitonOrBatchID": syncID, + }).Errorf("failed to get schema from stream: %v", err) return 0, fmt.Errorf("failed to get schema from stream: %w", err) } @@ -344,7 +364,9 @@ func (s *QRepAvroSyncMethod) writeToStage( ) } if qRecordOrErr.Err != nil { - log.Errorf("[bq_avro] failed to get record from stream: %v", qRecordOrErr.Err) + log.WithFields(log.Fields{ + "batchOrPartitionID": syncID, + }).Errorf("[bq_avro] failed to get record from stream: %v", qRecordOrErr.Err) return 0, fmt.Errorf("[bq_avro] failed to get record from stream: %w", qRecordOrErr.Err) } diff --git a/flow/connectors/bigquery/qrep_sync_method.go b/flow/connectors/bigquery/qrep_sync_method.go index f68463d458..2a8eb0f5cd 100644 --- a/flow/connectors/bigquery/qrep_sync_method.go +++ b/flow/connectors/bigquery/qrep_sync_method.go @@ -73,7 +73,10 @@ func (s *QRepStagingTableSync) SyncQRepRecords( schema, err := stream.Schema() if err != nil { - log.Errorf("failed to get schema from stream: %v", err) + log.WithFields(log.Fields{ + "flowName": flowJobName, + "partitionID": partitionID, + }).Errorf("failed to get schema from stream: %v", err) return 0, fmt.Errorf("failed to get schema from stream: %w", err) } @@ -81,7 +84,10 @@ func (s *QRepStagingTableSync) SyncQRepRecords( valueSaverRecords := make([]bigquery.ValueSaver, 0) for qRecordOrErr := range stream.Records { if qRecordOrErr.Err != nil { - log.Errorf("[bq] failed to get record from stream: %v", qRecordOrErr.Err) + log.WithFields(log.Fields{ + "flowName": flowJobName, + "partitionID": partitionID, + }).Errorf("[bq] failed to get record from stream: %v", qRecordOrErr.Err) return 0, fmt.Errorf("[bq] failed to get record from stream: %w", qRecordOrErr.Err) } diff --git a/flow/connectors/eventhub/eventhub.go b/flow/connectors/eventhub/eventhub.go index 13e62b2897..f94b5b0d3d 100644 --- a/flow/connectors/eventhub/eventhub.go +++ b/flow/connectors/eventhub/eventhub.go @@ -232,7 +232,10 @@ func (c *EventHubConnector) CreateRawTable(req *protos.CreateRawTableInput) (*pr for _, table := range tableMap { err := c.ensureEventHub(c.ctx, table) if err != nil { - log.Errorf("failed to get event hub properties: %v", err) + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + "table": table, + }).Errorf("failed to get event hub properties: %v", err) return nil, err } } diff --git a/flow/connectors/eventhub/metadata.go b/flow/connectors/eventhub/metadata.go index 09cdb6c315..84175b0d27 100644 --- a/flow/connectors/eventhub/metadata.go +++ b/flow/connectors/eventhub/metadata.go @@ -137,7 +137,9 @@ func (c *EventHubConnector) GetLastOffset(jobName string) (*protos.LastSyncState } func (c *EventHubConnector) GetLastSyncBatchID(jobName string) (int64, error) { - log.Errorf("GetLastSyncBatchID not supported for EventHub") + log.WithFields(log.Fields{ + "flowName": jobName, + }).Errorf("GetLastSyncBatchID not supported for EventHub") return 0, fmt.Errorf("GetLastSyncBatchID not supported for EventHub connector") } @@ -162,7 +164,9 @@ func (c *EventHubConnector) UpdateLastOffset(jobName string, offset int64) error `, jobName, offset) if err != nil { - log.Errorf("failed to update last offset: %v", err) + log.WithFields(log.Fields{ + "flowName": jobName, + }).Errorf("failed to update last offset: %v", err) return err } diff --git a/flow/connectors/postgres/cdc.go b/flow/connectors/postgres/cdc.go index 92b1c94755..0ea09f923d 100644 --- a/flow/connectors/postgres/cdc.go +++ b/flow/connectors/postgres/cdc.go @@ -76,7 +76,9 @@ func (p *PostgresCDCSource) PullRecords(req *model.PullRecordsRequest) (*model.R defer replicationConn.Release() pgConn := replicationConn.Conn().PgConn() - log.Infof("created replication connection") + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + }).Infof("created replication connection") sysident, err := pglogrepl.IdentifySystem(p.ctx, pgConn) if err != nil { @@ -96,7 +98,9 @@ func (p *PostgresCDCSource) PullRecords(req *model.PullRecordsRequest) (*model.R if err != nil { return nil, fmt.Errorf("error starting replication at startLsn - %d: %w", p.startLSN, err) } - log.Infof("started replication on slot %s at startLSN: %d", replicationSlot, p.startLSN) + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + }).Infof("started replication on slot %s at startLSN: %d", p.slot, p.startLSN) return p.consumeStream(pgConn, req, p.startLSN) } @@ -119,7 +123,9 @@ func (p *PostgresCDCSource) consumeStream( defer func() { err := conn.Close(p.ctx) if err != nil { - log.Errorf("unexpected error closing replication connection: %v", err) + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + }).Errorf("unexpected error closing replication connection: %v", err) } }() diff --git a/flow/connectors/postgres/client.go b/flow/connectors/postgres/client.go index 0799917565..b10938f405 100644 --- a/flow/connectors/postgres/client.go +++ b/flow/connectors/postgres/client.go @@ -562,7 +562,9 @@ func (c *PostgresConnector) getApproxTableCounts(tables []string) (int64, error) var count int64 err := row.Scan(&count) if err != nil { - log.Errorf("error while scanning row: %v", err) + log.WithFields(log.Fields{ + "table": table, + }).Errorf("error while scanning row: %v", err) return fmt.Errorf("error while scanning row: %w", err) } totalCount += count diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index 46de9889b6..00b82705b7 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -154,7 +154,7 @@ func (c *PostgresConnector) GetLastOffset(jobName string) (*protos.LastSyncState defer rows.Close() if !rows.Next() { - log.Warnf("No row found for job %s, returning nil", jobName) + log.Infof("No row found for job %s, returning nil", jobName) return nil, nil } var result int64 @@ -193,14 +193,23 @@ func (c *PostgresConnector) PullRecords(req *model.PullRecordsRequest) (*model.R } if !exists.PublicationExists { - log.Warnf("publication %s does not exist", publicationName) + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + }).Warnf("publication %s does not exist", publicationName) publicationName = "" } if !exists.SlotExists { + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + }).Warnf("slot %s does not exist", slotName) return nil, fmt.Errorf("replication slot %s does not exist", slotName) } + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + }).Infof("PullRecords: performed checks for slot and publication") + cdc, err := NewPostgresCDCSource(&PostgresCDCConfig{ AppContext: c.ctx, Connection: c.replPool, @@ -242,7 +251,9 @@ func (c *PostgresConnector) PullRecords(req *model.PullRecordsRequest) (*model.R // SyncRecords pushes records to the destination. func (c *PostgresConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.SyncResponse, error) { rawTableIdentifier := getRawTableIdentifier(req.FlowJobName) - log.Printf("pushing %d records to Postgres table %s via COPY", len(req.Records.Records), rawTableIdentifier) + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + }).Printf("pushing %d records to Postgres table %s via COPY", len(req.Records.Records), rawTableIdentifier) syncBatchID, err := c.GetLastSyncBatchID(req.FlowJobName) if err != nil { @@ -338,7 +349,9 @@ func (c *PostgresConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S defer func() { deferErr := syncRecordsTx.Rollback(c.ctx) if deferErr != pgx.ErrTxClosed && deferErr != nil { - log.Errorf("unexpected error rolling back transaction for syncing records: %v", err) + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + }).Errorf("unexpected error rolling back transaction for syncing records: %v", err) } }() @@ -356,7 +369,9 @@ func (c *PostgresConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S } metrics.LogSyncMetrics(c.ctx, req.FlowJobName, syncedRecordsCount, time.Since(startTime)) - log.Printf("synced %d records to Postgres table %s via COPY", syncedRecordsCount, rawTableIdentifier) + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + }).Printf("synced %d records to Postgres table %s via COPY", syncedRecordsCount, rawTableIdentifier) // updating metadata with new offset and syncBatchID err = c.updateSyncMetadata(req.FlowJobName, lastCP, syncBatchID, syncRecordsTx) @@ -394,7 +409,9 @@ func (c *PostgresConnector) NormalizeRecords(req *model.NormalizeRecordsRequest) } // normalize has caught up with sync or no SyncFlow has run, chill until more records are loaded. if syncBatchID == normalizeBatchID || !jobMetadataExists { - log.Printf("no records to normalize: syncBatchID %d, normalizeBatchID %d", syncBatchID, normalizeBatchID) + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + }).Printf("no records to normalize: syncBatchID %d, normalizeBatchID %d", syncBatchID, normalizeBatchID) return &model.NormalizeResponse{ Done: true, StartBatchID: normalizeBatchID, @@ -414,7 +431,9 @@ func (c *PostgresConnector) NormalizeRecords(req *model.NormalizeRecordsRequest) defer func() { deferErr := normalizeRecordsTx.Rollback(c.ctx) if deferErr != pgx.ErrTxClosed && deferErr != nil { - log.Errorf("unexpected error rolling back transaction for normalizing records: %v", err) + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + }).Errorf("unexpected error rolling back transaction for normalizing records: %v", err) } }() @@ -443,7 +462,9 @@ func (c *PostgresConnector) NormalizeRecords(req *model.NormalizeRecordsRequest) return nil, fmt.Errorf("error executing merge statements: %w", err) } } - log.Printf("normalized %d records", totalRowsAffected) + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + }).Infof("normalized %d records", totalRowsAffected) if totalRowsAffected > 0 { totalRowsAtTarget, err := c.getApproxTableCounts(maps.Keys(unchangedToastColsMap)) if err != nil { @@ -487,7 +508,9 @@ func (c *PostgresConnector) CreateRawTable(req *protos.CreateRawTableInput) (*pr defer func() { deferErr := createRawTableTx.Rollback(c.ctx) if deferErr != pgx.ErrTxClosed && deferErr != nil { - log.Errorf("unexpected error rolling back transaction for creating raw table: %v", err) + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + }).Errorf("unexpected error rolling back transaction for creating raw table: %v", err) } }() @@ -579,10 +602,13 @@ func (c *PostgresConnector) SetupNormalizedTables(req *protos.SetupNormalizedTab if err != nil { return nil, fmt.Errorf("error starting transaction for creating raw table: %w", err) } + defer func() { deferErr := createNormalizedTablesTx.Rollback(c.ctx) if deferErr != pgx.ErrTxClosed && deferErr != nil { - log.Errorf("unexpected error rolling back transaction for creating raw table: %v", err) + log.WithFields(log.Fields{ + "tableMapping": req.TableNameSchemaMapping, + }).Errorf("unexpected error rolling back transaction for creating raw table: %v", err) } }() @@ -707,7 +733,9 @@ func (c *PostgresConnector) PullFlowCleanup(jobName string) error { defer func() { deferErr := pullFlowCleanupTx.Rollback(c.ctx) if deferErr != pgx.ErrTxClosed && deferErr != nil { - log.Errorf("unexpected error rolling back transaction for flow cleanup: %v", err) + log.WithFields(log.Fields{ + "flowName": jobName, + }).Errorf("unexpected error rolling back transaction for flow cleanup: %v", err) } }() @@ -737,7 +765,9 @@ func (c *PostgresConnector) SyncFlowCleanup(jobName string) error { defer func() { deferErr := syncFlowCleanupTx.Rollback(c.ctx) if deferErr != sql.ErrTxDone && deferErr != nil { - log.Errorf("unexpected error while rolling back transaction for flow cleanup: %v", deferErr) + log.WithFields(log.Fields{ + "flowName": jobName, + }).Errorf("unexpected error while rolling back transaction for flow cleanup: %v", deferErr) } }() diff --git a/flow/connectors/postgres/qrep.go b/flow/connectors/postgres/qrep.go index 5c11f7fac7..6d44e25bc9 100644 --- a/flow/connectors/postgres/qrep.go +++ b/flow/connectors/postgres/qrep.go @@ -44,7 +44,9 @@ func (c *PostgresConnector) GetQRepPartitions( defer func() { deferErr := tx.Rollback(c.ctx) if deferErr != pgx.ErrTxClosed && deferErr != nil { - log.Errorf("unexpected error rolling back transaction for get partitions: %v", err) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Errorf("unexpected error rolling back transaction for get partitions: %v", err) } }() @@ -187,7 +189,9 @@ func (c *PostgresConnector) getNumRowsPartitions( rows, err = tx.Query(c.ctx, partitionsQuery) } if err != nil { - log.Errorf("failed to query for partitions: %v", err) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Errorf("failed to query for partitions: %v", err) return nil, fmt.Errorf("failed to query for partitions: %w", err) } @@ -246,7 +250,9 @@ func (c *PostgresConnector) getMinMaxValues( minQuery := fmt.Sprintf("SELECT MIN(%[1]s) FROM %[2]s", quotedWatermarkColumn, config.WatermarkTable) row := tx.QueryRow(c.ctx, minQuery) if err := row.Scan(&minValue); err != nil { - log.Errorf("failed to query [%s] for min value: %v", minQuery, err) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Errorf("failed to query [%s] for min value: %v", minQuery, err) return nil, nil, fmt.Errorf("failed to query for min value: %w", err) } @@ -272,8 +278,11 @@ func (c *PostgresConnector) PullQRepRecords( config *protos.QRepConfig, partition *protos.QRepPartition) (*model.QRecordBatch, error) { if partition.FullTablePartition { - log.Infof("pulling full table partition for flow job %s", config.FlowJobName) - executor := NewQRepQueryExecutorSnapshot(c.pool, c.ctx, c.config.TransactionSnapshot) + log.WithFields(log.Fields{ + "partitionId": partition.PartitionId, + }).Infof("pulling full table partition for flow job %s", config.FlowJobName) + executor := NewQRepQueryExecutorSnapshot(c.pool, c.ctx, c.config.TransactionSnapshot, + config.FlowJobName, partition.PartitionId) query := config.Query return executor.ExecuteAndProcessQuery(query) } @@ -303,16 +312,22 @@ func (c *PostgresConnector) PullQRepRecords( default: return nil, fmt.Errorf("unknown range type: %v", x) } + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + "partition": partition.PartitionId, + }).Infof("Obtained ranges for partition for PullQRep") // Build the query to pull records within the range from the source table // Be sure to order the results by the watermark column to ensure consistency across pulls - query, err := BuildQuery(config.Query) + query, err := BuildQuery(config.Query, config.FlowJobName) if err != nil { return nil, err } - executor := NewQRepQueryExecutorSnapshot(c.pool, c.ctx, c.config.TransactionSnapshot) - records, err := executor.ExecuteAndProcessQuery(query, rangeStart, rangeEnd) + executor := NewQRepQueryExecutorSnapshot(c.pool, c.ctx, c.config.TransactionSnapshot, + config.FlowJobName, partition.PartitionId) + records, err := executor.ExecuteAndProcessQuery(query, + rangeStart, rangeEnd) if err != nil { return nil, err } @@ -331,12 +346,20 @@ func (c *PostgresConnector) PullQRepRecordStream( stream *model.QRecordStream, ) (int, error) { if partition.FullTablePartition { - log.Infof("pulling full table partition for flow job %s", config.FlowJobName) - executor := NewQRepQueryExecutorSnapshot(c.pool, c.ctx, c.config.TransactionSnapshot) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + "partitionId": partition.PartitionId, + }).Infof("pulling full table partition for flow job %s", config.FlowJobName) + executor := NewQRepQueryExecutorSnapshot(c.pool, c.ctx, c.config.TransactionSnapshot, + config.FlowJobName, partition.PartitionId) query := config.Query _, err := executor.ExecuteAndProcessQueryStream(stream, query) return 0, err } + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + "partition": partition.PartitionId, + }).Infof("Obtained ranges for partition for PullQRepStream") var rangeStart interface{} var rangeEnd interface{} @@ -366,12 +389,13 @@ func (c *PostgresConnector) PullQRepRecordStream( // Build the query to pull records within the range from the source table // Be sure to order the results by the watermark column to ensure consistency across pulls - query, err := BuildQuery(config.Query) + query, err := BuildQuery(config.Query, config.FlowJobName) if err != nil { return 0, err } - executor := NewQRepQueryExecutorSnapshot(c.pool, c.ctx, c.config.TransactionSnapshot) + executor := NewQRepQueryExecutorSnapshot(c.pool, c.ctx, c.config.TransactionSnapshot, + config.FlowJobName, partition.PartitionId) numRecords, err := executor.ExecuteAndProcessQueryStream(stream, query, rangeStart, rangeEnd) if err != nil { return 0, err @@ -382,7 +406,9 @@ func (c *PostgresConnector) PullQRepRecordStream( return 0, err } metrics.LogQRepPullMetrics(c.ctx, config.FlowJobName, numRecords, totalRecordsAtSource) - log.Infof("pulled %d records for flow job %s", numRecords, config.FlowJobName) + log.WithFields(log.Fields{ + "partition": partition.PartitionId, + }).Infof("pulled %d records for flow job %s", numRecords, config.FlowJobName) return numRecords, nil } @@ -411,9 +437,15 @@ func (c *PostgresConnector) SyncQRepRecords( } if done { - log.Infof("partition %s already synced", partition.PartitionId) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Infof("partition %s already synced", partition.PartitionId) return 0, nil } + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + "partition": partition.PartitionId, + }).Infof("SyncRecords called and initial checks complete.") syncMode := config.SyncMode switch syncMode { @@ -445,11 +477,14 @@ func (c *PostgresConnector) SetupQRepMetadataTables(config *protos.QRepConfig) e if err != nil { return fmt.Errorf("failed to create table %s: %w", qRepMetadataTableName, err) } + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Infof("Setup metadata table.") return nil } -func BuildQuery(query string) (string, error) { +func BuildQuery(query string, flowJobName string) (string, error) { tmpl, err := template.New("query").Parse(query) if err != nil { return "", err @@ -468,7 +503,9 @@ func BuildQuery(query string) (string, error) { } res := buf.String() - log.Infof("templated query: %s", res) + log.WithFields(log.Fields{ + "flowName": flowJobName, + }).Infof("templated query: %s", res) return res, nil } diff --git a/flow/connectors/postgres/qrep_bench_test.go b/flow/connectors/postgres/qrep_bench_test.go index 24e425830d..a8ee28ef15 100644 --- a/flow/connectors/postgres/qrep_bench_test.go +++ b/flow/connectors/postgres/qrep_bench_test.go @@ -21,7 +21,7 @@ func BenchmarkQRepQueryExecutor(b *testing.B) { defer pool.Close() // Create a new QRepQueryExecutor instance - qe := NewQRepQueryExecutor(pool, context.Background()) + qe := NewQRepQueryExecutor(pool, context.Background(), "test flow", "test part") // Run the benchmark b.ResetTimer() diff --git a/flow/connectors/postgres/qrep_query_build_test.go b/flow/connectors/postgres/qrep_query_build_test.go index f62fb2682b..25c1fef6e6 100644 --- a/flow/connectors/postgres/qrep_query_build_test.go +++ b/flow/connectors/postgres/qrep_query_build_test.go @@ -34,7 +34,7 @@ func TestBuildQuery(t *testing.T) { for _, tc := range testCases { t.Run(tc.name, func(t *testing.T) { - actual, err := BuildQuery(tc.query) + actual, err := BuildQuery(tc.query, "test_flow") if err != nil { t.Fatalf("Error returned by BuildQuery: %v", err) } diff --git a/flow/connectors/postgres/qrep_query_executor.go b/flow/connectors/postgres/qrep_query_executor.go index 8b5cbb7d3c..8857e8eb04 100644 --- a/flow/connectors/postgres/qrep_query_executor.go +++ b/flow/connectors/postgres/qrep_query_executor.go @@ -17,25 +17,37 @@ import ( ) type QRepQueryExecutor struct { - pool *pgxpool.Pool - ctx context.Context - snapshot string - testEnv bool + pool *pgxpool.Pool + ctx context.Context + snapshot string + testEnv bool + flowJobName string + partitionID string } -func NewQRepQueryExecutor(pool *pgxpool.Pool, ctx context.Context) *QRepQueryExecutor { +func NewQRepQueryExecutor(pool *pgxpool.Pool, ctx context.Context, + flowJobName string, partitionID string) *QRepQueryExecutor { return &QRepQueryExecutor{ - pool: pool, - ctx: ctx, - snapshot: "", + pool: pool, + ctx: ctx, + snapshot: "", + flowJobName: flowJobName, + partitionID: partitionID, } } -func NewQRepQueryExecutorSnapshot(pool *pgxpool.Pool, ctx context.Context, snapshot string) *QRepQueryExecutor { +func NewQRepQueryExecutorSnapshot(pool *pgxpool.Pool, ctx context.Context, snapshot string, + flowJobName string, partitionID string) *QRepQueryExecutor { + log.WithFields(log.Fields{ + "flowName": flowJobName, + "partitionID": partitionID, + }).Info("Declared new qrep executor for snapshot") return &QRepQueryExecutor{ - pool: pool, - ctx: ctx, - snapshot: snapshot, + pool: pool, + ctx: ctx, + snapshot: snapshot, + flowJobName: flowJobName, + partitionID: partitionID, } } @@ -52,6 +64,10 @@ func (qe *QRepQueryExecutor) ExecuteQuery(query string, args ...interface{}) (pg } func (qe *QRepQueryExecutor) executeQueryInTx(tx pgx.Tx, cursorName string, fetchSize int) (pgx.Rows, error) { + log.WithFields(log.Fields{ + "flowName": qe.flowJobName, + "partitionID": qe.partitionID, + }).Info("Executing query in transaction") q := fmt.Sprintf("FETCH %d FROM %s", fetchSize, cursorName) if !qe.testEnv { @@ -96,7 +112,10 @@ func (qe *QRepQueryExecutor) ProcessRows( ) (*model.QRecordBatch, error) { // Initialize the record slice records := make([]*model.QRecord, 0) - + log.WithFields(log.Fields{ + "flowName": qe.flowJobName, + "partitionID": qe.partitionID, + }).Info("Processing rows") // Iterate over the rows for rows.Next() { record, err := mapRowToQRecord(rows, fieldDescriptions) @@ -117,7 +136,10 @@ func (qe *QRepQueryExecutor) ProcessRows( Schema: fieldDescriptionsToSchema(fieldDescriptions), } - log.Infof("[postgres] pulled %d records", batch.NumRecords) + log.WithFields(log.Fields{ + "flowName": qe.flowJobName, + "partitionID": qe.partitionID, + }).Infof("[postgres] pulled %d records", batch.NumRecords) return batch, nil } @@ -154,7 +176,10 @@ func (qe *QRepQueryExecutor) processRowsStream( } qe.recordHeartbeat("cursor %s - fetch completed - %d records", cursorName, numRows) - + log.WithFields(log.Fields{ + "flowName": qe.flowJobName, + "partitionID": qe.partitionID, + }).Infof("processed row stream") return numRows, nil } @@ -179,7 +204,9 @@ func (qe *QRepQueryExecutor) processFetchedRows( stream.Records <- &model.QRecordOrError{ Err: err, } - log.Errorf("[pg_query_executor] failed to execute query in tx: %v", err) + log.WithFields(log.Fields{ + "query": query, + }).Errorf("[pg_query_executor] failed to execute query in tx: %v", err) return 0, fmt.Errorf("[pg_query_executor] failed to execute query in tx: %w", err) } @@ -215,10 +242,12 @@ func (qe *QRepQueryExecutor) ExecuteAndProcessQuery( args ...interface{}, ) (*model.QRecordBatch, error) { stream := model.NewQRecordStream(1024) - errors := make(chan error, 1) defer close(errors) - + log.WithFields(log.Fields{ + "flowName": qe.flowJobName, + "partitionID": qe.partitionID, + }).Infof("Executing and processing query '%s'", query) go func() { _, err := qe.ExecuteAndProcessQueryStream(stream, query, args...) if err != nil { @@ -256,6 +285,10 @@ func (qe *QRepQueryExecutor) ExecuteAndProcessQueryStream( query string, args ...interface{}, ) (int, error) { + log.WithFields(log.Fields{ + "flowName": qe.flowJobName, + "partitionID": qe.partitionID, + }).Infof("Executing and processing query stream '%s'", query) defer close(stream.Records) tx, err := qe.pool.BeginTx(qe.ctx, pgx.TxOptions{ @@ -263,7 +296,10 @@ func (qe *QRepQueryExecutor) ExecuteAndProcessQueryStream( IsoLevel: pgx.RepeatableRead, }) if err != nil { - log.Errorf("[pg_query_executor] failed to begin transaction: %v", err) + log.WithFields(log.Fields{ + "flowName": qe.flowJobName, + "partitionID": qe.partitionID, + }).Errorf("[pg_query_executor] failed to begin transaction: %v", err) return 0, fmt.Errorf("[pg_query_executor] failed to begin transaction: %w", err) } @@ -280,7 +316,11 @@ func (qe *QRepQueryExecutor) ExecuteAndProcessQueryStream( stream.Records <- &model.QRecordOrError{ Err: fmt.Errorf("failed to set snapshot: %w", err), } - log.Errorf("[pg_query_executor] failed to set snapshot: %v", err) + log.WithFields(log.Fields{ + "flowName": qe.flowJobName, + "partitionID": qe.partitionID, + "query": query, + }).Errorf("[pg_query_executor] failed to set snapshot: %v", err) return 0, fmt.Errorf("[pg_query_executor] failed to set snapshot: %w", err) } } @@ -295,18 +335,23 @@ func (qe *QRepQueryExecutor) ExecuteAndProcessQueryStream( cursorName := fmt.Sprintf("peerdb_cursor_%d", randomUint) fetchSize := shared.FetchAndChannelSize - cursorQuery := fmt.Sprintf("DECLARE %s CURSOR FOR %s", cursorName, query) _, err = tx.Exec(qe.ctx, cursorQuery, args...) if err != nil { stream.Records <- &model.QRecordOrError{ Err: fmt.Errorf("failed to declare cursor: %w", err), } - log.Errorf("[pg_query_executor] failed to declare cursor: %v", err) + log.WithFields(log.Fields{ + "flowName": qe.flowJobName, + "partitionID": qe.partitionID, + }).Infof("[pg_query_executor] failed to declare cursor with query %v: %v", cursorQuery, err) return 0, fmt.Errorf("[pg_query_executor] failed to declare cursor: %w", err) } - log.Infof("[pg_query_executor] declared cursor '%s' for query '%s'", cursorName, query) + log.WithFields(log.Fields{ + "flowName": qe.flowJobName, + "partitionID": qe.partitionID, + }).Infof("[pg_query_executor] declared cursor '%s' for query '%s'", cursorName, query) totalRecordsFetched := 0 numFetchOpsComplete := 0 @@ -316,7 +361,10 @@ func (qe *QRepQueryExecutor) ExecuteAndProcessQueryStream( return 0, err } - log.Infof("[pg_query_executor] fetched %d rows for query '%s'", numRows, query) + log.WithFields(log.Fields{ + "flowName": qe.flowJobName, + "partitionID": qe.partitionID, + }).Infof("[pg_query_executor] fetched %d rows for query '%s'", numRows, query) totalRecordsFetched += numRows if numRows == 0 { @@ -335,7 +383,10 @@ func (qe *QRepQueryExecutor) ExecuteAndProcessQueryStream( return 0, fmt.Errorf("[pg_query_executor] failed to commit transaction: %w", err) } - log.Infof("[pg_query_executor] committed transaction for query '%s', rows = %d", + log.WithFields(log.Fields{ + "flowName": qe.flowJobName, + "partitionID": qe.partitionID, + }).Infof("[pg_query_executor] committed transaction for query '%s', rows = %d", query, totalRecordsFetched) return totalRecordsFetched, nil } diff --git a/flow/connectors/postgres/qrep_query_executor_test.go b/flow/connectors/postgres/qrep_query_executor_test.go index c147a01f46..8b4ff0c116 100644 --- a/flow/connectors/postgres/qrep_query_executor_test.go +++ b/flow/connectors/postgres/qrep_query_executor_test.go @@ -49,7 +49,7 @@ func TestNewQRepQueryExecutor(t *testing.T) { defer teardownDB(t, pool, schema) ctx := context.Background() - qe := NewQRepQueryExecutor(pool, ctx) + qe := NewQRepQueryExecutor(pool, ctx, "test flow", "test part") if qe == nil { t.Fatalf("expected QRepQueryExecutor, got nil") @@ -64,7 +64,7 @@ func TestExecuteAndProcessQuery(t *testing.T) { ctx := context.Background() - qe := NewQRepQueryExecutor(pool, ctx) + qe := NewQRepQueryExecutor(pool, ctx, "test flow", "test part") qe.SetTestEnv(true) query := fmt.Sprintf("CREATE TABLE IF NOT EXISTS %s.test(id SERIAL PRIMARY KEY, data TEXT);", schemaName) @@ -104,7 +104,7 @@ func TestAllDataTypes(t *testing.T) { defer teardownDB(t, pool, schemaName) ctx := context.Background() - qe := NewQRepQueryExecutor(pool, ctx) + qe := NewQRepQueryExecutor(pool, ctx, "test flow", "test part") // Create a table that contains every data type we want to test query := fmt.Sprintf(` diff --git a/flow/connectors/postgres/qrep_sync_method.go b/flow/connectors/postgres/qrep_sync_method.go index 2f7b4030a4..1990dd95c5 100644 --- a/flow/connectors/postgres/qrep_sync_method.go +++ b/flow/connectors/postgres/qrep_sync_method.go @@ -54,7 +54,11 @@ func (s *QRepStagingTableSync) SyncQRepRecords( ) _, err = pool.Exec(context.Background(), tmpTableStmt) if err != nil { - log.Errorf( + log.WithFields(log.Fields{ + "flowName": flowJobName, + "partitionID": partitionID, + "destinationTable": dstTableName, + }).Errorf( "failed to create staging temporary table %s, statement: '%s'. Error: %v", stagingTable, tmpTableStmt, @@ -65,7 +69,11 @@ func (s *QRepStagingTableSync) SyncQRepRecords( schema, err := stream.Schema() if err != nil { - log.Errorf("failed to get schema from stream: %v", err) + log.WithFields(log.Fields{ + "flowName": flowJobName, + "destinationTable": dstTableName, + "partitionID": partitionID, + }).Errorf("failed to get schema from stream: %v", err) return 0, fmt.Errorf("failed to get schema from stream: %w", err) } @@ -94,7 +102,11 @@ func (s *QRepStagingTableSync) SyncQRepRecords( defer func() { if err := tx2.Rollback(context.Background()); err != nil { if err != pgx.ErrTxClosed { - log.Errorf("failed to rollback transaction tx2: %v", err) + log.WithFields(log.Fields{ + "flowName": flowJobName, + "partitionID": partitionID, + "destinationTable": dstTableName, + }).Errorf("failed to rollback transaction tx2: %v", err) } } }() @@ -105,7 +117,10 @@ func (s *QRepStagingTableSync) SyncQRepRecords( colNames[i] = fmt.Sprintf("\"%s\"", colName) } colNamesStr := strings.Join(colNames, ", ") - + log.WithFields(log.Fields{ + "flowName": flowJobName, + "partitionID": partitionID, + }).Infof("Obtained column names and quoted them in QRep sync") insertFromStagingStmt := fmt.Sprintf( "INSERT INTO %s (%s) SELECT %s FROM %s", dstTableName.String(), @@ -116,7 +131,11 @@ func (s *QRepStagingTableSync) SyncQRepRecords( _, err = tx2.Exec(context.Background(), insertFromStagingStmt) if err != nil { - log.Errorf("failed to execute statement '%s': %v", insertFromStagingStmt, err) + log.WithFields(log.Fields{ + "flowName": flowJobName, + "partitionID": partitionID, + "destinationTable": dstTableName, + }).Errorf("failed to execute statement '%s': %v", insertFromStagingStmt, err) return -1, fmt.Errorf("failed to execute statements in a transaction: %v", err) } @@ -131,6 +150,11 @@ func (s *QRepStagingTableSync) SyncQRepRecords( "INSERT INTO %s VALUES ($1, $2, $3, $4, $5);", qRepMetadataTableName, ) + log.WithFields(log.Fields{ + "flowName": flowJobName, + "partitionID": partitionID, + "destinationTable": dstTableName, + }).Infof("Executing transaction inside Qrep sync") rows, err := tx2.Exec( context.Background(), insertMetadataStmt, @@ -156,6 +180,9 @@ func (s *QRepStagingTableSync) SyncQRepRecords( } numRowsInserted := copySource.NumRecords() - log.Printf("pushed %d records to %s", numRowsInserted, dstTableName) + log.WithFields(log.Fields{ + "flowName": flowJobName, + "partitionID": partitionID, + }).Infof("pushed %d records to %s", numRowsInserted, dstTableName) return numRowsInserted, nil } diff --git a/flow/connectors/s3/qrep.go b/flow/connectors/s3/qrep.go index 29fbb22e20..ba051e0c66 100644 --- a/flow/connectors/s3/qrep.go +++ b/flow/connectors/s3/qrep.go @@ -29,7 +29,10 @@ func (c *S3Connector) SyncQRepRecords( ) (int, error) { schema, err := stream.Schema() if err != nil { - log.Errorf("failed to get schema from stream: %v", err) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + "partitionID": partition.PartitionId, + }).Errorf("failed to get schema from stream: %v", err) return 0, fmt.Errorf("failed to get schema from stream: %w", err) } diff --git a/flow/connectors/snowflake/qrep.go b/flow/connectors/snowflake/qrep.go index 16b2559010..3f795600ca 100644 --- a/flow/connectors/snowflake/qrep.go +++ b/flow/connectors/snowflake/qrep.go @@ -42,6 +42,12 @@ func (c *SnowflakeConnector) SyncQRepRecords( if err != nil { return 0, fmt.Errorf("failed to get schema of table %s: %w", destTable, err) } + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + "partition": partition.PartitionId, + }).Infof("Called QRep sync function and "+ + "obtained table schema for destination table %s", + destTable) done, err := c.isPartitionSynced(partition.PartitionId) if err != nil { @@ -49,7 +55,9 @@ func (c *SnowflakeConnector) SyncQRepRecords( } if done { - log.Infof("Partition %s has already been synced", partition.PartitionId) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Infof("Partition %s has already been synced", partition.PartitionId) return 0, nil } @@ -187,24 +195,32 @@ func (c *SnowflakeConnector) createStage(stageName string, config *protos.QRepCo // Execute the query _, err := c.database.Exec(createStageStmt) if err != nil { - log.Errorf("failed to create stage %s: %v", stageName, err) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Errorf("failed to create stage %s: %v", stageName, err) return fmt.Errorf("failed to create stage %s: %w", stageName, err) } - log.Infof("Created stage %s", stageName) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Infof("Created stage %s", stageName) return nil } func (c *SnowflakeConnector) createExternalStage(stageName string, config *protos.QRepConfig) (string, error) { awsCreds, err := utils.GetAWSSecrets() if err != nil { - log.Errorf("failed to get AWS secrets: %v", err) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Errorf("failed to get AWS secrets: %v", err) return "", fmt.Errorf("failed to get AWS secrets: %w", err) } s3o, err := utils.NewS3BucketAndPrefix(config.StagingPath) if err != nil { - log.Errorf("failed to extract S3 bucket and prefix: %v", err) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Errorf("failed to extract S3 bucket and prefix: %v", err) return "", fmt.Errorf("failed to extract S3 bucket and prefix: %w", err) } @@ -244,13 +260,17 @@ func (c *SnowflakeConnector) ConsolidateQRepPartitions(config *protos.QRepConfig case protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO: allCols, err := c.getColsFromTable(destTable) if err != nil { - log.Errorf("failed to get columns from table %s: %v", destTable, err) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Errorf("failed to get columns from table %s: %v", destTable, err) return fmt.Errorf("failed to get columns from table %s: %w", destTable, err) } err = CopyStageToDestination(c, config, destTable, stageName, allCols) if err != nil { - log.Errorf("failed to copy stage to destination: %v", err) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Errorf("failed to copy stage to destination: %v", err) return fmt.Errorf("failed to copy stage to destination: %w", err) } @@ -262,7 +282,9 @@ func (c *SnowflakeConnector) ConsolidateQRepPartitions(config *protos.QRepConfig // CleanupQRepFlow function for snowflake connector func (c *SnowflakeConnector) CleanupQRepFlow(config *protos.QRepConfig) error { - log.Infof("Cleaning up flow job %s", config.FlowJobName) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Infof("Cleaning up flow job %s", config.FlowJobName) return c.dropStage(config.StagingPath, config.FlowJobName) } @@ -316,7 +338,9 @@ func (c *SnowflakeConnector) dropStage(stagingPath string, job string) error { if strings.HasPrefix(stagingPath, "s3://") { s3o, err := utils.NewS3BucketAndPrefix(stagingPath) if err != nil { - log.Errorf("failed to create S3 bucket and prefix: %v", err) + log.WithFields(log.Fields{ + "flowName": job, + }).Errorf("failed to create S3 bucket and prefix: %v", err) return fmt.Errorf("failed to create S3 bucket and prefix: %w", err) } @@ -325,7 +349,9 @@ func (c *SnowflakeConnector) dropStage(stagingPath string, job string) error { // deleting the contents of the bucket with prefix s3svc, err := utils.CreateS3Client() if err != nil { - log.Errorf("failed to create S3 client: %v", err) + log.WithFields(log.Fields{ + "flowName": job, + }).Errorf("failed to create S3 client: %v", err) return fmt.Errorf("failed to create S3 client: %w", err) } @@ -338,14 +364,18 @@ func (c *SnowflakeConnector) dropStage(stagingPath string, job string) error { // Iterate through the objects in the bucket with the prefix and delete them s3Client := s3manager.NewBatchDeleteWithClient(s3svc) if err := s3Client.Delete(aws.BackgroundContext(), iter); err != nil { - log.Errorf("failed to delete objects from bucket: %v", err) + log.WithFields(log.Fields{ + "flowName": job, + }).Errorf("failed to delete objects from bucket: %v", err) return fmt.Errorf("failed to delete objects from bucket: %w", err) } log.Infof("Deleted contents of bucket %s with prefix %s/%s", s3o.Bucket, s3o.Prefix, job) } - log.Infof("Dropped stage %s", stageName) + log.WithFields(log.Fields{ + "flowName": job, + }).Infof("Dropped stage %s", stageName) return nil } diff --git a/flow/connectors/snowflake/qrep_avro_sync.go b/flow/connectors/snowflake/qrep_avro_sync.go index b1e96968de..6abfe71fa9 100644 --- a/flow/connectors/snowflake/qrep_avro_sync.go +++ b/flow/connectors/snowflake/qrep_avro_sync.go @@ -35,6 +35,7 @@ func NewSnowflakeAvroSyncMethod( func (s *SnowflakeAvroSyncMethod) SyncRecords( dstTableSchema []*sql.ColumnType, stream *model.QRecordStream, + flowJobName string, ) (int, error) { dstTableName := s.config.DestinationTableIdentifier @@ -43,22 +44,34 @@ func (s *SnowflakeAvroSyncMethod) SyncRecords( return -1, fmt.Errorf("failed to get schema from stream: %w", err) } - avroSchema, err := s.getAvroSchema(dstTableName, schema) + log.WithFields(log.Fields{ + "destinationTable": dstTableName, + "flowName": flowJobName, + }).Infof("sync function called and schema acquired") + + avroSchema, err := s.getAvroSchema(dstTableName, schema, flowJobName) if err != nil { return 0, err } - numRecords, localFilePath, err := s.writeToAvroFile(stream, avroSchema, "17") + numRecords, localFilePath, err := s.writeToAvroFile(stream, avroSchema, "17", flowJobName) if err != nil { return 0, err } - log.Infof("written %d records to Avro file", numRecords) + log.WithFields(log.Fields{ + "destinationTable": dstTableName, + "flowName": flowJobName, + }).Infof("written %d records to Avro file", numRecords) stage := s.connector.getStageNameForJob(s.config.FlowJobName) err = s.connector.createStage(stage, s.config) if err != nil { return 0, err } + log.WithFields(log.Fields{ + "destinationTable": dstTableName, + "flowName": flowJobName, + }).Infof("Created stage %s", stage) allCols, err := s.connector.getColsFromTable(s.config.DestinationTableIdentifier) if err != nil { @@ -69,13 +82,17 @@ func (s *SnowflakeAvroSyncMethod) SyncRecords( if err != nil { return 0, err } - log.Infof("pushed avro file to stage") + log.WithFields(log.Fields{ + "destinationTable": dstTableName, + }).Infof("pushed avro file to stage") err = CopyStageToDestination(s.connector, s.config, s.config.DestinationTableIdentifier, stage, allCols) if err != nil { return 0, err } - log.Infof("copying records into %s from stage %s", s.config.DestinationTableIdentifier, stage) + log.WithFields(log.Fields{ + "destinationTable": dstTableName, + }).Infof("copying records into %s from stage %s", s.config.DestinationTableIdentifier, stage) return numRecords, nil } @@ -93,13 +110,17 @@ func (s *SnowflakeAvroSyncMethod) SyncQRepRecords( if err != nil { return -1, fmt.Errorf("failed to get schema from stream: %w", err) } + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + "partitionID": partition.PartitionId, + }).Infof("sync function called and schema acquired") - avroSchema, err := s.getAvroSchema(dstTableName, schema) + avroSchema, err := s.getAvroSchema(dstTableName, schema, config.FlowJobName) if err != nil { return 0, err } - numRecords, localFilePath, err := s.writeToAvroFile(stream, avroSchema, partition.PartitionId) + numRecords, localFilePath, err := s.writeToAvroFile(stream, avroSchema, partition.PartitionId, config.FlowJobName) if err != nil { return 0, err } @@ -109,7 +130,11 @@ func (s *SnowflakeAvroSyncMethod) SyncQRepRecords( log.Infof("removing temp file %s", localFilePath) err := os.Remove(localFilePath) if err != nil { - log.Errorf("failed to remove temp file %s: %v", localFilePath, err) + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + "partitionID": partition.PartitionId, + "destinationTable": dstTableName, + }).Errorf("failed to remove temp file %s: %v", localFilePath, err) } }() } @@ -121,6 +146,10 @@ func (s *SnowflakeAvroSyncMethod) SyncQRepRecords( if err != nil { return 0, err } + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + "partitionID": partition.PartitionId, + }).Infof("Put file to stage in Avro sync for snowflake") metrics.LogQRepSyncMetrics(s.connector.ctx, config.FlowJobName, int64(numRecords), time.Since(putFileStartTime)) @@ -137,13 +166,16 @@ func (s *SnowflakeAvroSyncMethod) SyncQRepRecords( func (s *SnowflakeAvroSyncMethod) getAvroSchema( dstTableName string, schema *model.QRecordSchema, + flowJobName string, ) (*model.QRecordAvroSchemaDefinition, error) { avroSchema, err := model.GetAvroSchemaDefinition(dstTableName, schema) if err != nil { return nil, fmt.Errorf("failed to define Avro schema: %w", err) } - log.Infof("Avro schema: %v\n", avroSchema) + log.WithFields(log.Fields{ + "flowName": flowJobName, + }).Infof("Avro schema: %v\n", avroSchema) return avroSchema, nil } @@ -151,6 +183,7 @@ func (s *SnowflakeAvroSyncMethod) writeToAvroFile( stream *model.QRecordStream, avroSchema *model.QRecordAvroSchemaDefinition, partitionID string, + flowJobName string, ) (int, string, error) { var numRecords int ocfWriter := avro.NewPeerDBOCFWriter(s.connector.ctx, stream, avroSchema) @@ -161,6 +194,10 @@ func (s *SnowflakeAvroSyncMethod) writeToAvroFile( } localFilePath := fmt.Sprintf("%s/%s.avro", tmpDir, partitionID) + log.WithFields(log.Fields{ + "flowName": flowJobName, + "partitionID": partitionID, + }).Infof("writing records to local file %s", localFilePath) numRecords, err = ocfWriter.WriteRecordsToAvroFile(localFilePath) if err != nil { return 0, "", fmt.Errorf("failed to write records to Avro file: %w", err) @@ -174,6 +211,10 @@ func (s *SnowflakeAvroSyncMethod) writeToAvroFile( } s3Key := fmt.Sprintf("%s/%s/%s.avro", s3o.Prefix, s.config.FlowJobName, partitionID) + log.WithFields(log.Fields{ + "flowName": flowJobName, + "partitionID": partitionID, + }).Infof("OCF: Writing records to S3") numRecords, err = ocfWriter.WriteRecordsToS3(s3o.Bucket, s3Key) if err != nil { return 0, "", fmt.Errorf("failed to write records to S3: %w", err) @@ -217,6 +258,9 @@ func CopyStageToDestination( stage string, allCols []string, ) error { + log.WithFields(log.Fields{ + "flowName": config.FlowJobName, + }).Infof("Copying stage to destination %s", dstTableName) copyOpts := []string{ "FILE_FORMAT = (TYPE = AVRO)", "MATCH_BY_COLUMN_NAME='CASE_INSENSITIVE'", @@ -260,16 +304,25 @@ func (s *SnowflakeAvroSyncMethod) insertMetadata( ) error { insertMetadataStmt, err := s.connector.createMetadataInsertStatement(partition, flowJobName, startTime) if err != nil { - log.Errorf("failed to create metadata insert statement: %v", err) + log.WithFields(log.Fields{ + "flowName": flowJobName, + "partitionID": partition.PartitionId, + }).Errorf("failed to create metadata insert statement: %v", err) return fmt.Errorf("failed to create metadata insert statement: %v", err) } if _, err := s.connector.database.Exec(insertMetadataStmt); err != nil { - log.Errorf("failed to execute metadata insert statement '%s': %v", insertMetadataStmt, err) + log.WithFields(log.Fields{ + "flowName": flowJobName, + "partitionID": partition.PartitionId, + }).Errorf("failed to execute metadata insert statement '%s': %v", insertMetadataStmt, err) return fmt.Errorf("failed to execute metadata insert statement: %v", err) } - log.Infof("inserted metadata for partition %s", partition) + log.WithFields(log.Fields{ + "flowName": flowJobName, + "partitionID": partition.PartitionId, + }).Infof("inserted metadata for partition %s", partition) return nil } @@ -393,7 +446,9 @@ func (s *SnowflakeAvroWriteHandler) HandleUpsertMode( if _, err := s.connector.database.Exec(createTempTableCmd); err != nil { return fmt.Errorf("failed to create temp table: %w", err) } - log.Infof("created temp table %s", tempTableName) + log.WithFields(log.Fields{ + "flowName": flowJobName, + }).Infof("created temp table %s", tempTableName) //nolint:gosec copyCmd := fmt.Sprintf("COPY INTO %s FROM @%s %s", @@ -423,10 +478,14 @@ func (s *SnowflakeAvroWriteHandler) HandleUpsertMode( metrics.LogQRepNormalizeMetrics(s.connector.ctx, flowJobName, rowCount, time.Since(startTime), totalRowsAtTarget) } else { - log.Errorf("failed to get rows affected: %v", err) + log.WithFields(log.Fields{ + "flowName": flowJobName, + }).Errorf("failed to get rows affected: %v", err) } - log.Infof("merged data from temp table %s into destination table %s", + log.WithFields(log.Fields{ + "flowName": flowJobName, + }).Infof("merged data from temp table %s into destination table %s", tempTableName, s.dstTableName) return nil } diff --git a/flow/connectors/snowflake/snowflake.go b/flow/connectors/snowflake/snowflake.go index 4f79463d7c..7852eedc9c 100644 --- a/flow/connectors/snowflake/snowflake.go +++ b/flow/connectors/snowflake/snowflake.go @@ -396,7 +396,10 @@ func (c *SnowflakeConnector) SyncRecords(req *model.SyncRecordsRequest) (*model. defer func() { deferErr := syncRecordsTx.Rollback() if deferErr != sql.ErrTxDone && deferErr != nil { - log.Errorf("unexpected error while rolling back transaction for SyncRecords: %v", deferErr) + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + "syncBatchID": syncBatchID - 1, + }).Errorf("unexpected error while rolling back transaction for SyncRecords: %v", deferErr) } }() @@ -726,7 +729,7 @@ func (c *SnowflakeConnector) syncRecordsViaAvro(req *model.SyncRecordsRequest, r startTime := time.Now() close(recordStream.Records) - numRecords, err := avroSyncer.SyncRecords(destinationTableSchema, recordStream) + numRecords, err := avroSyncer.SyncRecords(destinationTableSchema, recordStream, req.FlowJobName) if err != nil { return nil, err } @@ -789,7 +792,9 @@ func (c *SnowflakeConnector) NormalizeRecords(req *model.NormalizeRecordsRequest defer func() { deferErr := normalizeRecordsTx.Rollback() if deferErr != sql.ErrTxDone && deferErr != nil { - log.Errorf("unexpected error while rolling back transaction for NormalizeRecords: %v", deferErr) + log.WithFields(log.Fields{ + "flowName": req.FlowJobName, + }).Errorf("unexpected error while rolling back transaction for NormalizeRecords: %v", deferErr) } }() @@ -890,7 +895,9 @@ func (c *SnowflakeConnector) SyncFlowCleanup(jobName string) error { defer func() { deferErr := syncFlowCleanupTx.Rollback() if deferErr != sql.ErrTxDone && deferErr != nil { - log.Errorf("unexpected error while rolling back transaction for flow cleanup: %v", deferErr) + log.WithFields(log.Fields{ + "flowName": jobName, + }).Errorf("unexpected error while rolling back transaction for flow cleanup: %v", deferErr) } }() diff --git a/flow/e2e/qrep_flow_test.go b/flow/e2e/qrep_flow_test.go index 4e04d1a17c..c79157bcfd 100644 --- a/flow/e2e/qrep_flow_test.go +++ b/flow/e2e/qrep_flow_test.go @@ -246,7 +246,7 @@ func (s *E2EPeerFlowTestSuite) createQRepWorkflowConfig( func (s *E2EPeerFlowTestSuite) compareTableContentsBQ(tableName string, colsString string) { // read rows from source table - pgQueryExecutor := connpostgres.NewQRepQueryExecutor(s.pool, context.Background()) + pgQueryExecutor := connpostgres.NewQRepQueryExecutor(s.pool, context.Background(), "testflow", "testpart") pgQueryExecutor.SetTestEnv(true) pgRows, err := pgQueryExecutor.ExecuteAndProcessQuery( @@ -266,7 +266,7 @@ func (s *E2EPeerFlowTestSuite) compareTableContentsBQ(tableName string, colsStri func (s *E2EPeerFlowTestSuite) compareTableContentsSF(tableName string, selector string, caseSensitive bool) { // read rows from source table - pgQueryExecutor := connpostgres.NewQRepQueryExecutor(s.pool, context.Background()) + pgQueryExecutor := connpostgres.NewQRepQueryExecutor(s.pool, context.Background(), "testflow", "testpart") pgQueryExecutor.SetTestEnv(true) pgRows, err := pgQueryExecutor.ExecuteAndProcessQuery( fmt.Sprintf("SELECT %s FROM e2e_test.%s ORDER BY id", selector, tableName), @@ -382,7 +382,8 @@ func (s *E2EPeerFlowTestSuite) Test_Complete_QRep_Flow_Avro() { query := fmt.Sprintf("SELECT * FROM e2e_test.%s WHERE updated_at BETWEEN {{.start}} AND {{.end}}", tblName) - qrepConfig := s.createQRepWorkflowConfig("test_qrep_flow_avro", + qrepConfig := s.createQRepWorkflowConfig( + "test_qrep_flow_avro", "e2e_test."+tblName, tblName, query, From 7e523da662571cfcd2ec8cbed0fa8ac3b09727be Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Tue, 29 Aug 2023 11:34:53 -0700 Subject: [PATCH 03/12] replicate partitions in batches (#361) --- flow/activities/flowable.go | 18 +++ flow/generated/protos/flow.pb.go | 250 +++++++++++++++++++----------- flow/workflows/qrep_flow.go | 82 +++++----- nexus/pt/src/peerdb_flow.rs | 8 + nexus/pt/src/peerdb_flow.serde.rs | 115 ++++++++++++++ protos/flow.proto | 5 + 6 files changed, 352 insertions(+), 126 deletions(-) diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index 5893b15859..a2d9014e73 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -372,6 +372,24 @@ func (a *FlowableActivity) GetQRepPartitions(ctx context.Context, }, nil } +// ReplicateQRepPartition replicates a QRepPartition from the source to the destination. +func (a *FlowableActivity) ReplicateQRepPartitions(ctx context.Context, + config *protos.QRepConfig, + partitions *protos.QRepPartitionBatch, + runUUID string, +) error { + log.Infof("replicating partitions for job - %s - batch %d\n", config.FlowJobName, partitions.BatchId) + for _, p := range partitions.Partitions { + log.Infof("replicating partition - %s\n", p.PartitionId) + err := a.ReplicateQRepPartition(ctx, config, p, runUUID) + if err != nil { + return err + } + } + + return nil +} + // ReplicateQRepPartition replicates a QRepPartition from the source to the destination. func (a *FlowableActivity) ReplicateQRepPartition(ctx context.Context, config *protos.QRepConfig, diff --git a/flow/generated/protos/flow.pb.go b/flow/generated/protos/flow.pb.go index be7e7ae680..00cb9fa2f6 100644 --- a/flow/generated/protos/flow.pb.go +++ b/flow/generated/protos/flow.pb.go @@ -2289,6 +2289,61 @@ func (x *QRepPartition) GetFullTablePartition() bool { return false } +type QRepPartitionBatch struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + BatchId int32 `protobuf:"varint,1,opt,name=batch_id,json=batchId,proto3" json:"batch_id,omitempty"` + Partitions []*QRepPartition `protobuf:"bytes,2,rep,name=partitions,proto3" json:"partitions,omitempty"` +} + +func (x *QRepPartitionBatch) Reset() { + *x = QRepPartitionBatch{} + if protoimpl.UnsafeEnabled { + mi := &file_flow_proto_msgTypes[33] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *QRepPartitionBatch) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QRepPartitionBatch) ProtoMessage() {} + +func (x *QRepPartitionBatch) ProtoReflect() protoreflect.Message { + mi := &file_flow_proto_msgTypes[33] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QRepPartitionBatch.ProtoReflect.Descriptor instead. +func (*QRepPartitionBatch) Descriptor() ([]byte, []int) { + return file_flow_proto_rawDescGZIP(), []int{33} +} + +func (x *QRepPartitionBatch) GetBatchId() int32 { + if x != nil { + return x.BatchId + } + return 0 +} + +func (x *QRepPartitionBatch) GetPartitions() []*QRepPartition { + if x != nil { + return x.Partitions + } + return nil +} + type QRepParitionResult struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -2300,7 +2355,7 @@ type QRepParitionResult struct { func (x *QRepParitionResult) Reset() { *x = QRepParitionResult{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[33] + mi := &file_flow_proto_msgTypes[34] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2313,7 +2368,7 @@ func (x *QRepParitionResult) String() string { func (*QRepParitionResult) ProtoMessage() {} func (x *QRepParitionResult) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[33] + mi := &file_flow_proto_msgTypes[34] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2326,7 +2381,7 @@ func (x *QRepParitionResult) ProtoReflect() protoreflect.Message { // Deprecated: Use QRepParitionResult.ProtoReflect.Descriptor instead. func (*QRepParitionResult) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{33} + return file_flow_proto_rawDescGZIP(), []int{34} } func (x *QRepParitionResult) GetPartitions() []*QRepPartition { @@ -2347,7 +2402,7 @@ type DropFlowInput struct { func (x *DropFlowInput) Reset() { *x = DropFlowInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[34] + mi := &file_flow_proto_msgTypes[35] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2360,7 +2415,7 @@ func (x *DropFlowInput) String() string { func (*DropFlowInput) ProtoMessage() {} func (x *DropFlowInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[34] + mi := &file_flow_proto_msgTypes[35] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2373,7 +2428,7 @@ func (x *DropFlowInput) ProtoReflect() protoreflect.Message { // Deprecated: Use DropFlowInput.ProtoReflect.Descriptor instead. func (*DropFlowInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{34} + return file_flow_proto_rawDescGZIP(), []int{35} } func (x *DropFlowInput) GetFlowName() string { @@ -2863,33 +2918,40 @@ var file_flow_proto_rawDesc = []byte{ 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x30, 0x0a, 0x14, 0x66, 0x75, 0x6c, 0x6c, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x66, 0x75, 0x6c, 0x6c, - 0x54, 0x61, 0x62, 0x6c, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x50, - 0x0a, 0x12, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, - 0x73, 0x75, 0x6c, 0x74, 0x12, 0x3a, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, - 0x22, 0x2c, 0x0a, 0x0d, 0x44, 0x72, 0x6f, 0x70, 0x46, 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x70, 0x75, - 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x66, 0x6c, 0x6f, 0x77, 0x4e, 0x61, 0x6d, 0x65, 0x2a, 0x50, - 0x0a, 0x0c, 0x51, 0x52, 0x65, 0x70, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x1f, - 0x0a, 0x1b, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x53, 0x59, 0x4e, 0x43, 0x5f, 0x4d, 0x4f, 0x44, 0x45, - 0x5f, 0x4d, 0x55, 0x4c, 0x54, 0x49, 0x5f, 0x49, 0x4e, 0x53, 0x45, 0x52, 0x54, 0x10, 0x00, 0x12, - 0x1f, 0x0a, 0x1b, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x53, 0x59, 0x4e, 0x43, 0x5f, 0x4d, 0x4f, 0x44, - 0x45, 0x5f, 0x53, 0x54, 0x4f, 0x52, 0x41, 0x47, 0x45, 0x5f, 0x41, 0x56, 0x52, 0x4f, 0x10, 0x01, - 0x2a, 0x47, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, - 0x65, 0x12, 0x1a, 0x0a, 0x16, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x5f, - 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x41, 0x50, 0x50, 0x45, 0x4e, 0x44, 0x10, 0x00, 0x12, 0x1a, 0x0a, - 0x16, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x5f, 0x4d, 0x4f, 0x44, 0x45, - 0x5f, 0x55, 0x50, 0x53, 0x45, 0x52, 0x54, 0x10, 0x01, 0x42, 0x76, 0x0a, 0x0f, 0x63, 0x6f, 0x6d, - 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x42, 0x09, 0x46, 0x6c, - 0x6f, 0x77, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x10, 0x67, 0x65, 0x6e, 0x65, 0x72, - 0x61, 0x74, 0x65, 0x64, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x73, 0xa2, 0x02, 0x03, 0x50, 0x58, - 0x58, 0xaa, 0x02, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0xca, 0x02, - 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0xe2, 0x02, 0x16, 0x50, 0x65, - 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, - 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, - 0x77, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x6b, + 0x0a, 0x12, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x42, + 0x61, 0x74, 0x63, 0x68, 0x12, 0x19, 0x0a, 0x08, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x62, 0x61, 0x74, 0x63, 0x68, 0x49, 0x64, 0x12, + 0x3a, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, + 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x50, 0x0a, 0x12, 0x51, + 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c, + 0x74, 0x12, 0x3a, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, + 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, + 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x2c, 0x0a, + 0x0d, 0x44, 0x72, 0x6f, 0x70, 0x46, 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x1b, + 0x0a, 0x09, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x08, 0x66, 0x6c, 0x6f, 0x77, 0x4e, 0x61, 0x6d, 0x65, 0x2a, 0x50, 0x0a, 0x0c, 0x51, + 0x52, 0x65, 0x70, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x1f, 0x0a, 0x1b, 0x51, + 0x52, 0x45, 0x50, 0x5f, 0x53, 0x59, 0x4e, 0x43, 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x4d, 0x55, + 0x4c, 0x54, 0x49, 0x5f, 0x49, 0x4e, 0x53, 0x45, 0x52, 0x54, 0x10, 0x00, 0x12, 0x1f, 0x0a, 0x1b, + 0x51, 0x52, 0x45, 0x50, 0x5f, 0x53, 0x59, 0x4e, 0x43, 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x53, + 0x54, 0x4f, 0x52, 0x41, 0x47, 0x45, 0x5f, 0x41, 0x56, 0x52, 0x4f, 0x10, 0x01, 0x2a, 0x47, 0x0a, + 0x0d, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1a, + 0x0a, 0x16, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x5f, 0x4d, 0x4f, 0x44, + 0x45, 0x5f, 0x41, 0x50, 0x50, 0x45, 0x4e, 0x44, 0x10, 0x00, 0x12, 0x1a, 0x0a, 0x16, 0x51, 0x52, + 0x45, 0x50, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x55, 0x50, + 0x53, 0x45, 0x52, 0x54, 0x10, 0x01, 0x42, 0x76, 0x0a, 0x0f, 0x63, 0x6f, 0x6d, 0x2e, 0x70, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x42, 0x09, 0x46, 0x6c, 0x6f, 0x77, 0x50, + 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x10, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, + 0x64, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x73, 0xa2, 0x02, 0x03, 0x50, 0x58, 0x58, 0xaa, 0x02, + 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0xca, 0x02, 0x0a, 0x50, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0xe2, 0x02, 0x16, 0x50, 0x65, 0x65, 0x72, 0x64, + 0x62, 0x46, 0x6c, 0x6f, 0x77, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, + 0x61, 0xea, 0x02, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0x62, 0x06, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -2905,7 +2967,7 @@ func file_flow_proto_rawDescGZIP() []byte { } var file_flow_proto_enumTypes = make([]protoimpl.EnumInfo, 2) -var file_flow_proto_msgTypes = make([]protoimpl.MessageInfo, 45) +var file_flow_proto_msgTypes = make([]protoimpl.MessageInfo, 46) var file_flow_proto_goTypes = []interface{}{ (QRepSyncMode)(0), // 0: peerdb_flow.QRepSyncMode (QRepWriteType)(0), // 1: peerdb_flow.QRepWriteType @@ -2942,79 +3004,81 @@ var file_flow_proto_goTypes = []interface{}{ (*QRepWriteMode)(nil), // 32: peerdb_flow.QRepWriteMode (*QRepConfig)(nil), // 33: peerdb_flow.QRepConfig (*QRepPartition)(nil), // 34: peerdb_flow.QRepPartition - (*QRepParitionResult)(nil), // 35: peerdb_flow.QRepParitionResult - (*DropFlowInput)(nil), // 36: peerdb_flow.DropFlowInput - nil, // 37: peerdb_flow.FlowConnectionConfigs.TableNameMappingEntry - nil, // 38: peerdb_flow.FlowConnectionConfigs.SrcTableIdNameMappingEntry - nil, // 39: peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry - nil, // 40: peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry - nil, // 41: peerdb_flow.SetupReplicationInput.TableNameMappingEntry - nil, // 42: peerdb_flow.CreateRawTableInput.TableNameMappingEntry - nil, // 43: peerdb_flow.TableSchema.ColumnsEntry - nil, // 44: peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry - nil, // 45: peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry - nil, // 46: peerdb_flow.SetupNormalizedTableBatchOutput.TableExistsMappingEntry - (*Peer)(nil), // 47: peerdb_peers.Peer - (*timestamppb.Timestamp)(nil), // 48: google.protobuf.Timestamp + (*QRepPartitionBatch)(nil), // 35: peerdb_flow.QRepPartitionBatch + (*QRepParitionResult)(nil), // 36: peerdb_flow.QRepParitionResult + (*DropFlowInput)(nil), // 37: peerdb_flow.DropFlowInput + nil, // 38: peerdb_flow.FlowConnectionConfigs.TableNameMappingEntry + nil, // 39: peerdb_flow.FlowConnectionConfigs.SrcTableIdNameMappingEntry + nil, // 40: peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry + nil, // 41: peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry + nil, // 42: peerdb_flow.SetupReplicationInput.TableNameMappingEntry + nil, // 43: peerdb_flow.CreateRawTableInput.TableNameMappingEntry + nil, // 44: peerdb_flow.TableSchema.ColumnsEntry + nil, // 45: peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry + nil, // 46: peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry + nil, // 47: peerdb_flow.SetupNormalizedTableBatchOutput.TableExistsMappingEntry + (*Peer)(nil), // 48: peerdb_peers.Peer + (*timestamppb.Timestamp)(nil), // 49: google.protobuf.Timestamp } var file_flow_proto_depIdxs = []int32{ - 47, // 0: peerdb_flow.FlowConnectionConfigs.source:type_name -> peerdb_peers.Peer - 47, // 1: peerdb_flow.FlowConnectionConfigs.destination:type_name -> peerdb_peers.Peer + 48, // 0: peerdb_flow.FlowConnectionConfigs.source:type_name -> peerdb_peers.Peer + 48, // 1: peerdb_flow.FlowConnectionConfigs.destination:type_name -> peerdb_peers.Peer 20, // 2: peerdb_flow.FlowConnectionConfigs.table_schema:type_name -> peerdb_flow.TableSchema - 37, // 3: peerdb_flow.FlowConnectionConfigs.table_name_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.TableNameMappingEntry - 38, // 4: peerdb_flow.FlowConnectionConfigs.src_table_id_name_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.SrcTableIdNameMappingEntry - 39, // 5: peerdb_flow.FlowConnectionConfigs.table_name_schema_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry - 47, // 6: peerdb_flow.FlowConnectionConfigs.metadata_peer:type_name -> peerdb_peers.Peer + 38, // 3: peerdb_flow.FlowConnectionConfigs.table_name_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.TableNameMappingEntry + 39, // 4: peerdb_flow.FlowConnectionConfigs.src_table_id_name_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.SrcTableIdNameMappingEntry + 40, // 5: peerdb_flow.FlowConnectionConfigs.table_name_schema_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry + 48, // 6: peerdb_flow.FlowConnectionConfigs.metadata_peer:type_name -> peerdb_peers.Peer 0, // 7: peerdb_flow.FlowConnectionConfigs.snapshot_sync_mode:type_name -> peerdb_flow.QRepSyncMode 0, // 8: peerdb_flow.FlowConnectionConfigs.cdc_sync_mode:type_name -> peerdb_flow.QRepSyncMode - 48, // 9: peerdb_flow.LastSyncState.last_synced_at:type_name -> google.protobuf.Timestamp + 49, // 9: peerdb_flow.LastSyncState.last_synced_at:type_name -> google.protobuf.Timestamp 6, // 10: peerdb_flow.StartFlowInput.last_sync_state:type_name -> peerdb_flow.LastSyncState 3, // 11: peerdb_flow.StartFlowInput.flow_connection_configs:type_name -> peerdb_flow.FlowConnectionConfigs 4, // 12: peerdb_flow.StartFlowInput.sync_flow_options:type_name -> peerdb_flow.SyncFlowOptions 3, // 13: peerdb_flow.StartNormalizeInput.flow_connection_configs:type_name -> peerdb_flow.FlowConnectionConfigs - 47, // 14: peerdb_flow.GetLastSyncedIDInput.peer_connection_config:type_name -> peerdb_peers.Peer - 47, // 15: peerdb_flow.EnsurePullabilityInput.peer_connection_config:type_name -> peerdb_peers.Peer - 47, // 16: peerdb_flow.EnsurePullabilityBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer + 48, // 14: peerdb_flow.GetLastSyncedIDInput.peer_connection_config:type_name -> peerdb_peers.Peer + 48, // 15: peerdb_flow.EnsurePullabilityInput.peer_connection_config:type_name -> peerdb_peers.Peer + 48, // 16: peerdb_flow.EnsurePullabilityBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer 12, // 17: peerdb_flow.TableIdentifier.postgres_table_identifier:type_name -> peerdb_flow.PostgresTableIdentifier 13, // 18: peerdb_flow.EnsurePullabilityOutput.table_identifier:type_name -> peerdb_flow.TableIdentifier - 40, // 19: peerdb_flow.EnsurePullabilityBatchOutput.table_identifier_mapping:type_name -> peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry - 47, // 20: peerdb_flow.SetupReplicationInput.peer_connection_config:type_name -> peerdb_peers.Peer - 41, // 21: peerdb_flow.SetupReplicationInput.table_name_mapping:type_name -> peerdb_flow.SetupReplicationInput.TableNameMappingEntry - 47, // 22: peerdb_flow.SetupReplicationInput.destination_peer:type_name -> peerdb_peers.Peer - 47, // 23: peerdb_flow.CreateRawTableInput.peer_connection_config:type_name -> peerdb_peers.Peer - 42, // 24: peerdb_flow.CreateRawTableInput.table_name_mapping:type_name -> peerdb_flow.CreateRawTableInput.TableNameMappingEntry + 41, // 19: peerdb_flow.EnsurePullabilityBatchOutput.table_identifier_mapping:type_name -> peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry + 48, // 20: peerdb_flow.SetupReplicationInput.peer_connection_config:type_name -> peerdb_peers.Peer + 42, // 21: peerdb_flow.SetupReplicationInput.table_name_mapping:type_name -> peerdb_flow.SetupReplicationInput.TableNameMappingEntry + 48, // 22: peerdb_flow.SetupReplicationInput.destination_peer:type_name -> peerdb_peers.Peer + 48, // 23: peerdb_flow.CreateRawTableInput.peer_connection_config:type_name -> peerdb_peers.Peer + 43, // 24: peerdb_flow.CreateRawTableInput.table_name_mapping:type_name -> peerdb_flow.CreateRawTableInput.TableNameMappingEntry 0, // 25: peerdb_flow.CreateRawTableInput.cdc_sync_mode:type_name -> peerdb_flow.QRepSyncMode - 43, // 26: peerdb_flow.TableSchema.columns:type_name -> peerdb_flow.TableSchema.ColumnsEntry - 47, // 27: peerdb_flow.GetTableSchemaBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer - 44, // 28: peerdb_flow.GetTableSchemaBatchOutput.table_name_schema_mapping:type_name -> peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry - 47, // 29: peerdb_flow.SetupNormalizedTableInput.peer_connection_config:type_name -> peerdb_peers.Peer + 44, // 26: peerdb_flow.TableSchema.columns:type_name -> peerdb_flow.TableSchema.ColumnsEntry + 48, // 27: peerdb_flow.GetTableSchemaBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer + 45, // 28: peerdb_flow.GetTableSchemaBatchOutput.table_name_schema_mapping:type_name -> peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry + 48, // 29: peerdb_flow.SetupNormalizedTableInput.peer_connection_config:type_name -> peerdb_peers.Peer 20, // 30: peerdb_flow.SetupNormalizedTableInput.source_table_schema:type_name -> peerdb_flow.TableSchema - 47, // 31: peerdb_flow.SetupNormalizedTableBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer - 45, // 32: peerdb_flow.SetupNormalizedTableBatchInput.table_name_schema_mapping:type_name -> peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry - 46, // 33: peerdb_flow.SetupNormalizedTableBatchOutput.table_exists_mapping:type_name -> peerdb_flow.SetupNormalizedTableBatchOutput.TableExistsMappingEntry - 48, // 34: peerdb_flow.TimestampPartitionRange.start:type_name -> google.protobuf.Timestamp - 48, // 35: peerdb_flow.TimestampPartitionRange.end:type_name -> google.protobuf.Timestamp + 48, // 31: peerdb_flow.SetupNormalizedTableBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer + 46, // 32: peerdb_flow.SetupNormalizedTableBatchInput.table_name_schema_mapping:type_name -> peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry + 47, // 33: peerdb_flow.SetupNormalizedTableBatchOutput.table_exists_mapping:type_name -> peerdb_flow.SetupNormalizedTableBatchOutput.TableExistsMappingEntry + 49, // 34: peerdb_flow.TimestampPartitionRange.start:type_name -> google.protobuf.Timestamp + 49, // 35: peerdb_flow.TimestampPartitionRange.end:type_name -> google.protobuf.Timestamp 29, // 36: peerdb_flow.TIDPartitionRange.start:type_name -> peerdb_flow.TID 29, // 37: peerdb_flow.TIDPartitionRange.end:type_name -> peerdb_flow.TID 27, // 38: peerdb_flow.PartitionRange.int_range:type_name -> peerdb_flow.IntPartitionRange 28, // 39: peerdb_flow.PartitionRange.timestamp_range:type_name -> peerdb_flow.TimestampPartitionRange 30, // 40: peerdb_flow.PartitionRange.tid_range:type_name -> peerdb_flow.TIDPartitionRange 1, // 41: peerdb_flow.QRepWriteMode.write_type:type_name -> peerdb_flow.QRepWriteType - 47, // 42: peerdb_flow.QRepConfig.source_peer:type_name -> peerdb_peers.Peer - 47, // 43: peerdb_flow.QRepConfig.destination_peer:type_name -> peerdb_peers.Peer + 48, // 42: peerdb_flow.QRepConfig.source_peer:type_name -> peerdb_peers.Peer + 48, // 43: peerdb_flow.QRepConfig.destination_peer:type_name -> peerdb_peers.Peer 0, // 44: peerdb_flow.QRepConfig.sync_mode:type_name -> peerdb_flow.QRepSyncMode 32, // 45: peerdb_flow.QRepConfig.write_mode:type_name -> peerdb_flow.QRepWriteMode 31, // 46: peerdb_flow.QRepPartition.range:type_name -> peerdb_flow.PartitionRange - 34, // 47: peerdb_flow.QRepParitionResult.partitions:type_name -> peerdb_flow.QRepPartition - 20, // 48: peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry.value:type_name -> peerdb_flow.TableSchema - 13, // 49: peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry.value:type_name -> peerdb_flow.TableIdentifier - 20, // 50: peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry.value:type_name -> peerdb_flow.TableSchema - 20, // 51: peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry.value:type_name -> peerdb_flow.TableSchema - 52, // [52:52] is the sub-list for method output_type - 52, // [52:52] is the sub-list for method input_type - 52, // [52:52] is the sub-list for extension type_name - 52, // [52:52] is the sub-list for extension extendee - 0, // [0:52] is the sub-list for field type_name + 34, // 47: peerdb_flow.QRepPartitionBatch.partitions:type_name -> peerdb_flow.QRepPartition + 34, // 48: peerdb_flow.QRepParitionResult.partitions:type_name -> peerdb_flow.QRepPartition + 20, // 49: peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry.value:type_name -> peerdb_flow.TableSchema + 13, // 50: peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry.value:type_name -> peerdb_flow.TableIdentifier + 20, // 51: peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry.value:type_name -> peerdb_flow.TableSchema + 20, // 52: peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry.value:type_name -> peerdb_flow.TableSchema + 53, // [53:53] is the sub-list for method output_type + 53, // [53:53] is the sub-list for method input_type + 53, // [53:53] is the sub-list for extension type_name + 53, // [53:53] is the sub-list for extension extendee + 0, // [0:53] is the sub-list for field type_name } func init() { file_flow_proto_init() } @@ -3421,7 +3485,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*QRepParitionResult); i { + switch v := v.(*QRepPartitionBatch); i { case 0: return &v.state case 1: @@ -3433,6 +3497,18 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*QRepParitionResult); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_flow_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*DropFlowInput); i { case 0: return &v.state @@ -3459,7 +3535,7 @@ func file_flow_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_flow_proto_rawDesc, NumEnums: 2, - NumMessages: 45, + NumMessages: 46, NumExtensions: 0, NumServices: 0, }, diff --git a/flow/workflows/qrep_flow.go b/flow/workflows/qrep_flow.go index 9025fbe3ea..5a3b704e50 100644 --- a/flow/workflows/qrep_flow.go +++ b/flow/workflows/qrep_flow.go @@ -3,10 +3,8 @@ package peerflow import ( "fmt" - "math/rand" "time" - "github.com/PeerDB-io/peer-flow/concurrency" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/google/uuid" "go.temporal.io/api/enums/v1" @@ -70,10 +68,8 @@ func (q *QRepFlowExecution) GetPartitions( return partitions, nil } -// ReplicateParititon replicates the given partition. -func (q *QRepFlowExecution) ReplicatePartition(ctx workflow.Context, partition *protos.QRepPartition) error { - q.logger.Info("replicating partition - ", partition.PartitionId) - +// ReplicatePartitions replicates the partition batch. +func (q *QRepFlowExecution) ReplicatePartitions(ctx workflow.Context, partitions *protos.QRepPartitionBatch) error { ctx = workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ StartToCloseTimeout: 24 * 5 * time.Hour, RetryPolicy: &temporal.RetryPolicy{ @@ -82,12 +78,12 @@ func (q *QRepFlowExecution) ReplicatePartition(ctx workflow.Context, partition * HeartbeatTimeout: 1 * time.Hour, }) + q.logger.Info("replicating partition", "partition", partitions.BatchId) if err := workflow.ExecuteActivity(ctx, - flowable.ReplicateQRepPartition, q.config, partition, q.runUUID).Get(ctx, nil); err != nil { + flowable.ReplicateQRepPartitions, q.config, partitions, q.runUUID).Get(ctx, nil); err != nil { return fmt.Errorf("failed to replicate partition: %w", err) } - q.logger.Info("replicated partition - ", partition.PartitionId) return nil } @@ -107,12 +103,11 @@ func (q *QRepFlowExecution) getPartitionWorkflowID(ctx workflow.Context) (string // startChildWorkflow starts a single child workflow. func (q *QRepFlowExecution) startChildWorkflow( - boundSelector *concurrency.BoundSelector, ctx workflow.Context, - partition *protos.QRepPartition) error { + partitions *protos.QRepPartitionBatch) (workflow.Future, error) { wid, err := q.getPartitionWorkflowID(ctx) if err != nil { - return fmt.Errorf("failed to get child workflow ID: %w", err) + return nil, fmt.Errorf("failed to get child workflow ID: %w", err) } partFlowCtx := workflow.WithChildOptions(ctx, workflow.ChildWorkflowOptions{ WorkflowID: wid, @@ -122,15 +117,10 @@ func (q *QRepFlowExecution) startChildWorkflow( }, }) - boundSelector.SpawnChild( - partFlowCtx, - QRepPartitionWorkflow, - q.config, - partition, - q.runUUID, - ) + future := workflow.ExecuteChildWorkflow( + partFlowCtx, QRepPartitionWorkflow, q.config, partitions, q.runUUID) - return nil + return future, nil } // processPartitions handles the logic for processing the partitions. @@ -139,18 +129,42 @@ func (q *QRepFlowExecution) processPartitions( maxParallelWorkers int, partitions []*protos.QRepPartition, ) error { - boundSelector := concurrency.NewBoundSelector(maxParallelWorkers, len(partitions), ctx) + chunkSize := len(partitions) / maxParallelWorkers + if chunkSize == 0 { + chunkSize = 1 + } + + batches := make([][]*protos.QRepPartition, 0) + for i := 0; i < len(partitions); i += chunkSize { + end := i + chunkSize + if end > len(partitions) { + end = len(partitions) + } + + batches = append(batches, partitions[i:end]) + } + + q.logger.Info("processing partitions in batches", "num batches", len(batches)) - for _, partition := range partitions { - err := q.startChildWorkflow(boundSelector, ctx, partition) + futures := make([]workflow.Future, 0) + for i, parts := range batches { + batch := &protos.QRepPartitionBatch{ + Partitions: parts, + BatchId: int32(i), + } + future, err := q.startChildWorkflow(ctx, batch) if err != nil { - return err + return fmt.Errorf("failed to start child workflow: %w", err) } + + futures = append(futures, future) } - err := boundSelector.Wait() - if err != nil { - return fmt.Errorf("failed to process partitions: %w", err) + // wait for all the child workflows to complete + for _, future := range futures { + if err := future.Get(ctx, nil); err != nil { + return fmt.Errorf("failed to wait for child workflow: %w", err) + } } q.logger.Info("all partitions in batch processed") @@ -261,16 +275,6 @@ func QRepFlowWorkflow( return fmt.Errorf("failed to get partitions: %w", err) } - workflow.SideEffect(ctx, func(ctx workflow.Context) interface{} { - numPartitions := len(partitions.Partitions) - if numPartitions > 0 { - rand.Shuffle(len(partitions.Partitions), func(i, j int) { - partitions.Partitions[i], partitions.Partitions[j] = partitions.Partitions[j], partitions.Partitions[i] - }) - } - return nil - }) - logger.Info("partitions to replicate - ", len(partitions.Partitions)) if err = q.processPartitions(ctx, maxParallelWorkers, partitions.Partitions); err != nil { return err @@ -315,13 +319,13 @@ func QRepFlowWorkflow( return workflow.NewContinueAsNewError(ctx, QRepFlowWorkflow, config, lastPartition, numPartitionsProcessed) } -// QRepPartitionWorkflow replicate a single partition. +// QRepPartitionWorkflow replicate a partition batch func QRepPartitionWorkflow( ctx workflow.Context, config *protos.QRepConfig, - partition *protos.QRepPartition, + partitions *protos.QRepPartitionBatch, runUUID string, ) error { q := NewQRepFlowExecution(ctx, config, runUUID) - return q.ReplicatePartition(ctx, partition) + return q.ReplicatePartitions(ctx, partitions) } diff --git a/nexus/pt/src/peerdb_flow.rs b/nexus/pt/src/peerdb_flow.rs index 3da275341f..6e996b372f 100644 --- a/nexus/pt/src/peerdb_flow.rs +++ b/nexus/pt/src/peerdb_flow.rs @@ -375,6 +375,14 @@ pub struct QRepPartition { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct QRepPartitionBatch { + #[prost(int32, tag="1")] + pub batch_id: i32, + #[prost(message, repeated, tag="2")] + pub partitions: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct QRepParitionResult { #[prost(message, repeated, tag="1")] pub partitions: ::prost::alloc::vec::Vec, diff --git a/nexus/pt/src/peerdb_flow.serde.rs b/nexus/pt/src/peerdb_flow.serde.rs index 0ec540285a..2d2dc32e9f 100644 --- a/nexus/pt/src/peerdb_flow.serde.rs +++ b/nexus/pt/src/peerdb_flow.serde.rs @@ -2743,6 +2743,121 @@ impl<'de> serde::Deserialize<'de> for QRepPartition { deserializer.deserialize_struct("peerdb_flow.QRepPartition", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for QRepPartitionBatch { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.batch_id != 0 { + len += 1; + } + if !self.partitions.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_flow.QRepPartitionBatch", len)?; + if self.batch_id != 0 { + struct_ser.serialize_field("batchId", &self.batch_id)?; + } + if !self.partitions.is_empty() { + struct_ser.serialize_field("partitions", &self.partitions)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for QRepPartitionBatch { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "batch_id", + "batchId", + "partitions", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + BatchId, + Partitions, + __SkipField__, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "batchId" | "batch_id" => Ok(GeneratedField::BatchId), + "partitions" => Ok(GeneratedField::Partitions), + _ => Ok(GeneratedField::__SkipField__), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = QRepPartitionBatch; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct peerdb_flow.QRepPartitionBatch") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut batch_id__ = None; + let mut partitions__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::BatchId => { + if batch_id__.is_some() { + return Err(serde::de::Error::duplicate_field("batchId")); + } + batch_id__ = + Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::Partitions => { + if partitions__.is_some() { + return Err(serde::de::Error::duplicate_field("partitions")); + } + partitions__ = Some(map.next_value()?); + } + GeneratedField::__SkipField__ => { + let _ = map.next_value::()?; + } + } + } + Ok(QRepPartitionBatch { + batch_id: batch_id__.unwrap_or_default(), + partitions: partitions__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("peerdb_flow.QRepPartitionBatch", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for QRepSyncMode { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result diff --git a/protos/flow.proto b/protos/flow.proto index d76be176a7..a279a91914 100644 --- a/protos/flow.proto +++ b/protos/flow.proto @@ -252,6 +252,11 @@ message QRepPartition { bool full_table_partition = 4; } +message QRepPartitionBatch { + int32 batch_id = 1; + repeated QRepPartition partitions = 2; +} + message QRepParitionResult { repeated QRepPartition partitions = 1; } From 1592cee0562c195e2f09122f4b5f3ab1f1204152 Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Tue, 29 Aug 2023 12:19:58 -0700 Subject: [PATCH 04/12] better logging for partition batches (#362) --- flow/activities/flowable.go | 16 ++++++++++------ flow/workflows/qrep_flow.go | 5 +++-- 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index a2d9014e73..49e2f6a27a 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -378,10 +378,12 @@ func (a *FlowableActivity) ReplicateQRepPartitions(ctx context.Context, partitions *protos.QRepPartitionBatch, runUUID string, ) error { - log.Infof("replicating partitions for job - %s - batch %d\n", config.FlowJobName, partitions.BatchId) - for _, p := range partitions.Partitions { - log.Infof("replicating partition - %s\n", p.PartitionId) - err := a.ReplicateQRepPartition(ctx, config, p, runUUID) + numPartitions := len(partitions.Partitions) + log.Infof("replicating partitions for job - %s - batch %d - size: %d\n", + config.FlowJobName, partitions.BatchId, numPartitions) + for i, p := range partitions.Partitions { + log.Infof("batch-%d - replicating partition - %s\n", partitions.BatchId, p.PartitionId) + err := a.replicateQRepPartition(ctx, config, i+1, numPartitions, p, runUUID) if err != nil { return err } @@ -391,8 +393,10 @@ func (a *FlowableActivity) ReplicateQRepPartitions(ctx context.Context, } // ReplicateQRepPartition replicates a QRepPartition from the source to the destination. -func (a *FlowableActivity) ReplicateQRepPartition(ctx context.Context, +func (a *FlowableActivity) replicateQRepPartition(ctx context.Context, config *protos.QRepConfig, + idx int, + total int, partition *protos.QRepPartition, runUUID string, ) error { @@ -466,7 +470,7 @@ func (a *FlowableActivity) ReplicateQRepPartition(ctx context.Context, } shutdown := utils.HeartbeatRoutine(ctx, 5*time.Minute, func() string { - return fmt.Sprintf("syncing partition - %s", partition.PartitionId) + return fmt.Sprintf("syncing partition - %s: %d of %d total.", partition.PartitionId, idx, total) }) defer func() { diff --git a/flow/workflows/qrep_flow.go b/flow/workflows/qrep_flow.go index 5a3b704e50..10d3e089d0 100644 --- a/flow/workflows/qrep_flow.go +++ b/flow/workflows/qrep_flow.go @@ -78,7 +78,8 @@ func (q *QRepFlowExecution) ReplicatePartitions(ctx workflow.Context, partitions HeartbeatTimeout: 1 * time.Hour, }) - q.logger.Info("replicating partition", "partition", partitions.BatchId) + msg := fmt.Sprintf("replicating partition batch - %d", partitions.BatchId) + q.logger.Info(msg) if err := workflow.ExecuteActivity(ctx, flowable.ReplicateQRepPartitions, q.config, partitions, q.runUUID).Get(ctx, nil); err != nil { return fmt.Errorf("failed to replicate partition: %w", err) @@ -150,7 +151,7 @@ func (q *QRepFlowExecution) processPartitions( for i, parts := range batches { batch := &protos.QRepPartitionBatch{ Partitions: parts, - BatchId: int32(i), + BatchId: int32(i + 1), } future, err := q.startChildWorkflow(ctx, batch) if err != nil { From 16702bb2ce488dbc033ca4de4db0d23391586d5f Mon Sep 17 00:00:00 2001 From: Arunprasad Rajkumar Date: Wed, 30 Aug 2023 20:26:42 +0530 Subject: [PATCH 05/12] Fix Linux docker-compose error and add TEMPORAL_CSRF_COOKIE_INSECURE=true (#363) This addition helps cancel workflows from the temporal UI, avoiding the "Missing csrf token in request header" error. Reference: [1] [1] https://community.temporal.io/t/missing-csrf-token-in-request-header/7410 Signed-off-by: Arunprasad Rajkumar --- docker-compose.yml | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/docker-compose.yml b/docker-compose.yml index d902a71b4f..8f65b14b4d 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -13,11 +13,11 @@ x-flow-worker-env: &flow-worker-env AWS_SECRET_ACCESS_KEY: ${AWS_SECRET_ACCESS_KEY:-""} AWS_REGION: ${AWS_REGION:-""} # enables worker profiling using Go's pprof - ENABLE_PROFILING: true + ENABLE_PROFILING: "true" # enables exporting of mirror metrics to Prometheus for visualization using Grafana - ENABLE_METRICS: true + ENABLE_METRICS: "true" # enables exporting of mirror metrics to Catalog in the PEERDB_STATS schema. - ENABLE_STATS: true + ENABLE_STATS: "true" services: catalog: @@ -83,6 +83,7 @@ services: environment: - TEMPORAL_ADDRESS=temporal:7233 - TEMPORAL_CORS_ORIGINS=http://localhost:3000 + - TEMPORAL_CSRF_COOKIE_INSECURE=true image: temporalio/ui:2.17.2 ports: - 8085:8080 From f52b4dc5257b39e151c7498be20c203feb074b2d Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Wed, 30 Aug 2023 20:54:21 +0530 Subject: [PATCH 06/12] QRep Flow: 5 min timeout and removes retry (#364) --- flow/workflows/qrep_flow.go | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/flow/workflows/qrep_flow.go b/flow/workflows/qrep_flow.go index 10d3e089d0..bceb1fa7f7 100644 --- a/flow/workflows/qrep_flow.go +++ b/flow/workflows/qrep_flow.go @@ -72,10 +72,7 @@ func (q *QRepFlowExecution) GetPartitions( func (q *QRepFlowExecution) ReplicatePartitions(ctx workflow.Context, partitions *protos.QRepPartitionBatch) error { ctx = workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ StartToCloseTimeout: 24 * 5 * time.Hour, - RetryPolicy: &temporal.RetryPolicy{ - MaximumAttempts: 20, - }, - HeartbeatTimeout: 1 * time.Hour, + HeartbeatTimeout: 5 * time.Minute, }) msg := fmt.Sprintf("replicating partition batch - %d", partitions.BatchId) From c7acbf0bdadbefc37201405df028dcf3705aa2e6 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Wed, 30 Aug 2023 21:16:27 +0530 Subject: [PATCH 07/12] Changes to ubuntu-latest in stable debian file (#365) --- .github/workflows/stable-debian.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/stable-debian.yml b/.github/workflows/stable-debian.yml index 96c14812dd..6f337c1dc9 100644 --- a/.github/workflows/stable-debian.yml +++ b/.github/workflows/stable-debian.yml @@ -8,7 +8,7 @@ on: jobs: release: name: build and release - runs-on: ubuntu-latest-64-core + runs-on: ubuntu-latest permissions: contents: write steps: From 87a54a1b71a4af88839c34cd1529e8f3995874da Mon Sep 17 00:00:00 2001 From: Kevin K Biju <52661649+heavycrystal@users.noreply.github.com> Date: Wed, 30 Aug 2023 23:37:05 +0530 Subject: [PATCH 08/12] optimizing flow tests by separating and parallelizing test suites (#356) 1. Split each connector into its own test suite, and rewrote them to allow running multiple test suites in parallel. 2. Better separation on source PG instance [shared by all test suites of a run] and destination BQ and SF instances [shared by all runs]. 3. some code cleanup and fixing some tests up. 4. Switched to using a vanilla PG instance, but configured with higher logical replication limits. --- .github/workflows/flow.yml | 20 +- flow/e2e/{ => bigquery}/bigquery_helper.go | 5 +- flow/e2e/bigquery/peer_flow_bq_test.go | 925 +++++++++ flow/e2e/bigquery/qrep_flow_bq_test.go | 123 ++ flow/e2e/congen.go | 99 + flow/e2e/{ => eventhub}/eventhub_helper.go | 2 +- flow/e2e/{ => eventhub}/peer_flow_eh_test.go | 74 +- flow/e2e/peer_flow_s3_test.go | 153 -- flow/e2e/peer_flow_test.go | 1809 ----------------- flow/e2e/postgres/qrep_flow_pg_test.go | 147 ++ flow/e2e/qrep_flow_test.go | 625 ------ flow/e2e/s3/qrep_flow_s3_test.go | 183 ++ flow/e2e/{ => s3}/s3_helper.go | 2 +- flow/e2e/snowflake/peer_flow_sf_test.go | 800 ++++++++ flow/e2e/snowflake/qrep_flow_sf_test.go | 232 +++ flow/e2e/snowflake/snowflake_helper.go | 131 ++ flow/e2e/snowflake_helper.go | 104 - .../qrep_flow_sqlserver_test.go | 102 +- flow/e2e/{ => sqlserver}/sqlserver_helper.go | 2 +- flow/e2e/test_utils.go | 290 ++- 20 files changed, 3098 insertions(+), 2730 deletions(-) rename flow/e2e/{ => bigquery}/bigquery_helper.go (99%) create mode 100644 flow/e2e/bigquery/peer_flow_bq_test.go create mode 100644 flow/e2e/bigquery/qrep_flow_bq_test.go rename flow/e2e/{ => eventhub}/eventhub_helper.go (99%) rename flow/e2e/{ => eventhub}/peer_flow_eh_test.go (57%) delete mode 100644 flow/e2e/peer_flow_s3_test.go delete mode 100644 flow/e2e/peer_flow_test.go create mode 100644 flow/e2e/postgres/qrep_flow_pg_test.go delete mode 100644 flow/e2e/qrep_flow_test.go create mode 100644 flow/e2e/s3/qrep_flow_s3_test.go rename flow/e2e/{ => s3}/s3_helper.go (99%) create mode 100644 flow/e2e/snowflake/peer_flow_sf_test.go create mode 100644 flow/e2e/snowflake/qrep_flow_sf_test.go create mode 100644 flow/e2e/snowflake/snowflake_helper.go delete mode 100644 flow/e2e/snowflake_helper.go rename flow/e2e/{ => sqlserver}/qrep_flow_sqlserver_test.go (53%) rename flow/e2e/{ => sqlserver}/sqlserver_helper.go (99%) diff --git a/.github/workflows/flow.yml b/.github/workflows/flow.yml index b7e2085b47..5f1623eba3 100644 --- a/.github/workflows/flow.yml +++ b/.github/workflows/flow.yml @@ -1,10 +1,10 @@ name: Flow build and test on: - push: - branches: [main] pull_request: branches: [main] + push: + branches: [main] jobs: flow_test: @@ -12,7 +12,7 @@ jobs: timeout-minutes: 30 services: pg_cdc: - image: debezium/postgres:14-alpine + image: postgres:15.4-alpine ports: - 7132:5432 env: @@ -20,6 +20,7 @@ jobs: POSTGRES_PASSWORD: postgres POSTGRES_DB: postgres options: >- + --name pg_cdc --health-cmd pg_isready --health-interval 10s --health-timeout 5s @@ -55,18 +56,21 @@ jobs: name: "snowflake_creds.json" json: ${{ secrets.SNOWFLAKE_GH_CI_PKEY }} - - name: create hstore extension + - name: create hstore extension and increase logical replication limits run: | - sudo apt-get update - sudo apt-get install -y postgresql-client - psql -h localhost -p 7132 -U postgres -c "CREATE EXTENSION hstore;" + docker exec pg_cdc psql -h localhost -p 5432 -U postgres -c "CREATE EXTENSION hstore;" + docker exec pg_cdc psql -h localhost -p 5432 -U postgres -c "ALTER SYSTEM SET wal_level=logical;" + docker exec pg_cdc psql -h localhost -p 5432 -U postgres -c "ALTER SYSTEM SET max_replication_slots=100;" + docker exec pg_cdc psql -h localhost -p 5432 -U postgres -c "ALTER SYSTEM SET max_wal_senders=100;" + docker restart pg_cdc working-directory: ./flow env: + PG_CDC: PGPASSWORD: postgres - name: run tests run: | - gotestsum --format testname -- -p 1 ./... -timeout 1200s + gotestsum --format testname -- -p 4 ./... -timeout 1200s working-directory: ./flow env: AWS_ACCESS_KEY_ID: ${{ secrets.AWS_ACCESS_KEY_ID }} diff --git a/flow/e2e/bigquery_helper.go b/flow/e2e/bigquery/bigquery_helper.go similarity index 99% rename from flow/e2e/bigquery_helper.go rename to flow/e2e/bigquery/bigquery_helper.go index 5e9083ec98..72e8b3edea 100644 --- a/flow/e2e/bigquery_helper.go +++ b/flow/e2e/bigquery/bigquery_helper.go @@ -1,4 +1,4 @@ -package e2e +package e2e_bigquery import ( "context" @@ -12,6 +12,7 @@ import ( "cloud.google.com/go/bigquery" "cloud.google.com/go/civil" peer_bq "github.com/PeerDB-io/peer-flow/connectors/bigquery" + "github.com/PeerDB-io/peer-flow/e2e" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" @@ -45,7 +46,7 @@ func NewBigQueryTestHelper() (*BigQueryTestHelper, error) { return nil, fmt.Errorf("TEST_BQ_CREDS env var not set") } - content, err := readFileToBytes(jsonPath) + content, err := e2e.ReadFileToBytes(jsonPath) if err != nil { return nil, fmt.Errorf("failed to read file: %w", err) } diff --git a/flow/e2e/bigquery/peer_flow_bq_test.go b/flow/e2e/bigquery/peer_flow_bq_test.go new file mode 100644 index 0000000000..655a317854 --- /dev/null +++ b/flow/e2e/bigquery/peer_flow_bq_test.go @@ -0,0 +1,925 @@ +package e2e_bigquery + +import ( + "context" + "fmt" + "testing" + + "github.com/PeerDB-io/peer-flow/e2e" + "github.com/PeerDB-io/peer-flow/generated/protos" + peerflow "github.com/PeerDB-io/peer-flow/workflows" + "github.com/jackc/pgx/v5/pgxpool" + "github.com/joho/godotenv" + log "github.com/sirupsen/logrus" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + "go.temporal.io/sdk/testsuite" +) + +const bigquerySuffix = "bigquery" + +type PeerFlowE2ETestSuiteBQ struct { + suite.Suite + testsuite.WorkflowTestSuite + + pool *pgxpool.Pool + bqHelper *BigQueryTestHelper +} + +func TestPeerFlowE2ETestSuiteBQ(t *testing.T) { + suite.Run(t, new(PeerFlowE2ETestSuiteBQ)) +} + +func (s *PeerFlowE2ETestSuiteBQ) attachSchemaSuffix(tableName string) string { + return fmt.Sprintf("e2e_test_%s.%s", bigquerySuffix, tableName) +} + +func (s *PeerFlowE2ETestSuiteBQ) attachSuffix(input string) string { + return fmt.Sprintf("%s_%s", input, bigquerySuffix) +} + +// setupBigQuery sets up the bigquery connection. +func (s *PeerFlowE2ETestSuiteBQ) setupBigQuery() error { + bqHelper, err := NewBigQueryTestHelper() + if err != nil { + return fmt.Errorf("failed to create bigquery helper: %w", err) + } + + err = bqHelper.RecreateDataset() + if err != nil { + return fmt.Errorf("failed to recreate bigquery dataset: %w", err) + } + + s.bqHelper = bqHelper + return nil +} + +// Implement SetupAllSuite interface to setup the test suite +func (s *PeerFlowE2ETestSuiteBQ) SetupSuite() { + err := godotenv.Load() + if err != nil { + // it's okay if the .env file is not present + // we will use the default values + log.Infof("Unable to load .env file, using default values from env") + } + + log.SetReportCaller(true) + + pool, err := e2e.SetupPostgres(bigquerySuffix) + if err != nil { + s.Fail("failed to setup postgres", err) + } + s.pool = pool + + err = s.setupBigQuery() + if err != nil { + s.Fail("failed to setup bigquery", err) + } +} + +// Implement TearDownAllSuite interface to tear down the test suite +func (s *PeerFlowE2ETestSuiteBQ) TearDownSuite() { + err := e2e.TearDownPostgres(s.pool, bigquerySuffix) + if err != nil { + s.Fail("failed to drop Postgres schema", err) + } + + err = s.bqHelper.DropDataset() + if err != nil { + s.Fail("failed to drop bigquery dataset", err) + } +} + +func (s *PeerFlowE2ETestSuiteBQ) Test_Invalid_Connection_Config() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + // TODO (kaushikiska): ensure flow name can only be alpha numeric and underscores. + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 1, + MaxBatchSize: 1, + } + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, nil, &limits, nil) + + // Verify workflow completes + s.True(env.IsWorkflowCompleted()) + err := env.GetWorkflowError() + + // assert that error contains "invalid connection configs" + s.Error(err) + s.Contains(err.Error(), "invalid connection configs") + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteBQ) Test_Complete_Flow_No_Data() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_no_data") + dstTableName := "test_no_data" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id SERIAL PRIMARY KEY, + key TEXT NOT NULL, + value VARCHAR(255) NOT NULL + ); + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_complete_flow_no_data"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.bqHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 1, + MaxBatchSize: 1, + } + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteBQ) Test_Char_ColType_Error() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_char_coltype") + dstTableName := "test_char_coltype" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id SERIAL PRIMARY KEY, + key TEXT NOT NULL, + value CHAR(255) NOT NULL + ); + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_char_table"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.bqHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 1, + MaxBatchSize: 1, + } + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + env.AssertExpectations(s.T()) +} + +// Test_Complete_Simple_Flow_BQ tests a complete flow with data in the source table. +// The test inserts 10 rows into the source table and verifies that the data is +// correctly synced to the destination table after sync flow completes. +func (s *PeerFlowE2ETestSuiteBQ) Test_Complete_Simple_Flow_BQ() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_simple_flow_bq") + dstTableName := "test_simple_flow_bq" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id SERIAL PRIMARY KEY, + key TEXT NOT NULL, + value TEXT NOT NULL + ); + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_complete_simple_flow"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.bqHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 2, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and then insert 10 rows into the source table + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + // insert 10 rows into the source table + for i := 0; i < 10; i++ { + testKey := fmt.Sprintf("test_key_%d", i) + testValue := fmt.Sprintf("test_value_%d", i) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(key, value) VALUES ($1, $2) + `, srcTableName), testKey, testValue) + s.NoError(err) + } + fmt.Println("Inserted 10 rows into the source table") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + count, err := s.bqHelper.CountRows(dstTableName) + s.NoError(err) + s.Equal(10, count) + + // TODO: verify that the data is correctly synced to the destination table + // on the bigquery side + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteBQ) Test_Toast_BQ() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_toast_bq_1") + dstTableName := "test_toast_bq_1" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id SERIAL PRIMARY KEY, + t1 text, + t2 text, + k int + );CREATE OR REPLACE FUNCTION random_string( int ) RETURNS TEXT as $$ + SELECT string_agg(substring('0123456789bcdfghjkmnpqrstvwxyz', + round(random() * 30)::integer, 1), '') FROM generate_series(1, $1); + $$ language sql; + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_toast_bq_1"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.bqHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 1, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and execute a transaction touching toast columns + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + /* + Executing a transaction which + 1. changes both toast column + 2. changes no toast column + 2. changes 1 toast column + */ + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + BEGIN; + INSERT INTO %s(t1,t2,k) SELECT random_string(9000),random_string(9000), + 1 FROM generate_series(1,2); + UPDATE %s SET k=102 WHERE id=1; + UPDATE %s SET t1='dummy' WHERE id=2; + END; + `, srcTableName, srcTableName, srcTableName)) + s.NoError(err) + fmt.Println("Executed a transaction touching toast columns") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + s.compareTableContentsBQ(dstTableName, "id,t1,t2,k") + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteBQ) Test_Toast_Nochanges_BQ() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_toast_bq_2") + dstTableName := "test_toast_bq_2" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id SERIAL PRIMARY KEY, + t1 text, + t2 text, + k int + );CREATE OR REPLACE FUNCTION random_string( int ) RETURNS TEXT as $$ + SELECT string_agg(substring('0123456789bcdfghjkmnpqrstvwxyz', + round(random() * 30)::integer, 1), '') FROM generate_series(1, $1); + $$ language sql; + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_toast_bq_2"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.bqHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 1, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and execute a transaction touching toast columns + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + /* transaction updating no rows */ + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + BEGIN; + UPDATE %s SET k=102 WHERE id=1; + UPDATE %s SET t1='dummy' WHERE id=2; + END; + `, srcTableName, srcTableName)) + s.NoError(err) + fmt.Println("Executed a transaction touching toast columns") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + s.compareTableContentsBQ(dstTableName, "id,t1,t2,k") + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_1_BQ() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_toast_bq_3") + dstTableName := "test_toast_bq_3" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id SERIAL PRIMARY KEY, + t1 text, + t2 text, + k int + );CREATE OR REPLACE FUNCTION random_string( int ) RETURNS TEXT as $$ + SELECT string_agg(substring('0123456789bcdfghjkmnpqrstvwxyz', + round(random() * 30)::integer, 1), '') FROM generate_series(1, $1); + $$ language sql; + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_toast_bq_3"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.bqHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 1, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and execute a transaction touching toast columns + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + //complex transaction with random DMLs on a table with toast columns + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + BEGIN; + INSERT INTO %s(t1,t2,k) SELECT random_string(9000),random_string(9000), + 1 FROM generate_series(1,2); + UPDATE %s SET k=102 WHERE id=1; + UPDATE %s SET t1='dummy' WHERE id=2; + UPDATE %s SET t2='dummy' WHERE id=2; + DELETE FROM %s WHERE id=1; + INSERT INTO %s(t1,t2,k) SELECT random_string(9000),random_string(9000), + 1 FROM generate_series(1,2); + UPDATE %s SET k=1 WHERE id=1; + UPDATE %s SET t1='dummy1',t2='dummy2' WHERE id=1; + UPDATE %s SET t1='dummy3' WHERE id=3; + DELETE FROM %s WHERE id=2; + DELETE FROM %s WHERE id=3; + DELETE FROM %s WHERE id=2; + END; + `, srcTableName, srcTableName, srcTableName, srcTableName, srcTableName, srcTableName, + srcTableName, srcTableName, srcTableName, srcTableName, srcTableName, srcTableName)) + s.NoError(err) + fmt.Println("Executed a transaction touching toast columns") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + s.compareTableContentsBQ(dstTableName, "id,t1,t2,k") + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_2_BQ() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_toast_bq_4") + dstTableName := "test_toast_bq_4" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE %s ( + id SERIAL PRIMARY KEY, + t1 text, + k int + );CREATE OR REPLACE FUNCTION random_string( int ) RETURNS TEXT as $$ + SELECT string_agg(substring('0123456789bcdfghjkmnpqrstvwxyz', + round(random() * 30)::integer, 1), '') FROM generate_series(1, $1); + $$ language sql; + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_toast_bq_4"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.bqHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 1, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and execute a transaction touching toast columns + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + //complex transaction with random DMLs on a table with toast columns + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + BEGIN; + INSERT INTO %s(t1,k) SELECT random_string(9000), + 1 FROM generate_series(1,1); + UPDATE %s SET t1=sub.t1 FROM (SELECT random_string(9000) t1 + FROM generate_series(1,1) ) sub WHERE id=1; + UPDATE %s SET k=2 WHERE id=1; + UPDATE %s SET k=3 WHERE id=1; + UPDATE %s SET t1=sub.t1 FROM (SELECT random_string(9000) t1 + FROM generate_series(1,1)) sub WHERE id=1; + UPDATE %s SET k=4 WHERE id=1; + END; + `, srcTableName, srcTableName, srcTableName, srcTableName, srcTableName, srcTableName)) + s.NoError(err) + fmt.Println("Executed a transaction touching toast columns") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + s.compareTableContentsBQ(dstTableName, "id,t1,k") + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_3_BQ() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_toast_bq_5") + dstTableName := "test_toast_bq_5" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id SERIAL PRIMARY KEY, + t1 text, + t2 text, + k int + );CREATE OR REPLACE FUNCTION random_string( int ) RETURNS TEXT as $$ + SELECT string_agg(substring('0123456789bcdfghjkmnpqrstvwxyz', + round(random() * 30)::integer, 1), '') FROM generate_series(1, $1); + $$ language sql; + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_toast_bq_5"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.bqHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 1, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and execute a transaction touching toast columns + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + /* + transaction updating a single row + multiple times with changed/unchanged toast columns + */ + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + BEGIN; + INSERT INTO %s(t1,t2,k) SELECT random_string(9000),random_string(9000), + 1 FROM generate_series(1,1); + UPDATE %s SET k=102 WHERE id=1; + UPDATE %s SET t1='dummy' WHERE id=1; + UPDATE %s SET t2='dummy' WHERE id=1; + END; + `, srcTableName, srcTableName, srcTableName, srcTableName)) + s.NoError(err) + fmt.Println("Executed a transaction touching toast columns") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + s.compareTableContentsBQ(dstTableName, "id,t1,t2,k") + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteBQ) Test_Types_BQ() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_types_bq") + dstTableName := "test_types_bq" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s (id serial PRIMARY KEY,c1 BIGINT,c2 BIT,c3 VARBIT,c4 BOOLEAN, + c6 BYTEA,c7 CHARACTER,c8 varchar,c9 CIDR,c11 DATE,c12 FLOAT,c13 DOUBLE PRECISION, + c14 INET,c15 INTEGER,c16 INTERVAL,c17 JSON,c18 JSONB,c21 MACADDR,c22 MONEY, + c23 NUMERIC,c24 OID,c28 REAL,c29 SMALLINT,c30 SMALLSERIAL,c31 SERIAL,c32 TEXT, + c33 TIMESTAMP,c34 TIMESTAMPTZ,c35 TIME, c36 TIMETZ,c37 TSQUERY,c38 TSVECTOR, + c39 TXID_SNAPSHOT,c40 UUID,c41 XML, c42 INT[], c43 FLOAT[], c44 TEXT[]); + CREATE OR REPLACE FUNCTION random_bytea(bytea_length integer) + RETURNS bytea AS $body$ + SELECT decode(string_agg(lpad(to_hex(width_bucket(random(), 0, 1, 256)-1),2,'0') ,''), 'hex') + FROM generate_series(1, $1); + $body$ + LANGUAGE 'sql' + VOLATILE + SET search_path = 'pg_catalog'; + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_types_bq"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.bqHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + + TotalSyncFlows: 1, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and execute a transaction touching toast columns + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + /* test inserting various types*/ + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s SELECT 2,2,b'1',b'101', + true,random_bytea(32),'s','test','1.1.10.2'::cidr, + CURRENT_DATE,1.23,1.234,'192.168.1.5'::inet,1, + '5 years 2 months 29 days 1 minute 2 seconds 200 milliseconds 20000 microseconds'::interval, + '{"sai":1}'::json,'{"sai":1}'::jsonb,'08:00:2b:01:02:03'::macaddr, + 1.2,1.23,4::oid,1.23,1,1,1,'test',now(),now(),now()::time,now()::timetz, + 'fat & rat'::tsquery,'a fat cat sat on a mat and ate a fat rat'::tsvector, + txid_current_snapshot(), + '66073c38-b8df-4bdb-bbca-1c97596b8940'::uuid,xmlcomment('hello'), + ARRAY[10299301,2579827], + ARRAY[0.0003, 8902.0092], + ARRAY['hello','bye']; + `, srcTableName)) + s.NoError(err) + fmt.Println("Executed an insert with all types") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + noNulls, err := s.bqHelper.CheckNull(dstTableName, []string{"c41", "c1", "c2", "c3", "c4", + "c6", "c39", "c40", "id", "c9", "c11", "c12", "c13", "c14", "c15", "c16", "c17", "c18", + "c21", "c22", "c23", "c24", "c28", "c29", "c30", "c31", "c33", "c34", "c35", "c36", + "c37", "c38", "c7", "c8", "c32", "c42", "c43", "c44"}) + if err != nil { + fmt.Println("error %w", err) + } + // Make sure that there are no nulls + s.True(noNulls) + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteBQ) Test_Types_Avro_BQ() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_types_avro_bq") + dstTableName := "test_types_avro_bq" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s (id serial PRIMARY KEY,c1 BIGINT,c2 BIT,c3 VARBIT,c4 BOOLEAN, + c6 BYTEA,c7 CHARACTER,c8 varchar,c9 CIDR,c11 DATE,c12 FLOAT,c13 DOUBLE PRECISION, + c14 INET,c15 INTEGER,c16 INTERVAL,c17 JSON,c18 JSONB,c21 MACADDR,c22 MONEY, + c23 NUMERIC,c24 OID,c28 REAL,c29 SMALLINT,c30 SMALLSERIAL,c31 SERIAL,c32 TEXT, + c33 TIMESTAMP,c34 TIMESTAMPTZ,c35 TIME, c36 TIMETZ,c37 TSQUERY,c38 TSVECTOR, + c39 TXID_SNAPSHOT,c40 UUID,c41 XML, c42 INT[], c43 FLOAT[], c44 TEXT[]); + CREATE OR REPLACE FUNCTION random_bytea(bytea_length integer) + RETURNS bytea AS $body$ + SELECT decode(string_agg(lpad(to_hex(width_bucket(random(), 0, 1, 256)-1),2,'0') ,''), 'hex') + FROM generate_series(1, $1); + $body$ + LANGUAGE 'sql' + VOLATILE + SET search_path = 'pg_catalog'; + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_types_avro_bq"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.bqHelper.Peer, + CDCSyncMode: protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, + CdcStagingPath: "peerdb_staging", + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + + TotalSyncFlows: 1, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and execute a transaction touching toast columns + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + /* test inserting various types*/ + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s SELECT 2,2,b'1',b'101', + true,random_bytea(32),'s','test','1.1.10.2'::cidr, + CURRENT_DATE,1.23,1.234,'192.168.1.5'::inet,1, + '5 years 2 months 29 days 1 minute 2 seconds 200 milliseconds 20000 microseconds'::interval, + '{"sai":1}'::json,'{"sai":1}'::jsonb,'08:00:2b:01:02:03'::macaddr, + 1.2,1.23,4::oid,1.23,1,1,1,'test',now(),now(),now()::time,now()::timetz, + 'fat & rat'::tsquery,'a fat cat sat on a mat and ate a fat rat'::tsvector, + txid_current_snapshot(), + '66073c38-b8df-4bdb-bbca-1c97596b8940'::uuid,xmlcomment('hello'), + ARRAY[9301,239827], + ARRAY[0.0003, 1039.0034], + ARRAY['hello','bye']; + `, srcTableName)) + s.NoError(err) + fmt.Println("Executed an insert with all types") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + noNulls, err := s.bqHelper.CheckNull(dstTableName, []string{"c41", "c1", "c2", "c3", "c4", + "c6", "c39", "c40", "id", "c9", "c11", "c12", "c13", "c14", "c15", "c16", "c17", "c18", + "c21", "c22", "c23", "c24", "c28", "c29", "c30", "c31", "c33", "c34", "c35", "c36", + "c37", "c38", "c7", "c8", "c32", "c42", "c43", "c44"}) + if err != nil { + fmt.Println("error %w", err) + } + // Make sure that there are no nulls + s.True(noNulls) + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteBQ) Test_Simple_Flow_BQ_Avro_CDC() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_simple_flow_bq_avro_cdc") + dstTableName := "test_simple_flow_bq_avro_cdc" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id SERIAL PRIMARY KEY, + key TEXT NOT NULL, + value TEXT NOT NULL + ); + `, srcTableName)) + s.NoError(err) + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_simple_flow_bq_avro_cdc"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.bqHelper.Peer, + CDCSyncMode: protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, + CdcStagingPath: "peerdb_staging", + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 2, + MaxBatchSize: 100, + } + + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + for i := 0; i < 10; i++ { + testKey := fmt.Sprintf("test_key_%d", i) + testValue := fmt.Sprintf("test_value_%d", i) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s (key, value) VALUES ($1, $2) + `, srcTableName), testKey, testValue) + s.NoError(err) + } + fmt.Println("Inserted 10 rows into the source table") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + count, err := s.bqHelper.CountRows(dstTableName) + s.NoError(err) + s.Equal(10, count) + + // TODO: verify that the data is correctly synced to the destination table + // on the bigquery side + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteBQ) Test_Multi_Table_BQ() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTable1Name := s.attachSchemaSuffix("test1_bq") + dstTable1Name := "test1_bq" + srcTable2Name := s.attachSchemaSuffix("test2_bq") + dstTable2Name := "test2_bq" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE %s (id serial primary key, c1 int, c2 text); + CREATE TABLE %s(id serial primary key, c1 int, c2 text); + `, srcTable1Name, srcTable2Name)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_multi_table_bq"), + TableNameMapping: map[string]string{srcTable1Name: dstTable1Name, srcTable2Name: dstTable2Name}, + PostgresPort: e2e.PostgresPort, + Destination: s.bqHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 1, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and execute a transaction touching toast columns + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + /* inserting across multiple tables*/ + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s (c1,c2) VALUES (1,'dummy_1'); + INSERT INTO %s (c1,c2) VALUES (-1,'dummy_-1'); + `, srcTable1Name, srcTable2Name)) + s.NoError(err) + fmt.Println("Executed an insert on two tables") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + require.True(s.T(), env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + count1, err := s.bqHelper.CountRows(dstTable1Name) + s.NoError(err) + count2, err := s.bqHelper.CountRows(dstTable2Name) + s.NoError(err) + + s.Equal(1, count1) + s.Equal(1, count2) + + env.AssertExpectations(s.T()) +} diff --git a/flow/e2e/bigquery/qrep_flow_bq_test.go b/flow/e2e/bigquery/qrep_flow_bq_test.go new file mode 100644 index 0000000000..5e6374cc1a --- /dev/null +++ b/flow/e2e/bigquery/qrep_flow_bq_test.go @@ -0,0 +1,123 @@ +package e2e_bigquery + +import ( + "context" + "fmt" + + connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" + "github.com/PeerDB-io/peer-flow/e2e" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/stretchr/testify/require" +) + +func (s *PeerFlowE2ETestSuiteBQ) setupSourceTable(tableName string, rowCount int) { + err := e2e.CreateSourceTableQRep(s.pool, bigquerySuffix, tableName) + s.NoError(err) + err = e2e.PopulateSourceTable(s.pool, bigquerySuffix, tableName, rowCount) + s.NoError(err) +} + +func (s *PeerFlowE2ETestSuiteBQ) setupBQDestinationTable(dstTable string) { + schema := e2e.GetOwnersSchema() + err := s.bqHelper.CreateTable(dstTable, schema) + + // fail if table creation fails + require.NoError(s.T(), err) + + fmt.Printf("created table on bigquery: %s.%s. %v\n", s.bqHelper.Config.DatasetId, dstTable, err) +} + +func (s *PeerFlowE2ETestSuiteBQ) compareTableContentsBQ(tableName string, colsString string) { + // read rows from source table + pgQueryExecutor := connpostgres.NewQRepQueryExecutor(s.pool, context.Background(), "testflow", "testpart") + pgQueryExecutor.SetTestEnv(true) + + pgRows, err := pgQueryExecutor.ExecuteAndProcessQuery( + fmt.Sprintf("SELECT %s FROM e2e_test_%s.%s ORDER BY id", colsString, bigquerySuffix, tableName), + ) + s.NoError(err) + + // read rows from destination table + qualifiedTableName := fmt.Sprintf("`%s.%s`", s.bqHelper.Config.DatasetId, tableName) + bqRows, err := s.bqHelper.ExecuteAndProcessQuery( + fmt.Sprintf("SELECT %s FROM %s ORDER BY id", colsString, qualifiedTableName), + ) + s.NoError(err) + + s.True(pgRows.Equals(bqRows), "rows from source and destination tables are not equal") +} + +func (s *PeerFlowE2ETestSuiteBQ) Test_Complete_QRep_Flow_Avro() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + numRows := 10 + + tblName := "test_qrep_flow_avro" + s.setupSourceTable(tblName, numRows) + s.setupBQDestinationTable(tblName) + + query := fmt.Sprintf("SELECT * FROM e2e_test_%s.%s WHERE updated_at BETWEEN {{.start}} AND {{.end}}", + bigquerySuffix, tblName) + + qrepConfig, err := e2e.CreateQRepWorkflowConfig("test_qrep_flow_avro", + fmt.Sprintf("e2e_test_%s.%s", bigquerySuffix, tblName), + tblName, + query, + protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, + s.bqHelper.Peer, + "peerdb_staging") + s.NoError(err) + e2e.RunQrepFlowWorkflow(env, qrepConfig) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + + // assert that error contains "invalid connection configs" + err = env.GetWorkflowError() + s.NoError(err) + + s.compareTableContentsBQ(tblName, "*") + + env.AssertExpectations(s.T()) +} + +// NOTE: Disabled due to large JSON tests being added: https://github.com/PeerDB-io/peerdb/issues/309 + +// Test_Complete_QRep_Flow tests a complete flow with data in the source table. +// The test inserts 10 rows into the source table and verifies that the data is +// // correctly synced to the destination table this runs a QRep Flow. +// func (s *E2EPeerFlowTestSuite) Test_Complete_QRep_Flow_Multi_Insert() { +// env := s.NewTestWorkflowEnvironment() +// registerWorkflowsAndActivities(env) + +// numRows := 10 + +// tblName := "test_qrep_flow_multi_insert" +// s.setupSourceTable(tblName, numRows) +// s.setupBQDestinationTable(tblName) + +// query := fmt.Sprintf("SELECT * FROM e2e_test.%s WHERE updated_at BETWEEN {{.start}} AND {{.end}}", tblName) + +// qrepConfig := s.createQRepWorkflowConfig("test_qrep_flow_mi", +// "e2e_test."+tblName, +// tblName, +// query, +// protos.QRepSyncMode_QREP_SYNC_MODE_MULTI_INSERT, +// s.bqHelper.Peer) +// runQrepFlowWorkflow(env, qrepConfig) + +// // Verify workflow completes without error +// s.True(env.IsWorkflowCompleted()) + +// // assert that error contains "invalid connection configs" +// err := env.GetWorkflowError() +// s.NoError(err) + +// count, err := s.bqHelper.CountRows(tblName) +// s.NoError(err) + +// s.Equal(numRows, count) + +// env.AssertExpectations(s.T()) +// } diff --git a/flow/e2e/congen.go b/flow/e2e/congen.go index f82912947e..14725d7899 100644 --- a/flow/e2e/congen.go +++ b/flow/e2e/congen.go @@ -1,9 +1,108 @@ package e2e import ( + "context" + "fmt" + + "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/jackc/pgx/v5/pgxpool" +) + +const ( + postgresHost = "localhost" + postgresUser = "postgres" + postgresPassword = "postgres" + postgresDatabase = "postgres" + PostgresPort = 7132 ) +func GetTestPostgresConf() *protos.PostgresConfig { + return &protos.PostgresConfig{ + Host: postgresHost, + Port: uint32(PostgresPort), + User: postgresUser, + Password: postgresPassword, + Database: postgresDatabase, + } +} + +func cleanPostgres(pool *pgxpool.Pool, suffix string) error { + // drop the e2e_test schema with the given suffix if it exists + _, err := pool.Exec(context.Background(), fmt.Sprintf("DROP SCHEMA IF EXISTS e2e_test_%s CASCADE", suffix)) + if err != nil { + return fmt.Errorf("failed to drop e2e_test schema: %w", err) + } + + // drop all open slots with the given suffix + _, err = pool.Exec( + context.Background(), + "SELECT pg_drop_replication_slot(slot_name) FROM pg_replication_slots WHERE slot_name LIKE $1", + fmt.Sprintf("%%_%s", suffix), + ) + if err != nil { + return fmt.Errorf("failed to drop replication slots: %w", err) + } + + // list all publications from pg_publication table + rows, err := pool.Query(context.Background(), + "SELECT pubname FROM pg_publication WHERE pubname LIKE $1", + fmt.Sprintf("%%_%s", suffix), + ) + if err != nil { + return fmt.Errorf("failed to list publications: %w", err) + } + + // drop all publications with the given suffix + for rows.Next() { + var pubName string + err = rows.Scan(&pubName) + if err != nil { + return fmt.Errorf("failed to scan publication name: %w", err) + } + + _, err = pool.Exec(context.Background(), fmt.Sprintf("DROP PUBLICATION %s", pubName)) + if err != nil { + return fmt.Errorf("failed to drop publication %s: %w", pubName, err) + } + } + + return nil +} + +// setupPostgres sets up the postgres connection pool. +func SetupPostgres(suffix string) (*pgxpool.Pool, error) { + pool, err := pgxpool.New(context.Background(), utils.GetPGConnectionString(GetTestPostgresConf())) + if err != nil { + return nil, fmt.Errorf("failed to create postgres connection pool: %w", err) + } + + err = cleanPostgres(pool, suffix) + if err != nil { + return nil, err + } + + // create an e2e_test schema + _, err = pool.Exec(context.Background(), fmt.Sprintf("CREATE SCHEMA e2e_test_%s", suffix)) + if err != nil { + return nil, fmt.Errorf("failed to create e2e_test schema: %w", err) + } + + return pool, nil +} + +func TearDownPostgres(pool *pgxpool.Pool, suffix string) error { + // drop the e2e_test schema + if pool != nil { + err := cleanPostgres(pool, suffix) + if err != nil { + return err + } + pool.Close() + } + return nil +} + // GeneratePostgresPeer generates a postgres peer config for testing. func GeneratePostgresPeer(postgresPort int) *protos.Peer { ret := &protos.Peer{} diff --git a/flow/e2e/eventhub_helper.go b/flow/e2e/eventhub/eventhub_helper.go similarity index 99% rename from flow/e2e/eventhub_helper.go rename to flow/e2e/eventhub/eventhub_helper.go index 2457361cb6..e488da4287 100644 --- a/flow/e2e/eventhub_helper.go +++ b/flow/e2e/eventhub/eventhub_helper.go @@ -1,4 +1,4 @@ -package e2e +package e2e_eventhub import ( "context" diff --git a/flow/e2e/peer_flow_eh_test.go b/flow/e2e/eventhub/peer_flow_eh_test.go similarity index 57% rename from flow/e2e/peer_flow_eh_test.go rename to flow/e2e/eventhub/peer_flow_eh_test.go index 23a85abde8..687f17face 100644 --- a/flow/e2e/peer_flow_eh_test.go +++ b/flow/e2e/eventhub/peer_flow_eh_test.go @@ -1,23 +1,44 @@ -package e2e +package e2e_eventhub import ( "context" "fmt" "os" + "testing" "time" + "github.com/PeerDB-io/peer-flow/e2e" util "github.com/PeerDB-io/peer-flow/utils" peerflow "github.com/PeerDB-io/peer-flow/workflows" + "github.com/jackc/pgx/v5/pgxpool" + "github.com/joho/godotenv" + log "github.com/sirupsen/logrus" "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + "go.temporal.io/sdk/testsuite" ) -func (s *E2EPeerFlowTestSuite) setupEventHub() error { +const eventhubSuffix = "eventhub" + +type PeerFlowE2ETestSuiteEH struct { + suite.Suite + testsuite.WorkflowTestSuite + + pool *pgxpool.Pool + ehHelper *EventHubTestHelper +} + +func TestPeerFlowE2ETestSuiteEH(t *testing.T) { + suite.Run(t, new(PeerFlowE2ETestSuiteEH)) +} + +func (s *PeerFlowE2ETestSuiteEH) setupEventHub() error { enableEHT := os.Getenv("ENABLE_EVENT_HUB_TESTS") if enableEHT == "" { return nil } - pgConf := GetTestPostgresConf() + pgConf := e2e.GetTestPostgresConf() helper, err := NewEventHubTestHelper(pgConf) if err != nil { return err @@ -27,13 +48,50 @@ func (s *E2EPeerFlowTestSuite) setupEventHub() error { return nil } -func (s *E2EPeerFlowTestSuite) Test_Complete_Simple_Flow_EH() { +func (s *PeerFlowE2ETestSuiteEH) SetupSuite() { + err := godotenv.Load() + if err != nil { + // it's okay if the .env file is not present + // we will use the default values + log.Infof("Unable to load .env file, using default values from env") + } + + log.SetReportCaller(true) + + pool, err := e2e.SetupPostgres(eventhubSuffix) + if err != nil { + s.Fail("failed to setup postgres", err) + } + s.pool = pool + + err = s.setupEventHub() + if err != nil { + s.Fail("failed to setup eventhub", err) + } +} + +// Implement TearDownAllSuite interface to tear down the test suite +func (s *PeerFlowE2ETestSuiteEH) TearDownSuite() { + err := e2e.TearDownPostgres(s.pool, eventhubSuffix) + if err != nil { + s.Fail("failed to drop Postgres schema", err) + } + + if s.ehHelper != nil { + err = s.ehHelper.CleanUp() + if err != nil { + s.Fail("failed to clean up eventhub", err) + } + } +} + +func (s *PeerFlowE2ETestSuiteEH) Test_Complete_Simple_Flow_EH() { if s.ehHelper == nil { s.T().Skip("Skipping EventHub test") } env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env) ru, err := util.RandomUInt64() s.NoError(err) @@ -49,10 +107,10 @@ func (s *E2EPeerFlowTestSuite) Test_Complete_Simple_Flow_EH() { `) s.NoError(err) - connectionGen := FlowConnectionGenerationConfig{ + connectionGen := e2e.FlowConnectionGenerationConfig{ FlowJobName: jobName, TableNameMapping: map[string]string{schemaQualifiedName: jobName}, - PostgresPort: postgresPort, + PostgresPort: e2e.PostgresPort, Destination: s.ehHelper.GetPeer(), } @@ -67,7 +125,7 @@ func (s *E2EPeerFlowTestSuite) Test_Complete_Simple_Flow_EH() { // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup // and then insert 10 rows into the source table go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) + e2e.SetupPeerFlowStatusQuery(env, connectionGen) // insert 10 rows into the source table for i := 0; i < 10; i++ { testKey := fmt.Sprintf("test_key_%d", i) diff --git a/flow/e2e/peer_flow_s3_test.go b/flow/e2e/peer_flow_s3_test.go deleted file mode 100644 index 7cc6e7e387..0000000000 --- a/flow/e2e/peer_flow_s3_test.go +++ /dev/null @@ -1,153 +0,0 @@ -package e2e - -import ( - "context" - "fmt" - "time" - - "github.com/PeerDB-io/peer-flow/generated/protos" - util "github.com/PeerDB-io/peer-flow/utils" - "github.com/stretchr/testify/require" -) - -func (s *E2EPeerFlowTestSuite) setupS3() error { - helper, err := NewS3TestHelper() - if err != nil { - return err - } - - s.s3Helper = helper - return nil -} - -func (s *E2EPeerFlowTestSuite) Test_Complete_QRep_Flow_S3() { - if s.s3Helper == nil { - s.T().Skip("Skipping S3 test") - } - - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - ru, err := util.RandomUInt64() - s.NoError(err) - - jobName := fmt.Sprintf("test_complete_flow_s3_%d", ru) - schemaQualifiedName := fmt.Sprintf("e2e_test.%s", jobName) - _, err = s.pool.Exec(context.Background(), ` - CREATE TABLE `+schemaQualifiedName+` ( - id SERIAL PRIMARY KEY, - key TEXT NOT NULL, - value TEXT NOT NULL - ); - `) - s.NoError(err) - - tblName := "test_qrep_flow_s3_1" - s.setupSourceTable(tblName, 10) - query := fmt.Sprintf("SELECT * FROM e2e_test.%s WHERE updated_at >= {{.start}} AND updated_at < {{.end}}", tblName) - qrepConfig := s.createQRepWorkflowConfig( - jobName, - "e2e_test."+tblName, - "e2e_dest_1", - query, - protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, - s.s3Helper.GetPeer(), - "stage", - ) - qrepConfig.StagingPath = s.s3Helper.s3Config.Url - - runQrepFlowWorkflow(env, qrepConfig) - - go func() { - // insert 10 rows into the source table - for i := 0; i < 10; i++ { - testKey := fmt.Sprintf("test_key_%d", i) - testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.pool.Exec(context.Background(), ` - INSERT INTO `+schemaQualifiedName+` (key, value) VALUES ($1, $2) - `, testKey, testValue) - s.NoError(err) - } - fmt.Println("Inserted 10 rows into the source table") - }() - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - s.NoError(err) - - // Verify destination has 1 file - // make context with timeout - ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) - defer cancel() - - files, err := s.s3Helper.ListAllFiles(ctx, jobName) - - require.NoError(s.T(), err) - - require.Equal(s.T(), 1, len(files)) - - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Complete_QRep_Flow_S3_CTID() { - if s.s3Helper == nil { - s.T().Skip("Skipping S3 test") - } - - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - ru, err := util.RandomUInt64() - s.NoError(err) - - jobName := fmt.Sprintf("test_complete_flow_s3_ctid_%d", ru) - schemaQualifiedName := fmt.Sprintf("e2e_test.%s", jobName) - _, err = s.pool.Exec(context.Background(), ` - CREATE TABLE `+schemaQualifiedName+` ( - id SERIAL PRIMARY KEY, - key TEXT NOT NULL, - value TEXT NOT NULL - ); - `) - s.NoError(err) - - tblName := "test_qrep_flow_s3_ctid" - s.setupSourceTable(tblName, 20000) - query := fmt.Sprintf("SELECT * FROM e2e_test.%s WHERE ctid BETWEEN {{.start}} AND {{.end}}", tblName) - qrepConfig := s.createQRepWorkflowConfig( - jobName, - "e2e_test."+tblName, - "e2e_dest_ctid", - query, - protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, - s.s3Helper.GetPeer(), - "stage", - ) - qrepConfig.StagingPath = s.s3Helper.s3Config.Url - qrepConfig.NumRowsPerPartition = 2000 - qrepConfig.InitialCopyOnly = true - qrepConfig.WatermarkColumn = "ctid" - - runQrepFlowWorkflow(env, qrepConfig) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - s.NoError(err) - - // Verify destination has 1 file - // make context with timeout - ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) - defer cancel() - - files, err := s.s3Helper.ListAllFiles(ctx, jobName) - - require.NoError(s.T(), err) - - require.Equal(s.T(), 10, len(files)) - - env.AssertExpectations(s.T()) -} diff --git a/flow/e2e/peer_flow_test.go b/flow/e2e/peer_flow_test.go deleted file mode 100644 index 4df215cd3f..0000000000 --- a/flow/e2e/peer_flow_test.go +++ /dev/null @@ -1,1809 +0,0 @@ -package e2e - -import ( - "context" - "fmt" - "math/rand" - "os" - "testing" - "time" - - "github.com/PeerDB-io/peer-flow/activities" - "github.com/PeerDB-io/peer-flow/connectors/utils" - "github.com/PeerDB-io/peer-flow/generated/protos" - util "github.com/PeerDB-io/peer-flow/utils" - peerflow "github.com/PeerDB-io/peer-flow/workflows" - "github.com/jackc/pgx/v5/pgxpool" - "github.com/joho/godotenv" - log "github.com/sirupsen/logrus" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" - "go.temporal.io/sdk/testsuite" -) - -type E2EPeerFlowTestSuite struct { - suite.Suite - testsuite.WorkflowTestSuite - - pgConnStr string - pool *pgxpool.Pool - - bqHelper *BigQueryTestHelper - sfHelper *SnowflakeTestHelper - ehHelper *EventHubTestHelper - s3Helper *S3TestHelper - sqlsHelper *SQLServerHelper -} - -func TestE2EPeerFlowTestSuite(t *testing.T) { - suite.Run(t, new(E2EPeerFlowTestSuite)) -} - -const ( - postgresHost = "localhost" - postgresUser = "postgres" - postgresPassword = "postgres" - postgresDatabase = "postgres" - postgresPort = 7132 -) - -func GetTestPostgresConf() *protos.PostgresConfig { - return &protos.PostgresConfig{ - Host: postgresHost, - Port: uint32(postgresPort), - User: postgresUser, - Password: postgresPassword, - Database: postgresDatabase, - } -} - -// setupPostgres sets up the postgres connection pool. -func (s *E2EPeerFlowTestSuite) setupPostgres() error { - s.pgConnStr = utils.GetPGConnectionString(GetTestPostgresConf()) - pool, err := pgxpool.New(context.Background(), s.pgConnStr) - if err != nil { - return fmt.Errorf("failed to create postgres connection pool: %w", err) - } - - s.pool = pool - - // drop the e2e_test schema if it exists - _, err = s.pool.Exec(context.Background(), "DROP SCHEMA IF EXISTS e2e_test CASCADE") - if err != nil { - return fmt.Errorf("failed to drop e2e_test schema: %w", err) - } - - // create an e2e_test schema - _, err = s.pool.Exec(context.Background(), "CREATE SCHEMA e2e_test") - if err != nil { - return fmt.Errorf("failed to create e2e_test schema: %w", err) - } - - // drop all open slots - _, err = s.pool.Exec( - context.Background(), - "SELECT pg_drop_replication_slot(slot_name) FROM pg_replication_slots", - ) - if err != nil { - return fmt.Errorf("failed to drop replication slots: %w", err) - } - - // list all publications from pg_publication table - rows, err := s.pool.Query(context.Background(), "SELECT pubname FROM pg_publication") - if err != nil { - return fmt.Errorf("failed to list publications: %w", err) - } - - // drop all publications - for rows.Next() { - var pubName string - err = rows.Scan(&pubName) - if err != nil { - return fmt.Errorf("failed to scan publication name: %w", err) - } - - _, err = s.pool.Exec(context.Background(), fmt.Sprintf("DROP PUBLICATION %s", pubName)) - if err != nil { - return fmt.Errorf("failed to drop publication %s: %w", pubName, err) - } - } - - return nil -} - -// setupBigQuery sets up the bigquery connection. -func (s *E2EPeerFlowTestSuite) setupBigQuery() error { - bqHelper, err := NewBigQueryTestHelper() - if err != nil { - return fmt.Errorf("failed to create bigquery helper: %w", err) - } - - err = bqHelper.RecreateDataset() - if err != nil { - return fmt.Errorf("failed to recreate bigquery dataset: %w", err) - } - - s.bqHelper = bqHelper - return nil -} - -// setupSnowflake sets up the snowflake connection. -func (s *E2EPeerFlowTestSuite) setupSnowflake() error { - runID, err := util.RandomUInt64() - if err != nil { - return fmt.Errorf("failed to generate random uint64: %w", err) - } - - testSchemaName := fmt.Sprintf("e2e_test_%d", runID) - - sfHelper, err := NewSnowflakeTestHelper(testSchemaName) - if err != nil { - return fmt.Errorf("failed to create snowflake helper: %w", err) - } - - err = sfHelper.RecreateSchema() - if err != nil { - return fmt.Errorf("failed to recreate snowflake schema: %w", err) - } - s.sfHelper = sfHelper - - // for every test, drop the _PEERDB_INTERNAL schema - err = s.sfHelper.client.DropSchema("_PEERDB_INTERNAL") - require.NoError(s.T(), err) - - return nil -} - -// setup sql server connection -func (s *E2EPeerFlowTestSuite) setupSQLServer() { - env := os.Getenv("ENABLE_SQLSERVER_TESTS") - if env != "true" { - s.sqlsHelper = nil - return - } - - sqlsHelper, err := NewSQLServerHelper("test_sqlserver_peer") - require.NoError(s.T(), err) - s.sqlsHelper = sqlsHelper -} - -// Implement SetupAllSuite interface to setup the test suite -func (s *E2EPeerFlowTestSuite) SetupSuite() { - err := godotenv.Load() - if err != nil { - // it's okay if the .env file is not present - // we will use the default values - log.Infof("Unable to load .env file, using default values from env") - } - - log.SetReportCaller(true) - - // seed the random number generator with current time - rand.Seed(time.Now().UnixNano()) - - err = s.setupPostgres() - if err != nil { - s.Fail("failed to setup postgres", err) - } - - err = s.setupBigQuery() - if err != nil { - s.Fail("failed to setup bigquery", err) - } - - err = s.setupSnowflake() - if err != nil { - s.Fail("failed to setup snowflake", err) - } - - err = s.setupEventHub() - if err != nil { - s.Fail("failed to setup eventhub", err) - } - - err = s.setupS3() - if err != nil { - s.Fail("failed to setup s3", err) - } - - s.setupSQLServer() -} - -// Implement TearDownAllSuite interface to tear down the test suite -func (s *E2EPeerFlowTestSuite) TearDownSuite() { - // drop the e2e_test schema - _, err := s.pool.Exec(context.Background(), "DROP SCHEMA e2e_test CASCADE") - if err != nil { - s.Fail("failed to drop e2e_test schema", err) - } - - if s.pool != nil { - s.pool.Close() - } - - err = s.bqHelper.DropDataset() - if err != nil { - s.Fail("failed to drop bigquery dataset", err) - } - - if s.sfHelper != nil { - err = s.sfHelper.DropSchema() - if err != nil { - s.Fail("failed to drop snowflake schema", err) - } - } else { - s.Fail("snowflake helper is nil, unable to drop snowflake schema") - } - - if s.ehHelper != nil { - err = s.ehHelper.CleanUp() - if err != nil { - s.Fail("failed to clean up eventhub", err) - } - } - - if s.s3Helper != nil { - err = s.s3Helper.CleanUp() - if err != nil { - s.Fail("failed to clean up s3", err) - } - } - - if s.sqlsHelper != nil { - err = s.sqlsHelper.CleanUp() - if err != nil { - s.Fail("failed to clean up sqlserver", err) - } - } -} - -func (s *E2EPeerFlowTestSuite) TearDownTest() { - // clear all replication slots - _, err := s.pool.Exec( - context.Background(), - "SELECT pg_drop_replication_slot(slot_name) FROM pg_replication_slots", - ) - if err != nil { - s.Fail("failed to drop replication slots", err) - } -} - -func registerWorkflowsAndActivities(env *testsuite.TestWorkflowEnvironment) { - // set a 300 second timeout for the workflow to execute a few runs. - env.SetTestTimeout(300 * time.Second) - - env.RegisterWorkflow(peerflow.PeerFlowWorkflow) - env.RegisterWorkflow(peerflow.PeerFlowWorkflowWithConfig) - env.RegisterWorkflow(peerflow.SyncFlowWorkflow) - env.RegisterWorkflow(peerflow.SetupFlowWorkflow) - env.RegisterWorkflow(peerflow.SnapshotFlowWorkflow) - env.RegisterWorkflow(peerflow.NormalizeFlowWorkflow) - env.RegisterWorkflow(peerflow.QRepFlowWorkflow) - env.RegisterWorkflow(peerflow.QRepPartitionWorkflow) - env.RegisterActivity(&activities.FetchConfigActivity{}) - env.RegisterActivity(&activities.FlowableActivity{}) - env.RegisterActivity(&activities.SnapshotActivity{}) -} - -func (s *E2EPeerFlowTestSuite) Test_Invalid_Connection_Config() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - // TODO (kaushikiska): ensure flow name can only be alpha numeric and underscores. - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 1, - MaxBatchSize: 1, - } - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, nil, &limits, nil) - - // Verify workflow completes - s.True(env.IsWorkflowCompleted()) - err := env.GetWorkflowError() - - // assert that error contains "invalid connection configs" - s.Error(err) - s.Contains(err.Error(), "invalid connection configs") - - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Complete_Flow_No_Data() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - CREATE TABLE e2e_test.test ( - id SERIAL PRIMARY KEY, - key TEXT NOT NULL, - value VARCHAR(255) NOT NULL - ); - `) - s.NoError(err) - - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_complete_flow_no_data", - TableNameMapping: map[string]string{"e2e_test.test": "test"}, - PostgresPort: postgresPort, - Destination: s.bqHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 1, - MaxBatchSize: 1, - } - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Char_ColType_Error() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - CREATE TABLE e2e_test.test_char_table ( - id SERIAL PRIMARY KEY, - key TEXT NOT NULL, - value CHAR(255) NOT NULL - ); - `) - s.NoError(err) - - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_char_table", - TableNameMapping: map[string]string{"e2e_test.test_char_table": "test"}, - PostgresPort: postgresPort, - Destination: s.bqHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 1, - MaxBatchSize: 1, - } - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - env.AssertExpectations(s.T()) -} - -// Test_Complete_Simple_Flow_BQ tests a complete flow with data in the source table. -// The test inserts 10 rows into the source table and verifies that the data is -// correctly synced to the destination table after sync flow completes. -func (s *E2EPeerFlowTestSuite) Test_Complete_Simple_Flow_BQ() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - CREATE TABLE e2e_test.test_simple_flow_bq ( - id SERIAL PRIMARY KEY, - key TEXT NOT NULL, - value TEXT NOT NULL - ); - `) - s.NoError(err) - - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_complete_single_col_flow_bq", - TableNameMapping: map[string]string{"e2e_test.test_simple_flow_bq": "test_simple_flow_bq"}, - PostgresPort: postgresPort, - Destination: s.bqHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 2, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and then insert 10 rows into the source table - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - // insert 10 rows into the source table - for i := 0; i < 10; i++ { - testKey := fmt.Sprintf("test_key_%d", i) - testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.pool.Exec(context.Background(), ` - INSERT INTO e2e_test.test_simple_flow_bq (key, value) VALUES ($1, $2) - `, testKey, testValue) - s.NoError(err) - } - fmt.Println("Inserted 10 rows into the source table") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - // TODO: verify that the data is correctly synced to the destination table - // on the bigquery side - - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Toast_BQ() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - - CREATE TABLE e2e_test.test_toast_bq_1 ( - id SERIAL PRIMARY KEY, - t1 text, - t2 text, - k int - );CREATE OR REPLACE FUNCTION random_string( int ) RETURNS TEXT as $$ - SELECT string_agg(substring('0123456789bcdfghjkmnpqrstvwxyz', - round(random() * 30)::integer, 1), '') FROM generate_series(1, $1); - $$ language sql; - `) - s.NoError(err) - - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_toast_bq_1", - TableNameMapping: map[string]string{"e2e_test.test_toast_bq_1": "test_toast_bq_1"}, - PostgresPort: postgresPort, - Destination: s.bqHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 1, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and execute a transaction touching toast columns - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - /* - Executing a transaction which - 1. changes both toast column - 2. changes no toast column - 2. changes 1 toast column - */ - _, err = s.pool.Exec(context.Background(), ` - BEGIN; - INSERT INTO e2e_test.test_toast_bq_1(t1,t2,k) SELECT random_string(9000),random_string(9000), - 1 FROM generate_series(1,2); - UPDATE e2e_test.test_toast_bq_1 SET k=102 WHERE id=1; - UPDATE e2e_test.test_toast_bq_1 SET t1='dummy' WHERE id=2; - END; - `) - s.NoError(err) - fmt.Println("Executed a transaction touching toast columns") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - s.compareTableContentsBQ("test_toast_bq_1", "id,t1,t2,k") - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Toast_Nochanges_BQ() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - - CREATE TABLE e2e_test.test_toast_bq_2 ( - id SERIAL PRIMARY KEY, - t1 text, - t2 text, - k int - );CREATE OR REPLACE FUNCTION random_string( int ) RETURNS TEXT as $$ - SELECT string_agg(substring('0123456789bcdfghjkmnpqrstvwxyz', - round(random() * 30)::integer, 1), '') FROM generate_series(1, $1); - $$ language sql; - `) - s.NoError(err) - - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_toast_bq_2", - TableNameMapping: map[string]string{"e2e_test.test_toast_bq_2": "test_toast_bq_2"}, - PostgresPort: postgresPort, - Destination: s.bqHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 1, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and execute a transaction touching toast columns - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - /* transaction updating no rows */ - _, err = s.pool.Exec(context.Background(), ` - BEGIN; - UPDATE e2e_test.test_toast_bq_2 SET k=102 WHERE id=1; - UPDATE e2e_test.test_toast_bq_2 SET t1='dummy' WHERE id=2; - END; - `) - s.NoError(err) - fmt.Println("Executed a transaction touching toast columns") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - s.compareTableContentsBQ("test_toast_bq_2", "id,t1,t2,k") - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Toast_Advance_1_BQ() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - - CREATE TABLE e2e_test.test_toast_bq_3 ( - id SERIAL PRIMARY KEY, - t1 text, - t2 text, - k int - );CREATE OR REPLACE FUNCTION random_string( int ) RETURNS TEXT as $$ - SELECT string_agg(substring('0123456789bcdfghjkmnpqrstvwxyz', - round(random() * 30)::integer, 1), '') FROM generate_series(1, $1); - $$ language sql; - `) - s.NoError(err) - - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_toast_bq_3", - TableNameMapping: map[string]string{"e2e_test.test_toast_bq_3": "test_toast_bq_3"}, - PostgresPort: postgresPort, - Destination: s.bqHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 1, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and execute a transaction touching toast columns - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - //complex transaction with random DMLs on a table with toast columns - _, err = s.pool.Exec(context.Background(), ` - BEGIN; - INSERT INTO e2e_test.test_toast_bq_3(t1,t2,k) SELECT random_string(9000),random_string(9000), - 1 FROM generate_series(1,2); - UPDATE e2e_test.test_toast_bq_3 SET k=102 WHERE id=1; - UPDATE e2e_test.test_toast_bq_3 SET t1='dummy' WHERE id=2; - UPDATE e2e_test.test_toast_bq_3 SET t2='dummy' WHERE id=2; - DELETE FROM e2e_test.test_toast_bq_3 WHERE id=1; - INSERT INTO e2e_test.test_toast_bq_3(t1,t2,k) SELECT random_string(9000),random_string(9000), - 1 FROM generate_series(1,2); - UPDATE e2e_test.test_toast_bq_3 SET k=1 WHERE id=1; - UPDATE e2e_test.test_toast_bq_3 SET t1='dummy1',t2='dummy2' WHERE id=1; - UPDATE e2e_test.test_toast_bq_3 SET t1='dummy3' WHERE id=3; - DELETE FROM e2e_test.test_toast_bq_3 WHERE id=2; - DELETE FROM e2e_test.test_toast_bq_3 WHERE id=3; - DELETE FROM e2e_test.test_toast_bq_3 WHERE id=2; - END; - `) - s.NoError(err) - fmt.Println("Executed a transaction touching toast columns") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - s.compareTableContentsBQ("test_toast_bq_3", "id,t1,t2,k") - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) SetupPeerFlowStatusQuery(env *testsuite.TestWorkflowEnvironment, - connectionGen FlowConnectionGenerationConfig) { - // wait for PeerFlowStatusQuery to finish setup - // sleep for 5 second to allow the workflow to start - time.Sleep(5 * time.Second) - for { - response, err := env.QueryWorkflow( - peerflow.PeerFlowStatusQuery, - connectionGen.FlowJobName, - ) - if err == nil { - var state peerflow.PeerFlowState - err = response.Get(&state) - s.NoError(err) - - if state.SetupComplete { - fmt.Println("query indicates setup is complete") - break - } - } else { - // log the error for informational purposes - fmt.Println(err) - } - time.Sleep(1 * time.Second) - } -} - -func (s *E2EPeerFlowTestSuite) Test_Toast_Advance_2_BQ() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - - CREATE TABLE e2e_test.test_toast_bq_4 ( - id SERIAL PRIMARY KEY, - t1 text, - k int - );CREATE OR REPLACE FUNCTION random_string( int ) RETURNS TEXT as $$ - SELECT string_agg(substring('0123456789bcdfghjkmnpqrstvwxyz', - round(random() * 30)::integer, 1), '') FROM generate_series(1, $1); - $$ language sql; - `) - s.NoError(err) - - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_toast_bq_4", - TableNameMapping: map[string]string{"e2e_test.test_toast_bq_4": "test_toast_bq_4"}, - PostgresPort: postgresPort, - Destination: s.bqHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 1, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and execute a transaction touching toast columns - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - //complex transaction with random DMLs on a table with toast columns - _, err = s.pool.Exec(context.Background(), ` - BEGIN; - INSERT INTO e2e_test.test_toast_bq_4(t1,k) SELECT random_string(9000), - 1 FROM generate_series(1,1); - UPDATE e2e_test.test_toast_bq_4 SET t1=sub.t1 FROM (SELECT random_string(9000) t1 - FROM generate_series(1,1) ) sub WHERE id=1; - UPDATE e2e_test.test_toast_bq_4 SET k=2 WHERE id=1; - UPDATE e2e_test.test_toast_bq_4 SET k=3 WHERE id=1; - UPDATE e2e_test.test_toast_bq_4 SET t1=sub.t1 FROM (SELECT random_string(9000) t1 - FROM generate_series(1,1)) sub WHERE id=1; - UPDATE e2e_test.test_toast_bq_4 SET k=4 WHERE id=1; - END; - `) - s.NoError(err) - fmt.Println("Executed a transaction touching toast columns") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - s.compareTableContentsBQ("test_toast_bq_4", "id,t1,k") - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Toast_Advance_3_BQ() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - - CREATE TABLE e2e_test.test_toast_bq_5 ( - id SERIAL PRIMARY KEY, - t1 text, - t2 text, - k int - );CREATE OR REPLACE FUNCTION random_string( int ) RETURNS TEXT as $$ - SELECT string_agg(substring('0123456789bcdfghjkmnpqrstvwxyz', - round(random() * 30)::integer, 1), '') FROM generate_series(1, $1); - $$ language sql; - `) - s.NoError(err) - - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_toast_bq_5", - TableNameMapping: map[string]string{"e2e_test.test_toast_bq_5": "test_toast_bq_5"}, - PostgresPort: postgresPort, - Destination: s.bqHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 1, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and execute a transaction touching toast columns - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - /* - transaction updating a single row - multiple times with changed/unchanged toast columns - */ - _, err = s.pool.Exec(context.Background(), ` - BEGIN; - INSERT INTO e2e_test.test_toast_bq_5(t1,t2,k) SELECT random_string(9000),random_string(9000), - 1 FROM generate_series(1,1); - UPDATE e2e_test.test_toast_bq_5 SET k=102 WHERE id=1; - UPDATE e2e_test.test_toast_bq_5 SET t1='dummy' WHERE id=1; - UPDATE e2e_test.test_toast_bq_5 SET t2='dummy' WHERE id=1; - END; - `) - s.NoError(err) - fmt.Println("Executed a transaction touching toast columns") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - s.compareTableContentsBQ("test_toast_bq_5", "id,t1,t2,k") - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Types_BQ() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - - CREATE TABLE e2e_test.test_types_bq(id serial PRIMARY KEY,c1 BIGINT,c2 BIT,c3 VARBIT,c4 BOOLEAN, - c6 BYTEA,c7 CHARACTER,c8 varchar,c9 CIDR,c11 DATE,c12 FLOAT,c13 DOUBLE PRECISION, - c14 INET,c15 INTEGER,c16 INTERVAL,c17 JSON,c18 JSONB,c21 MACADDR,c22 MONEY, - c23 NUMERIC,c24 OID,c28 REAL,c29 SMALLINT,c30 SMALLSERIAL,c31 SERIAL,c32 TEXT, - c33 TIMESTAMP,c34 TIMESTAMPTZ,c35 TIME, c36 TIMETZ,c37 TSQUERY,c38 TSVECTOR, - c39 TXID_SNAPSHOT,c40 UUID,c41 XML, c42 INT[], c43 FLOAT[]); - CREATE OR REPLACE FUNCTION random_bytea(bytea_length integer) - RETURNS bytea AS $body$ - SELECT decode(string_agg(lpad(to_hex(width_bucket(random(), 0, 1, 256)-1),2,'0') ,''), 'hex') - FROM generate_series(1, $1); - $body$ - LANGUAGE 'sql' - VOLATILE - SET search_path = 'pg_catalog'; - `) - s.NoError(err) - - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_types_bq", - TableNameMapping: map[string]string{"e2e_test.test_types_bq": "test_types_bq"}, - PostgresPort: postgresPort, - Destination: s.bqHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - - TotalSyncFlows: 1, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and execute a transaction touching toast columns - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - /* test inserting various types*/ - _, err = s.pool.Exec(context.Background(), ` - INSERT INTO e2e_test.test_types_bq SELECT 2,2,b'1',b'101', - true,random_bytea(32),'s','test','1.1.10.2'::cidr, - CURRENT_DATE,1.23,1.234,'192.168.1.5'::inet,1, - '5 years 2 months 29 days 1 minute 2 seconds 200 milliseconds 20000 microseconds'::interval, - '{"sai":1}'::json,'{"sai":1}'::jsonb,'08:00:2b:01:02:03'::macaddr, - 1.2,1.23,4::oid,1.23,1,1,1,'test',now(),now(),now()::time,now()::timetz, - 'fat & rat'::tsquery,'a fat cat sat on a mat and ate a fat rat'::tsvector, - txid_current_snapshot(), - '66073c38-b8df-4bdb-bbca-1c97596b8940'::uuid,xmlcomment('hello'), - ARRAY[10299301,2579827], - ARRAY[0.0003, 8902.0092]; - `) - s.NoError(err) - fmt.Println("Executed an insert with all types") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - noNulls, err := s.bqHelper.CheckNull("test_types_bq", []string{"c41", "c1", "c2", "c3", "c4", - "c6", "c39", "c40", "id", "c9", "c11", "c12", "c13", "c14", "c15", "c16", "c17", "c18", - "c21", "c22", "c23", "c24", "c28", "c29", "c30", "c31", "c33", "c34", "c35", "c36", - "c37", "c38", "c7", "c8", "c32", "c42", "c43"}) - if err != nil { - fmt.Println("error %w", err) - } - // Make sure that there are no nulls - s.Equal(noNulls, true) - - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Types_Avro_BQ() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - - CREATE TABLE e2e_test.test_types_avro_bq(id serial PRIMARY KEY,c1 BIGINT,c2 BIT,c3 VARBIT,c4 BOOLEAN, - c6 BYTEA,c7 CHARACTER,c8 varchar,c9 CIDR,c11 DATE,c12 FLOAT,c13 DOUBLE PRECISION, - c14 INET,c15 INTEGER,c16 INTERVAL,c17 JSON,c18 JSONB,c21 MACADDR,c22 MONEY, - c23 NUMERIC,c24 OID,c28 REAL,c29 SMALLINT,c30 SMALLSERIAL,c31 SERIAL,c32 TEXT, - c33 TIMESTAMP,c34 TIMESTAMPTZ,c35 TIME, c36 TIMETZ,c37 TSQUERY,c38 TSVECTOR, - c39 TXID_SNAPSHOT,c40 UUID,c41 XML, c42 INT[], c43 FLOAT[], c44 TEXT[]); - CREATE OR REPLACE FUNCTION random_bytea(bytea_length integer) - RETURNS bytea AS $body$ - SELECT decode(string_agg(lpad(to_hex(width_bucket(random(), 0, 1, 256)-1),2,'0') ,''), 'hex') - FROM generate_series(1, $1); - $body$ - LANGUAGE 'sql' - VOLATILE - SET search_path = 'pg_catalog'; - `) - s.NoError(err) - - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_types_avro_bq", - TableNameMapping: map[string]string{"e2e_test.test_types_avro_bq": "test_types_avro_bq"}, - PostgresPort: postgresPort, - Destination: s.bqHelper.Peer, - CDCSyncMode: protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, - CdcStagingPath: "peerdb_staging", - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - - TotalSyncFlows: 1, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and execute a transaction touching toast columns - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - /* test inserting various types*/ - _, err = s.pool.Exec(context.Background(), ` - INSERT INTO e2e_test.test_types_avro_bq SELECT 2,2,b'1',b'101', - true,random_bytea(32),'s','test','1.1.10.2'::cidr, - CURRENT_DATE,1.23,1.234,'192.168.1.5'::inet,1, - '5 years 2 months 29 days 1 minute 2 seconds 200 milliseconds 20000 microseconds'::interval, - '{"sai":1}'::json,'{"sai":1}'::jsonb,'08:00:2b:01:02:03'::macaddr, - 1.2,1.23,4::oid,1.23,1,1,1,'test',now(),now(),now()::time,now()::timetz, - 'fat & rat'::tsquery,'a fat cat sat on a mat and ate a fat rat'::tsvector, - txid_current_snapshot(), - '66073c38-b8df-4bdb-bbca-1c97596b8940'::uuid,xmlcomment('hello'), - ARRAY[9301,239827], - ARRAY[0.0003, 1039.0034], - ARRAY['hello','bye']; - `) - s.NoError(err) - fmt.Println("Executed an insert with all types") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - noNulls, err := s.bqHelper.CheckNull("test_types_avro_bq", []string{"c41", "c1", "c2", "c3", "c4", - "c6", "c39", "c40", "id", "c9", "c11", "c12", "c13", "c14", "c15", "c16", "c17", "c18", - "c21", "c22", "c23", "c24", "c28", "c29", "c30", "c31", "c33", "c34", "c35", "c36", - "c37", "c38", "c7", "c8", "c32", "c42", "c43"}) - if err != nil { - fmt.Println("error %w", err) - } - // Make sure that there are no nulls - s.Equal(noNulls, true) - - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Simple_Flow_BQ_Avro_CDC() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - CREATE TABLE e2e_test.test_simple_flow_bq_avro_cdc ( - id SERIAL PRIMARY KEY, - key TEXT NOT NULL, - value TEXT NOT NULL - ); - `) - s.NoError(err) - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_simple_flow_bq_avro_cdc", - TableNameMapping: map[string]string{"e2e_test.test_simple_flow_bq_avro_cdc": "test_simple_flow_bq_avro_cdc"}, - PostgresPort: postgresPort, - Destination: s.bqHelper.Peer, - CDCSyncMode: protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, - CdcStagingPath: "peerdb_staging", - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 2, - MaxBatchSize: 100, - } - - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - for i := 0; i < 10; i++ { - testKey := fmt.Sprintf("test_key_%d", i) - testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.pool.Exec(context.Background(), ` - INSERT INTO e2e_test.test_simple_flow_bq_avro_cdc (key, value) VALUES ($1, $2) - `, testKey, testValue) - s.NoError(err) - } - fmt.Println("Inserted 10 rows into the source table") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - count, err := s.bqHelper.CountRows("test_simple_flow_bq") - s.NoError(err) - s.Equal(10, count) - - // TODO: verify that the data is correctly synced to the destination table - // on the bigquery side - - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Multi_Table_BQ() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - CREATE TABLE e2e_test.test1_bq(id serial primary key, c1 int, c2 text); - CREATE TABLE e2e_test.test2_bq(id serial primary key, c1 int, c2 text); - `) - s.NoError(err) - - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_multi_table_bq", - TableNameMapping: map[string]string{"e2e_test.test1_bq": "test1_bq", "e2e_test.test2_bq": "test2_bq"}, - PostgresPort: postgresPort, - Destination: s.bqHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 1, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and execute a transaction touching toast columns - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - /* inserting across multiple tables*/ - _, err = s.pool.Exec(context.Background(), ` - INSERT INTO e2e_test.test1_bq(c1,c2) VALUES (1,'dummy_1'); - INSERT INTO e2e_test.test2_bq(c1,c2) VALUES (-1,'dummy_-1'); - `) - s.NoError(err) - fmt.Println("Executed an insert with all types") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - require.True(s.T(), env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - count1, err := s.bqHelper.CountRows("test1_bq") - s.NoError(err) - count2, err := s.bqHelper.CountRows("test2_bq") - s.NoError(err) - - s.Equal(1, count1) - s.Equal(1, count2) - - env.AssertExpectations(s.T()) -} - -// tests for snowflake - -func (s *E2EPeerFlowTestSuite) Test_Complete_Simple_Flow_SF() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - CREATE TABLE e2e_test.test_simple_flow_sf ( - id SERIAL PRIMARY KEY, - key TEXT NOT NULL, - value TEXT NOT NULL - ); - `) - s.NoError(err) - tableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_simple_flow_sf") - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_complete_single_col_flow_sf", - TableNameMapping: map[string]string{"e2e_test.test_simple_flow_sf": tableName}, - PostgresPort: postgresPort, - Destination: s.sfHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 2, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and then insert 10 rows into the source table - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - // insert 10 rows into the source table - for i := 0; i < 10; i++ { - testKey := fmt.Sprintf("test_key_%d", i) - testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.pool.Exec(context.Background(), ` - INSERT INTO e2e_test.test_simple_flow_sf (key, value) VALUES ($1, $2) - `, testKey, testValue) - s.NoError(err) - } - fmt.Println("Inserted 10 rows into the source table") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - count, err := s.sfHelper.CountRows("test_simple_flow_sf") - s.NoError(err) - s.Equal(10, count) - - // TODO: verify that the data is correctly synced to the destination table - // on the bigquery side - - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Complete_Simple_Flow_SF_Avro_CDC() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - CREATE TABLE e2e_test.test_simple_flow_sf_avro_cdc ( - id SERIAL PRIMARY KEY, - key TEXT NOT NULL, - value TEXT NOT NULL - ); - `) - s.NoError(err) - tableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_simple_flow_sf_avro_cdc") - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_complete_single_col_flow_sf", - TableNameMapping: map[string]string{"e2e_test.test_simple_flow_sf_avro_cdc": tableName}, - PostgresPort: postgresPort, - Destination: s.sfHelper.Peer, - CDCSyncMode: protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 2, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and then insert 10 rows into the source table - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - // insert 10 rows into the source table - for i := 0; i < 10; i++ { - testKey := fmt.Sprintf("test_key_%d", i) - testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.pool.Exec(context.Background(), ` - INSERT INTO e2e_test.test_simple_flow_sf_avro_cdc (key, value) VALUES ($1, $2) - `, testKey, testValue) - s.NoError(err) - } - fmt.Println("Inserted 10 rows into the source table") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - count, err := s.sfHelper.CountRows("test_simple_flow_sf") - s.NoError(err) - s.Equal(10, count) - - // TODO: verify that the data is correctly synced to the destination table - // on the bigquery side - - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Toast_SF() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - - CREATE TABLE e2e_test.test_toast_sf_1 ( - id SERIAL PRIMARY KEY, - t1 text, - t2 text, - k int - );CREATE OR REPLACE FUNCTION random_string( int ) RETURNS TEXT as $$ - SELECT string_agg(substring('0123456789bcdfghjkmnpqrstvwxyz', - round(random() * 30)::integer, 1), '') FROM generate_series(1, $1); - $$ language sql; - `) - s.NoError(err) - - tableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_1") - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_toast_sf_1", - TableNameMapping: map[string]string{"e2e_test.test_toast_sf_1": tableName}, - PostgresPort: postgresPort, - Destination: s.sfHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 1, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and execute a transaction touching toast columns - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - /* - Executing a transaction which - 1. changes both toast column - 2. changes no toast column - 2. changes 1 toast column - */ - _, err = s.pool.Exec(context.Background(), ` - BEGIN; - INSERT INTO e2e_test.test_toast_sf_1(t1,t2,k) SELECT random_string(9000),random_string(9000), - 1 FROM generate_series(1,2); - UPDATE e2e_test.test_toast_sf_1 SET k=102 WHERE id=1; - UPDATE e2e_test.test_toast_sf_1 SET t1='dummy' WHERE id=2; - END; - `) - s.NoError(err) - fmt.Println("Executed a transaction touching toast columns") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - s.compareTableContentsSF("test_toast_sf_1", `id,t1,t2,k`, false) - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Toast_Nochanges_SF() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - - CREATE TABLE e2e_test.test_toast_sf_2 ( - id SERIAL PRIMARY KEY, - t1 text, - t2 text, - k int - );CREATE OR REPLACE FUNCTION random_string( int ) RETURNS TEXT as $$ - SELECT string_agg(substring('0123456789bcdfghjkmnpqrstvwxyz', - round(random() * 30)::integer, 1), '') FROM generate_series(1, $1); - $$ language sql; - `) - s.NoError(err) - - tableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_2") - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_toast_sf_2", - TableNameMapping: map[string]string{"e2e_test.test_toast_sf_2": tableName}, - PostgresPort: postgresPort, - Destination: s.sfHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 1, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and execute a transaction touching toast columns - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - /* transaction updating no rows */ - _, err = s.pool.Exec(context.Background(), ` - BEGIN; - UPDATE e2e_test.test_toast_sf_2 SET k=102 WHERE id=1; - UPDATE e2e_test.test_toast_sf_2 SET t1='dummy' WHERE id=2; - END; - `) - s.NoError(err) - fmt.Println("Executed a transaction touching toast columns") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - s.compareTableContentsSF("test_toast_sf_2", `id,t1,t2,k`, false) - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Toast_Advance_1_SF() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - - CREATE TABLE e2e_test.test_toast_sf_3 ( - id SERIAL PRIMARY KEY, - t1 text, - t2 text, - k int - );CREATE OR REPLACE FUNCTION random_string( int ) RETURNS TEXT as $$ - SELECT string_agg(substring('0123456789bcdfghjkmnpqrstvwxyz', - round(random() * 30)::integer, 1), '') FROM generate_series(1, $1); - $$ language sql; - `) - s.NoError(err) - - tableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_3") - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_toast_sf_3", - TableNameMapping: map[string]string{"e2e_test.test_toast_sf_3": tableName}, - PostgresPort: postgresPort, - Destination: s.sfHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 2, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and execute a transaction touching toast columns - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - //complex transaction with random DMLs on a table with toast columns - _, err = s.pool.Exec(context.Background(), ` - BEGIN; - INSERT INTO e2e_test.test_toast_sf_3(t1,t2,k) SELECT random_string(9000),random_string(9000), - 1 FROM generate_series(1,2); - UPDATE e2e_test.test_toast_sf_3 SET k=102 WHERE id=1; - UPDATE e2e_test.test_toast_sf_3 SET t1='dummy' WHERE id=2; - UPDATE e2e_test.test_toast_sf_3 SET t2='dummy' WHERE id=2; - DELETE FROM e2e_test.test_toast_sf_3 WHERE id=1; - INSERT INTO e2e_test.test_toast_sf_3(t1,t2,k) SELECT random_string(9000),random_string(9000), - 1 FROM generate_series(1,2); - UPDATE e2e_test.test_toast_sf_3 SET k=1 WHERE id=1; - UPDATE e2e_test.test_toast_sf_3 SET t1='dummy1',t2='dummy2' WHERE id=1; - UPDATE e2e_test.test_toast_sf_3 SET t1='dummy3' WHERE id=3; - DELETE FROM e2e_test.test_toast_sf_3 WHERE id=2; - DELETE FROM e2e_test.test_toast_sf_3 WHERE id=3; - DELETE FROM e2e_test.test_toast_sf_3 WHERE id=2; - END; - `) - s.NoError(err) - fmt.Println("Executed a transaction touching toast columns") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - s.compareTableContentsSF("test_toast_sf_3", `id,t1,t2,k`, false) - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Toast_Advance_2_SF() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - - CREATE TABLE e2e_test.test_toast_sf_4 ( - id SERIAL PRIMARY KEY, - t1 text, - k int - );CREATE OR REPLACE FUNCTION random_string( int ) RETURNS TEXT as $$ - SELECT string_agg(substring('0123456789bcdfghjkmnpqrstvwxyz', - round(random() * 30)::integer, 1), '') FROM generate_series(1, $1); - $$ language sql; - `) - s.NoError(err) - - tableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_4") - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_toast_sf_4", - TableNameMapping: map[string]string{"e2e_test.test_toast_sf_4": tableName}, - PostgresPort: postgresPort, - Destination: s.sfHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 1, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and execute a transaction touching toast columns - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - //complex transaction with random DMLs on a table with toast columns - _, err = s.pool.Exec(context.Background(), ` - BEGIN; - INSERT INTO e2e_test.test_toast_sf_4(t1,k) SELECT random_string(9000), - 1 FROM generate_series(1,1); - UPDATE e2e_test.test_toast_sf_4 SET t1=sub.t1 FROM (SELECT random_string(9000) t1 - FROM generate_series(1,1) ) sub WHERE id=1; - UPDATE e2e_test.test_toast_sf_4 SET k=2 WHERE id=1; - UPDATE e2e_test.test_toast_sf_4 SET k=3 WHERE id=1; - UPDATE e2e_test.test_toast_sf_4 SET t1=sub.t1 FROM (SELECT random_string(9000) t1 - FROM generate_series(1,1)) sub WHERE id=1; - UPDATE e2e_test.test_toast_sf_4 SET k=4 WHERE id=1; - END; - `) - s.NoError(err) - fmt.Println("Executed a transaction touching toast columns") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - s.compareTableContentsSF("test_toast_sf_4", `id,t1,k`, false) - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Toast_Advance_3_SF() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - - CREATE TABLE e2e_test.test_toast_sf_5 ( - id SERIAL PRIMARY KEY, - t1 text, - t2 text, - k int - );CREATE OR REPLACE FUNCTION random_string( int ) RETURNS TEXT as $$ - SELECT string_agg(substring('0123456789bcdfghjkmnpqrstvwxyz', - round(random() * 30)::integer, 1), '') FROM generate_series(1, $1); - $$ language sql; - `) - s.NoError(err) - - tableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_5") - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_toast_sf_5", - TableNameMapping: map[string]string{"e2e_test.test_toast_sf_5": tableName}, - PostgresPort: postgresPort, - Destination: s.sfHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 1, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and execute a transaction touching toast columns - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - /* - transaction updating a single row - multiple times with changed/unchanged toast columns - */ - _, err = s.pool.Exec(context.Background(), ` - BEGIN; - INSERT INTO e2e_test.test_toast_sf_5(t1,t2,k) SELECT random_string(9000),random_string(9000), - 1 FROM generate_series(1,1); - UPDATE e2e_test.test_toast_sf_5 SET k=102 WHERE id=1; - UPDATE e2e_test.test_toast_sf_5 SET t1='dummy' WHERE id=1; - UPDATE e2e_test.test_toast_sf_5 SET t2='dummy' WHERE id=1; - END; - `) - s.NoError(err) - fmt.Println("Executed a transaction touching toast columns") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - s.compareTableContentsSF("test_toast_sf_5", `id,t1,t2,k`, false) - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Types_SF() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - - CREATE TABLE e2e_test.test_types_sf(id serial PRIMARY KEY,c1 BIGINT,c2 BIT,c3 VARBIT,c4 BOOLEAN, - c6 BYTEA,c7 CHARACTER,c8 varchar,c9 CIDR,c11 DATE,c12 FLOAT,c13 DOUBLE PRECISION, - c14 INET,c15 INTEGER,c16 INTERVAL,c17 JSON,c18 JSONB,c21 MACADDR,c22 MONEY, - c23 NUMERIC,c24 OID,c28 REAL,c29 SMALLINT,c30 SMALLSERIAL,c31 SERIAL,c32 TEXT, - c33 TIMESTAMP,c34 TIMESTAMPTZ,c35 TIME, c36 TIMETZ,c37 TSQUERY,c38 TSVECTOR, - c39 TXID_SNAPSHOT,c40 UUID,c41 XML); - CREATE OR REPLACE FUNCTION random_bytea(bytea_length integer) - RETURNS bytea AS $body$ - SELECT decode(string_agg(lpad(to_hex(width_bucket(random(), 0, 1, 256)-1),2,'0') ,''), 'hex') - FROM generate_series(1, $1); - $body$ - LANGUAGE 'sql' - VOLATILE - SET search_path = 'pg_catalog'; - `) - s.NoError(err) - - tableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_types_sf") - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_types_sf", - TableNameMapping: map[string]string{"e2e_test.test_types_sf": tableName}, - PostgresPort: postgresPort, - Destination: s.sfHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 1, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and execute a transaction touching toast columns - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - /* test inserting various types*/ - _, err = s.pool.Exec(context.Background(), ` - INSERT INTO e2e_test.test_types_sf SELECT 2,2,b'1',b'101', - true,random_bytea(32),'s','test','1.1.10.2'::cidr, - CURRENT_DATE,1.23,1.234,'192.168.1.5'::inet,1, - '5 years 2 months 29 days 1 minute 2 seconds 200 milliseconds 20000 microseconds'::interval, - '{"sai":1}'::json,'{"sai":1}'::jsonb,'08:00:2b:01:02:03'::macaddr, - 1.2,1.23,4::oid,1.23,1,1,1,'test',now(),now(),now()::time,now()::timetz, - 'fat & rat'::tsquery,'a fat cat sat on a mat and ate a fat rat'::tsvector, - txid_current_snapshot(), - '66073c38-b8df-4bdb-bbca-1c97596b8940'::uuid,xmlcomment('hello'); - `) - s.NoError(err) - fmt.Println("Executed an insert with all types") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - noNulls, err := s.sfHelper.CheckNull("test_types_sf", []string{"c41", "c1", "c2", "c3", "c4", - "c6", "c39", "c40", "id", "c9", "c11", "c12", "c13", "c14", "c15", "c16", "c17", "c18", - "c21", "c22", "c23", "c24", "c28", "c29", "c30", "c31", "c33", "c34", "c35", "c36", - "c37", "c38", "c7", "c8", "c32"}) - if err != nil { - fmt.Println("error %w", err) - } - // Make sure that there are no nulls - s.Equal(noNulls, true) - - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Types_SF_Avro_CDC() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - - CREATE TABLE e2e_test.test_types_sf_avro_cdc(id serial PRIMARY KEY,c1 BIGINT,c2 BIT,c3 VARBIT,c4 BOOLEAN, - c6 BYTEA,c7 CHARACTER,c8 varchar,c9 CIDR,c11 DATE,c12 FLOAT,c13 DOUBLE PRECISION, - c14 INET,c15 INTEGER,c16 INTERVAL,c17 JSON,c18 JSONB,c21 MACADDR,c22 MONEY, - c23 NUMERIC,c24 OID,c28 REAL,c29 SMALLINT,c30 SMALLSERIAL,c31 SERIAL,c32 TEXT, - c33 TIMESTAMP,c34 TIMESTAMPTZ,c35 TIME, c36 TIMETZ,c37 TSQUERY,c38 TSVECTOR, - c39 TXID_SNAPSHOT,c40 UUID,c41 XML); - CREATE OR REPLACE FUNCTION random_bytea(bytea_length integer) - RETURNS bytea AS $body$ - SELECT decode(string_agg(lpad(to_hex(width_bucket(random(), 0, 1, 256)-1),2,'0') ,''), 'hex') - FROM generate_series(1, $1); - $body$ - LANGUAGE 'sql' - VOLATILE - SET search_path = 'pg_catalog'; - `) - s.NoError(err) - - tableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_types_sf_avro_cdc") - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_types_sf", - TableNameMapping: map[string]string{"e2e_test.test_types_sf_avro_cdc": tableName}, - PostgresPort: postgresPort, - Destination: s.sfHelper.Peer, - CDCSyncMode: protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 1, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and execute a transaction touching toast columns - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - /* test inserting various types*/ - _, err = s.pool.Exec(context.Background(), ` - INSERT INTO e2e_test.test_types_sf_avro_cdc SELECT 2,2,b'1',b'101', - true,random_bytea(32),'s','test','1.1.10.2'::cidr, - CURRENT_DATE,1.23,1.234,'192.168.1.5'::inet,1, - '5 years 2 months 29 days 1 minute 2 seconds 200 milliseconds 20000 microseconds'::interval, - '{"sai":1}'::json,'{"sai":1}'::jsonb,'08:00:2b:01:02:03'::macaddr, - 1.2,1.23,4::oid,1.23,1,1,1,'test',now(),now(),now()::time,now()::timetz, - 'fat & rat'::tsquery,'a fat cat sat on a mat and ate a fat rat'::tsvector, - txid_current_snapshot(), - '66073c38-b8df-4bdb-bbca-1c97596b8940'::uuid,xmlcomment('hello'); - `) - s.NoError(err) - fmt.Println("Executed an insert with all types") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - // allow only continue as new error - s.Error(err) - s.Contains(err.Error(), "continue as new") - - noNulls, err := s.sfHelper.CheckNull("test_types_sf_avro_cdc", []string{"c41", "c1", "c2", "c3", "c4", - "c6", "c39", "c40", "id", "c9", "c11", "c12", "c13", "c14", "c15", "c16", "c17", "c18", - "c21", "c22", "c23", "c24", "c28", "c29", "c30", "c31", "c33", "c34", "c35", "c36", - "c37", "c38", "c7", "c8", "c32"}) - if err != nil { - fmt.Println("error %w", err) - } - // Make sure that there are no nulls - s.Equal(noNulls, true) - - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Multi_Table_SF() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - _, err := s.pool.Exec(context.Background(), ` - CREATE TABLE e2e_test.test1_sf(id serial primary key, c1 int, c2 text); - CREATE TABLE e2e_test.test2_sf(id serial primary key, c1 int, c2 text); - `) - s.NoError(err) - - table1 := fmt.Sprintf(s.sfHelper.testSchemaName + ".test1_sf") - table2 := fmt.Sprintf(s.sfHelper.testSchemaName + ".test2_sf") - connectionGen := FlowConnectionGenerationConfig{ - FlowJobName: "test_multi_table_sf", - TableNameMapping: map[string]string{"e2e_test.test1_sf": table1, "e2e_test.test2_sf": table2}, - PostgresPort: postgresPort, - Destination: s.sfHelper.Peer, - } - - flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() - s.NoError(err) - - limits := peerflow.PeerFlowLimits{ - TotalSyncFlows: 1, - MaxBatchSize: 100, - } - - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and execute a transaction touching toast columns - go func() { - s.SetupPeerFlowStatusQuery(env, connectionGen) - /* inserting across multiple tables*/ - _, err = s.pool.Exec(context.Background(), ` - INSERT INTO e2e_test.test1_sf(c1,c2) VALUES (1,'dummy_1'); - INSERT INTO e2e_test.test2_sf(c1,c2) VALUES (-1,'dummy_-1'); - `) - s.NoError(err) - fmt.Println("Executed an insert with all types") - }() - - env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - count1, err := s.sfHelper.CountRows("test1_sf") - s.NoError(err) - count2, err := s.sfHelper.CountRows("test2_sf") - s.NoError(err) - - s.Equal(1, count1) - s.Equal(1, count2) - - env.AssertExpectations(s.T()) -} diff --git a/flow/e2e/postgres/qrep_flow_pg_test.go b/flow/e2e/postgres/qrep_flow_pg_test.go new file mode 100644 index 0000000000..241659c229 --- /dev/null +++ b/flow/e2e/postgres/qrep_flow_pg_test.go @@ -0,0 +1,147 @@ +package e2e_postgres + +import ( + "context" + "fmt" + "testing" + + "github.com/PeerDB-io/peer-flow/e2e" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/jackc/pgx/v5/pgxpool" + "github.com/joho/godotenv" + log "github.com/sirupsen/logrus" + "github.com/stretchr/testify/suite" + "go.temporal.io/sdk/testsuite" +) + +const postgresSuffix = "postgres" + +type PeerFlowE2ETestSuitePG struct { + suite.Suite + testsuite.WorkflowTestSuite + + pool *pgxpool.Pool +} + +func TestPeerFlowE2ETestSuitePG(t *testing.T) { + suite.Run(t, new(PeerFlowE2ETestSuitePG)) +} + +// Implement SetupAllSuite interface to setup the test suite +func (s *PeerFlowE2ETestSuitePG) SetupSuite() { + err := godotenv.Load() + if err != nil { + // it's okay if the .env file is not present + // we will use the default values + log.Infof("Unable to load .env file, using default values from env") + } + + log.SetReportCaller(true) + + pool, err := e2e.SetupPostgres(postgresSuffix) + if err != nil { + s.Fail("failed to setup postgres", err) + } + s.pool = pool +} + +// Implement TearDownAllSuite interface to tear down the test suite +func (s *PeerFlowE2ETestSuitePG) TearDownSuite() { + err := e2e.TearDownPostgres(s.pool, postgresSuffix) + if err != nil { + s.Fail("failed to drop Postgres schema", err) + } +} + +func (s *PeerFlowE2ETestSuitePG) setupSourceTable(tableName string, rowCount int) { + err := e2e.CreateSourceTableQRep(s.pool, postgresSuffix, tableName) + s.NoError(err) + err = e2e.PopulateSourceTable(s.pool, postgresSuffix, tableName, rowCount) + s.NoError(err) +} + +func (s *PeerFlowE2ETestSuitePG) comparePGTables(srcSchemaQualified, dstSchemaQualified string) error { + // Execute the two EXCEPT queries + err := s.compareQuery(srcSchemaQualified, dstSchemaQualified) + if err != nil { + return err + } + + err = s.compareQuery(dstSchemaQualified, srcSchemaQualified) + if err != nil { + return err + } + + // If no error is returned, then the contents of the two tables are the same + return nil +} + +func (s *PeerFlowE2ETestSuitePG) compareQuery(schema1, schema2 string) error { + query := fmt.Sprintf("SELECT * FROM %s EXCEPT SELECT * FROM %s", schema1, schema2) + rows, _ := s.pool.Query(context.Background(), query) + + defer rows.Close() + for rows.Next() { + values, err := rows.Values() + if err != nil { + return err + } + + columns := rows.FieldDescriptions() + + for i, value := range values { + fmt.Printf("%s: %v\n", columns[i].Name, value) + } + fmt.Println("---") + } + + return rows.Err() +} + +func (s *PeerFlowE2ETestSuitePG) Test_Complete_QRep_Flow_Multi_Insert_PG() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + numRows := 10 + + srcTable := "test_qrep_flow_avro_pg_1" + s.setupSourceTable(srcTable, numRows) + + dstTable := "test_qrep_flow_avro_pg_2" + e2e.CreateSourceTableQRep(s.pool, postgresSuffix, dstTable) // the name is misleading, but this is the destination table + + srcSchemaQualified := fmt.Sprintf("%s_%s.%s", "e2e_test", postgresSuffix, srcTable) + dstSchemaQualified := fmt.Sprintf("%s_%s.%s", "e2e_test", postgresSuffix, dstTable) + + query := fmt.Sprintf("SELECT * FROM e2e_test_%s.%s WHERE updated_at BETWEEN {{.start}} AND {{.end}}", + postgresSuffix, srcTable) + + postgresPeer := e2e.GeneratePostgresPeer(e2e.PostgresPort) + + qrepConfig, err := e2e.CreateQRepWorkflowConfig( + "test_qrep_flow_avro_pg", + srcSchemaQualified, + dstSchemaQualified, + query, + protos.QRepSyncMode_QREP_SYNC_MODE_MULTI_INSERT, + postgresPeer, + "", + ) + s.NoError(err) + + e2e.RunQrepFlowWorkflow(env, qrepConfig) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + + // assert that error contains "invalid connection configs" + err = env.GetWorkflowError() + s.NoError(err) + + err = s.comparePGTables(srcSchemaQualified, dstSchemaQualified) + if err != nil { + s.FailNow(err.Error()) + } + + env.AssertExpectations(s.T()) +} diff --git a/flow/e2e/qrep_flow_test.go b/flow/e2e/qrep_flow_test.go deleted file mode 100644 index c79157bcfd..0000000000 --- a/flow/e2e/qrep_flow_test.go +++ /dev/null @@ -1,625 +0,0 @@ -package e2e - -import ( - "context" - "encoding/json" - "fmt" - "strings" - - connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/model" - "github.com/PeerDB-io/peer-flow/model/qvalue" - peerflow "github.com/PeerDB-io/peer-flow/workflows" - "github.com/google/uuid" - "github.com/stretchr/testify/require" - "go.temporal.io/sdk/testsuite" -) - -func (s *E2EPeerFlowTestSuite) createSourceTable(tableName string) { - tblFields := []string{ - "id UUID NOT NULL PRIMARY KEY", - "card_id UUID", - `"from" TIMESTAMP NOT NULL`, - "price NUMERIC", - "created_at TIMESTAMP NOT NULL", - "updated_at TIMESTAMP NOT NULL", - "transaction_hash BYTEA", - "ownerable_type VARCHAR", - "ownerable_id UUID", - "user_nonce INTEGER", - "transfer_type INTEGER DEFAULT 0 NOT NULL", - "blockchain INTEGER NOT NULL", - "deal_type VARCHAR", - "deal_id UUID", - "ethereum_transaction_id UUID", - "ignore_price BOOLEAN DEFAULT false", - "card_eth_value DOUBLE PRECISION", - "paid_eth_price DOUBLE PRECISION", - "card_bought_notified BOOLEAN DEFAULT false NOT NULL", - "address NUMERIC", - "account_id UUID", - "asset_id NUMERIC NOT NULL", - "status INTEGER", - "transaction_id UUID", - "settled_at TIMESTAMP", - "reference_id VARCHAR", - "settle_at TIMESTAMP", - "settlement_delay_reason INTEGER", - "f1 text[]", - "f2 bigint[]", - "f3 int[]", - "f4 varchar[]", - "f5 jsonb", - "f6 jsonb", - "f7 jsonb", - "f8 smallint", - } - - tblFieldStr := strings.Join(tblFields, ",") - - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` - CREATE TABLE e2e_test.%s ( - %s - );`, tableName, tblFieldStr)) - s.NoError(err) - - fmt.Printf("created table on postgres: e2e_test.%s\n", tableName) -} - -func (s *E2EPeerFlowTestSuite) populateSourceTable(tableName string, rowCount int) { - var ids []string - var rows []string - for i := 0; i < rowCount-1; i++ { - id := uuid.New().String() - ids = append(ids, id) - row := fmt.Sprintf(` - ( - '%s', '%s', CURRENT_TIMESTAMP, 3.86487206688919, CURRENT_TIMESTAMP, - CURRENT_TIMESTAMP, E'\\\\xDEADBEEF', 'type1', '%s', - 1, 0, 1, 'dealType1', - '%s', '%s', false, 1.2345, - 1.2345, false, 12345, '%s', - 12345, 1, '%s', CURRENT_TIMESTAMP, 'refID', - CURRENT_TIMESTAMP, 1, ARRAY['text1', 'text2'], ARRAY[123, 456], ARRAY[789, 012], - ARRAY['varchar1', 'varchar2'], '{"key": 8.5}', - '[{"key1": "value1", "key2": "value2", "key3": "value3"}]', - '{"key": "value"}', 15 - )`, - id, uuid.New().String(), uuid.New().String(), - uuid.New().String(), uuid.New().String(), uuid.New().String(), uuid.New().String()) - rows = append(rows, row) - } - - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` - INSERT INTO e2e_test.%s ( - id, card_id, "from", price, created_at, - updated_at, transaction_hash, ownerable_type, ownerable_id, - user_nonce, transfer_type, blockchain, deal_type, - deal_id, ethereum_transaction_id, ignore_price, card_eth_value, - paid_eth_price, card_bought_notified, address, account_id, - asset_id, status, transaction_id, settled_at, reference_id, - settle_at, settlement_delay_reason, f1, f2, f3, f4, f5, f6, f7, f8 - ) VALUES %s; - `, tableName, strings.Join(rows, ","))) - require.NoError(s.T(), err) - - // add a row where all the nullable fields are null - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` - INSERT INTO e2e_test.%s ( - id, "from", created_at, updated_at, - transfer_type, blockchain, card_bought_notified, asset_id - ) VALUES ( - '%s', CURRENT_TIMESTAMP, CURRENT_TIMESTAMP, CURRENT_TIMESTAMP, - 0, 1, false, 12345 - ); - `, tableName, uuid.New().String())) - require.NoError(s.T(), err) - - // generate a 20 MB json and update id[0]'s col f5 to it - v := s.generate20MBJson() - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` - UPDATE e2e_test.%s SET f5 = '%s' WHERE id = '%s'; - `, tableName, v, ids[0])) - require.NoError(s.T(), err) -} - -func (s *E2EPeerFlowTestSuite) generate20MBJson() []byte { - xn := make(map[string]interface{}) - for i := 0; i < 215000; i++ { - xn[uuid.New().String()] = uuid.New().String() - } - - v, err := json.Marshal(xn) - require.NoError(s.T(), err) - - return v -} - -func (s *E2EPeerFlowTestSuite) setupSourceTable(tableName string, rowCount int) { - s.createSourceTable(tableName) - s.populateSourceTable(tableName, rowCount) -} - -func getOwnersSchema() *model.QRecordSchema { - return &model.QRecordSchema{ - Fields: []*model.QField{ - {Name: "id", Type: qvalue.QValueKindString, Nullable: true}, - {Name: "card_id", Type: qvalue.QValueKindString, Nullable: true}, - {Name: "from", Type: qvalue.QValueKindTimestamp, Nullable: true}, - {Name: "price", Type: qvalue.QValueKindNumeric, Nullable: true}, - {Name: "created_at", Type: qvalue.QValueKindTimestamp, Nullable: true}, - {Name: "updated_at", Type: qvalue.QValueKindTimestamp, Nullable: true}, - {Name: "transaction_hash", Type: qvalue.QValueKindBytes, Nullable: true}, - {Name: "ownerable_type", Type: qvalue.QValueKindString, Nullable: true}, - {Name: "ownerable_id", Type: qvalue.QValueKindString, Nullable: true}, - {Name: "user_nonce", Type: qvalue.QValueKindInt64, Nullable: true}, - {Name: "transfer_type", Type: qvalue.QValueKindInt64, Nullable: true}, - {Name: "blockchain", Type: qvalue.QValueKindInt64, Nullable: true}, - {Name: "deal_type", Type: qvalue.QValueKindString, Nullable: true}, - {Name: "deal_id", Type: qvalue.QValueKindString, Nullable: true}, - {Name: "ethereum_transaction_id", Type: qvalue.QValueKindString, Nullable: true}, - {Name: "ignore_price", Type: qvalue.QValueKindBoolean, Nullable: true}, - {Name: "card_eth_value", Type: qvalue.QValueKindFloat64, Nullable: true}, - {Name: "paid_eth_price", Type: qvalue.QValueKindFloat64, Nullable: true}, - {Name: "card_bought_notified", Type: qvalue.QValueKindBoolean, Nullable: true}, - {Name: "address", Type: qvalue.QValueKindNumeric, Nullable: true}, - {Name: "account_id", Type: qvalue.QValueKindString, Nullable: true}, - {Name: "asset_id", Type: qvalue.QValueKindNumeric, Nullable: true}, - {Name: "status", Type: qvalue.QValueKindInt64, Nullable: true}, - {Name: "transaction_id", Type: qvalue.QValueKindString, Nullable: true}, - {Name: "settled_at", Type: qvalue.QValueKindTimestamp, Nullable: true}, - {Name: "reference_id", Type: qvalue.QValueKindString, Nullable: true}, - {Name: "settle_at", Type: qvalue.QValueKindTimestamp, Nullable: true}, - {Name: "settlement_delay_reason", Type: qvalue.QValueKindInt64, Nullable: true}, - {Name: "f1", Type: qvalue.QValueKindArrayString, Nullable: true}, - {Name: "f2", Type: qvalue.QValueKindArrayInt64, Nullable: true}, - {Name: "f3", Type: qvalue.QValueKindArrayInt32, Nullable: true}, - {Name: "f4", Type: qvalue.QValueKindArrayString, Nullable: true}, - {Name: "f5", Type: qvalue.QValueKindJSON, Nullable: true}, - {Name: "f6", Type: qvalue.QValueKindJSON, Nullable: true}, - {Name: "f7", Type: qvalue.QValueKindJSON, Nullable: true}, - {Name: "f8", Type: qvalue.QValueKindInt16, Nullable: true}, - }, - } -} - -func getOwnersSelectorString() string { - schema := getOwnersSchema() - var fields []string - for _, field := range schema.Fields { - // append quoted field name - fields = append(fields, fmt.Sprintf(`"%s"`, field.Name)) - } - return strings.Join(fields, ",") -} - -func (s *E2EPeerFlowTestSuite) setupBQDestinationTable(dstTable string) { - schema := getOwnersSchema() - err := s.bqHelper.CreateTable(dstTable, schema) - - // fail if table creation fails - require.NoError(s.T(), err) - - fmt.Printf("created table on bigquery: %s.%s. %v\n", s.bqHelper.Config.DatasetId, dstTable, err) -} - -func (s *E2EPeerFlowTestSuite) setupSFDestinationTable(dstTable string) { - schema := getOwnersSchema() - err := s.sfHelper.CreateTable(dstTable, schema) - - // fail if table creation fails - if err != nil { - s.FailNow("unable to create table on snowflake", err) - } - - fmt.Printf("created table on snowflake: %s.%s. %v\n", s.sfHelper.testSchemaName, dstTable, err) -} - -func (s *E2EPeerFlowTestSuite) createQRepWorkflowConfig( - flowJobName string, - sourceTable string, - dstTable string, - query string, - syncMode protos.QRepSyncMode, - dest *protos.Peer, - stagingPath string, -) *protos.QRepConfig { - connectionGen := QRepFlowConnectionGenerationConfig{ - FlowJobName: flowJobName, - WatermarkTable: sourceTable, - DestinationTableIdentifier: dstTable, - PostgresPort: postgresPort, - Destination: dest, - StagingPath: stagingPath, - } - - watermark := "updated_at" - - qrepConfig, err := connectionGen.GenerateQRepConfig(query, watermark, syncMode) - s.NoError(err) - - qrepConfig.InitialCopyOnly = true - - return qrepConfig -} - -func (s *E2EPeerFlowTestSuite) compareTableContentsBQ(tableName string, colsString string) { - // read rows from source table - pgQueryExecutor := connpostgres.NewQRepQueryExecutor(s.pool, context.Background(), "testflow", "testpart") - pgQueryExecutor.SetTestEnv(true) - - pgRows, err := pgQueryExecutor.ExecuteAndProcessQuery( - fmt.Sprintf("SELECT %s FROM e2e_test.%s ORDER BY id", colsString, tableName), - ) - s.NoError(err) - - // read rows from destination table - qualifiedTableName := fmt.Sprintf("`%s.%s`", s.bqHelper.Config.DatasetId, tableName) - bqRows, err := s.bqHelper.ExecuteAndProcessQuery( - fmt.Sprintf("SELECT %s FROM %s ORDER BY id", colsString, qualifiedTableName), - ) - s.NoError(err) - - s.True(pgRows.Equals(bqRows), "rows from source and destination tables are not equal") -} - -func (s *E2EPeerFlowTestSuite) compareTableContentsSF(tableName string, selector string, caseSensitive bool) { - // read rows from source table - pgQueryExecutor := connpostgres.NewQRepQueryExecutor(s.pool, context.Background(), "testflow", "testpart") - pgQueryExecutor.SetTestEnv(true) - pgRows, err := pgQueryExecutor.ExecuteAndProcessQuery( - fmt.Sprintf("SELECT %s FROM e2e_test.%s ORDER BY id", selector, tableName), - ) - require.NoError(s.T(), err) - - // read rows from destination table - qualifiedTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, tableName) - var sfSelQuery string - if caseSensitive { - sfSelQuery = fmt.Sprintf(`SELECT %s FROM %s ORDER BY "id"`, selector, qualifiedTableName) - } else { - sfSelQuery = fmt.Sprintf(`SELECT %s FROM %s ORDER BY id`, selector, qualifiedTableName) - } - fmt.Printf("running query on snowflake: %s\n", sfSelQuery) - - // sleep for 1 min for debugging - // time.Sleep(1 * time.Minute) - - sfRows, err := s.sfHelper.ExecuteAndProcessQuery(sfSelQuery) - require.NoError(s.T(), err) - - s.True(pgRows.Equals(sfRows), "rows from source and destination tables are not equal") -} - -func (s *E2EPeerFlowTestSuite) comparePGTables(srcSchemaQualified, dstSchemaQualified string) error { - // Execute the two EXCEPT queries - err := s.compareQuery(srcSchemaQualified, dstSchemaQualified) - if err != nil { - return err - } - - err = s.compareQuery(dstSchemaQualified, srcSchemaQualified) - if err != nil { - return err - } - - // If no error is returned, then the contents of the two tables are the same - return nil -} - -func (s *E2EPeerFlowTestSuite) compareQuery(schema1, schema2 string) error { - query := fmt.Sprintf("SELECT * FROM %s EXCEPT SELECT * FROM %s", schema1, schema2) - rows, _ := s.pool.Query(context.Background(), query) - - defer rows.Close() - for rows.Next() { - values, err := rows.Values() - if err != nil { - return err - } - - columns := rows.FieldDescriptions() - - for i, value := range values { - fmt.Printf("%s: %v\n", columns[i].Name, value) - } - fmt.Println("---") - } - - return rows.Err() -} - -// NOTE: Disabled due to large JSON tests being added: https://github.com/PeerDB-io/peerdb/issues/309 - -// Test_Complete_QRep_Flow tests a complete flow with data in the source table. -// The test inserts 10 rows into the source table and verifies that the data is -// // correctly synced to the destination table this runs a QRep Flow. -// func (s *E2EPeerFlowTestSuite) Test_Complete_QRep_Flow_Multi_Insert() { -// env := s.NewTestWorkflowEnvironment() -// registerWorkflowsAndActivities(env) - -// numRows := 10 - -// tblName := "test_qrep_flow_multi_insert" -// s.setupSourceTable(tblName, numRows) -// s.setupBQDestinationTable(tblName) - -// query := fmt.Sprintf("SELECT * FROM e2e_test.%s WHERE updated_at BETWEEN {{.start}} AND {{.end}}", tblName) - -// qrepConfig := s.createQRepWorkflowConfig("test_qrep_flow_mi", -// "e2e_test."+tblName, -// tblName, -// query, -// protos.QRepSyncMode_QREP_SYNC_MODE_MULTI_INSERT, -// s.bqHelper.Peer) -// runQrepFlowWorkflow(env, qrepConfig) - -// // Verify workflow completes without error -// s.True(env.IsWorkflowCompleted()) - -// // assert that error contains "invalid connection configs" -// err := env.GetWorkflowError() -// s.NoError(err) - -// count, err := s.bqHelper.CountRows(tblName) -// s.NoError(err) - -// s.Equal(numRows, count) - -// env.AssertExpectations(s.T()) -// } - -func (s *E2EPeerFlowTestSuite) Test_Complete_QRep_Flow_Avro() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - numRows := 10 - - tblName := "test_qrep_flow_avro" - s.setupSourceTable(tblName, numRows) - s.setupBQDestinationTable(tblName) - - query := fmt.Sprintf("SELECT * FROM e2e_test.%s WHERE updated_at BETWEEN {{.start}} AND {{.end}}", tblName) - - qrepConfig := s.createQRepWorkflowConfig( - "test_qrep_flow_avro", - "e2e_test."+tblName, - tblName, - query, - protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, - s.bqHelper.Peer, - "peerdb_staging") - runQrepFlowWorkflow(env, qrepConfig) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - - // assert that error contains "invalid connection configs" - err := env.GetWorkflowError() - s.NoError(err) - - s.compareTableContentsBQ(tblName, "*") - - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Complete_QRep_Flow_Avro_SF() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - numRows := 10 - - tblName := "test_qrep_flow_avro_sf" - s.setupSourceTable(tblName, numRows) - s.setupSFDestinationTable(tblName) - - dstSchemaQualified := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, tblName) - - query := fmt.Sprintf("SELECT * FROM e2e_test.%s WHERE updated_at BETWEEN {{.start}} AND {{.end}}", tblName) - - qrepConfig := s.createQRepWorkflowConfig( - "test_qrep_flow_avro_Sf", - "e2e_test."+tblName, - dstSchemaQualified, - query, - protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, - s.sfHelper.Peer, - "", - ) - - runQrepFlowWorkflow(env, qrepConfig) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - - // assert that error contains "invalid connection configs" - err := env.GetWorkflowError() - s.NoError(err) - - sel := getOwnersSelectorString() - s.compareTableContentsSF(tblName, sel, true) - - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Complete_QRep_Flow_Avro_SF_Upsert_Simple() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - numRows := 10 - - tblName := "test_qrep_flow_avro_sf_ups" - s.setupSourceTable(tblName, numRows) - s.setupSFDestinationTable(tblName) - - dstSchemaQualified := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, tblName) - - query := fmt.Sprintf("SELECT * FROM e2e_test.%s WHERE updated_at >= {{.start}} AND updated_at < {{.end}}", tblName) - - qrepConfig := s.createQRepWorkflowConfig( - "test_qrep_flow_avro_Sf", - "e2e_test."+tblName, - dstSchemaQualified, - query, - protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, - s.sfHelper.Peer, - "", - ) - qrepConfig.WriteMode = &protos.QRepWriteMode{ - WriteType: protos.QRepWriteType_QREP_WRITE_MODE_UPSERT, - UpsertKeyColumns: []string{"id"}, - } - - runQrepFlowWorkflow(env, qrepConfig) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - - // assert that error contains "invalid connection configs" - err := env.GetWorkflowError() - s.NoError(err) - - sel := getOwnersSelectorString() - s.compareTableContentsSF(tblName, sel, true) - - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Complete_QRep_Flow_Multi_Insert_PG() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - numRows := 10 - - srcTable := "test_qrep_flow_avro_pg_1" - s.setupSourceTable(srcTable, numRows) - - dstTable := "test_qrep_flow_avro_pg_2" - s.createSourceTable(dstTable) // the name is misleading, but this is the destination table - - srcSchemaQualified := fmt.Sprintf("%s.%s", "e2e_test", srcTable) - dstSchemaQualified := fmt.Sprintf("%s.%s", "e2e_test", dstTable) - - query := fmt.Sprintf("SELECT * FROM e2e_test.%s WHERE updated_at BETWEEN {{.start}} AND {{.end}}", srcTable) - - postgresPeer := GeneratePostgresPeer(postgresPort) - - qrepConfig := s.createQRepWorkflowConfig( - "test_qrep_flow_avro_pg", - srcSchemaQualified, - dstSchemaQualified, - query, - protos.QRepSyncMode_QREP_SYNC_MODE_MULTI_INSERT, - postgresPeer, - "", - ) - - runQrepFlowWorkflow(env, qrepConfig) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - - // assert that error contains "invalid connection configs" - err := env.GetWorkflowError() - s.NoError(err) - - err = s.comparePGTables(srcSchemaQualified, dstSchemaQualified) - if err != nil { - s.FailNow(err.Error()) - } - - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Complete_QRep_Flow_Avro_SF_S3() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - numRows := 10 - - tblName := "test_qrep_flow_avro_sf_s3" - s.setupSourceTable(tblName, numRows) - s.setupSFDestinationTable(tblName) - - dstSchemaQualified := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, tblName) - - query := fmt.Sprintf("SELECT * FROM e2e_test.%s WHERE updated_at >= {{.start}} AND updated_at < {{.end}}", tblName) - - qrepConfig := s.createQRepWorkflowConfig( - "test_qrep_flow_avro_sf", - "e2e_test."+tblName, - dstSchemaQualified, - query, - protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, - s.sfHelper.Peer, - "", - ) - qrepConfig.StagingPath = fmt.Sprintf("s3://peerdb-test-bucket/avro/%s", uuid.New()) - - runQrepFlowWorkflow(env, qrepConfig) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - - // assert that error contains "invalid connection configs" - err := env.GetWorkflowError() - s.NoError(err) - - sel := getOwnersSelectorString() - s.compareTableContentsSF(tblName, sel, true) - - env.AssertExpectations(s.T()) -} - -func (s *E2EPeerFlowTestSuite) Test_Complete_QRep_Flow_Avro_SF_S3_Integration() { - env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) - - numRows := 10 - - tblName := "test_qrep_flow_avro_sf_s3_int" - s.setupSourceTable(tblName, numRows) - s.setupSFDestinationTable(tblName) - - dstSchemaQualified := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, tblName) - - query := fmt.Sprintf("SELECT * FROM e2e_test.%s WHERE updated_at >= {{.start}} AND updated_at < {{.end}}", tblName) - - sfPeer := s.sfHelper.Peer - sfPeer.GetSnowflakeConfig().S3Integration = "peerdb_s3_integration" - - qrepConfig := s.createQRepWorkflowConfig( - "test_qrep_flow_avro_sf_int", - "e2e_test."+tblName, - dstSchemaQualified, - query, - protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, - sfPeer, - "", - ) - qrepConfig.StagingPath = fmt.Sprintf("s3://peerdb-test-bucket/avro/%s", uuid.New()) - - runQrepFlowWorkflow(env, qrepConfig) - - // Verify workflow completes without error - s.True(env.IsWorkflowCompleted()) - - // assert that error contains "invalid connection configs" - err := env.GetWorkflowError() - s.NoError(err) - - sel := getOwnersSelectorString() - s.compareTableContentsSF(tblName, sel, true) - - env.AssertExpectations(s.T()) -} - -func runQrepFlowWorkflow(env *testsuite.TestWorkflowEnvironment, config *protos.QRepConfig) { - lastPartition := &protos.QRepPartition{ - PartitionId: "not-applicable-partition", - Range: nil, - } - numPartitionsProcessed := 0 - env.ExecuteWorkflow(peerflow.QRepFlowWorkflow, config, lastPartition, numPartitionsProcessed) -} diff --git a/flow/e2e/s3/qrep_flow_s3_test.go b/flow/e2e/s3/qrep_flow_s3_test.go new file mode 100644 index 0000000000..e47d235922 --- /dev/null +++ b/flow/e2e/s3/qrep_flow_s3_test.go @@ -0,0 +1,183 @@ +package e2e_s3 + +import ( + "context" + "fmt" + "testing" + "time" + + "github.com/PeerDB-io/peer-flow/e2e" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/jackc/pgx/v5/pgxpool" + "github.com/joho/godotenv" + log "github.com/sirupsen/logrus" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + "go.temporal.io/sdk/testsuite" +) + +const s3Suffix = "s3" + +type PeerFlowE2ETestSuiteS3 struct { + suite.Suite + testsuite.WorkflowTestSuite + + pool *pgxpool.Pool + s3Helper *S3TestHelper +} + +func TestPeerFlowE2ETestSuiteS3(t *testing.T) { + suite.Run(t, new(PeerFlowE2ETestSuiteS3)) +} + +func (s *PeerFlowE2ETestSuiteS3) setupSourceTable(tableName string, rowCount int) { + err := e2e.CreateSourceTableQRep(s.pool, s3Suffix, tableName) + s.NoError(err) + err = e2e.PopulateSourceTable(s.pool, s3Suffix, tableName, rowCount) + s.NoError(err) +} + +func (s *PeerFlowE2ETestSuiteS3) setupS3() error { + helper, err := NewS3TestHelper() + if err != nil { + return err + } + + s.s3Helper = helper + return nil +} + +func (s *PeerFlowE2ETestSuiteS3) SetupSuite() { + err := godotenv.Load() + if err != nil { + // it's okay if the .env file is not present + // we will use the default values + log.Infof("Unable to load .env file, using default values from env") + } + + log.SetReportCaller(true) + + pool, err := e2e.SetupPostgres(s3Suffix) + if err != nil { + s.Fail("failed to setup postgres", err) + } + s.pool = pool + + err = s.setupS3() + if err != nil { + s.Fail("failed to setup S3", err) + } +} + +// Implement TearDownAllSuite interface to tear down the test suite +func (s *PeerFlowE2ETestSuiteS3) TearDownSuite() { + err := e2e.TearDownPostgres(s.pool, s3Suffix) + if err != nil { + s.Fail("failed to drop Postgres schema", err) + } + + if s.s3Helper != nil { + err = s.s3Helper.CleanUp() + if err != nil { + s.Fail("failed to clean up s3", err) + } + } +} + +func (s *PeerFlowE2ETestSuiteS3) Test_Complete_QRep_Flow_S3() { + if s.s3Helper == nil { + s.T().Skip("Skipping S3 test") + } + + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + jobName := "test_complete_flow_s3" + schemaQualifiedName := fmt.Sprintf("e2e_test_%s.%s", s3Suffix, jobName) + + s.setupSourceTable(jobName, 10) + query := fmt.Sprintf("SELECT * FROM %s WHERE updated_at >= {{.start}} AND updated_at < {{.end}}", + schemaQualifiedName) + qrepConfig, err := e2e.CreateQRepWorkflowConfig( + jobName, + schemaQualifiedName, + "e2e_dest_1", + query, + protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, + s.s3Helper.GetPeer(), + "stage", + ) + s.NoError(err) + qrepConfig.StagingPath = s.s3Helper.s3Config.Url + + e2e.RunQrepFlowWorkflow(env, qrepConfig) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + s.NoError(err) + + // Verify destination has 1 file + // make context with timeout + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + defer cancel() + + files, err := s.s3Helper.ListAllFiles(ctx, jobName) + + require.NoError(s.T(), err) + + require.Equal(s.T(), 1, len(files)) + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteS3) Test_Complete_QRep_Flow_S3_CTID() { + if s.s3Helper == nil { + s.T().Skip("Skipping S3 test") + } + + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + jobName := "test_complete_flow_s3_ctid" + schemaQualifiedName := fmt.Sprintf("e2e_test_%s.%s", s3Suffix, jobName) + + s.setupSourceTable(jobName, 20000) + query := fmt.Sprintf("SELECT * FROM %s WHERE ctid BETWEEN {{.start}} AND {{.end}}", schemaQualifiedName) + qrepConfig, err := e2e.CreateQRepWorkflowConfig( + jobName, + schemaQualifiedName, + "e2e_dest_ctid", + query, + protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, + s.s3Helper.GetPeer(), + "stage", + ) + s.NoError(err) + qrepConfig.StagingPath = s.s3Helper.s3Config.Url + qrepConfig.NumRowsPerPartition = 2000 + qrepConfig.InitialCopyOnly = true + qrepConfig.WatermarkColumn = "ctid" + + e2e.RunQrepFlowWorkflow(env, qrepConfig) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + s.NoError(err) + + // Verify destination has 1 file + // make context with timeout + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + defer cancel() + + files, err := s.s3Helper.ListAllFiles(ctx, jobName) + + require.NoError(s.T(), err) + + require.Equal(s.T(), 10, len(files)) + + env.AssertExpectations(s.T()) +} diff --git a/flow/e2e/s3_helper.go b/flow/e2e/s3/s3_helper.go similarity index 99% rename from flow/e2e/s3_helper.go rename to flow/e2e/s3/s3_helper.go index 49307e6242..9dbe958eef 100644 --- a/flow/e2e/s3_helper.go +++ b/flow/e2e/s3/s3_helper.go @@ -1,4 +1,4 @@ -package e2e +package e2e_s3 import ( "context" diff --git a/flow/e2e/snowflake/peer_flow_sf_test.go b/flow/e2e/snowflake/peer_flow_sf_test.go new file mode 100644 index 0000000000..db55046961 --- /dev/null +++ b/flow/e2e/snowflake/peer_flow_sf_test.go @@ -0,0 +1,800 @@ +package e2e_snowflake + +import ( + "context" + "fmt" + "testing" + + "github.com/PeerDB-io/peer-flow/e2e" + "github.com/PeerDB-io/peer-flow/generated/protos" + peerflow "github.com/PeerDB-io/peer-flow/workflows" + "github.com/jackc/pgx/v5/pgxpool" + "github.com/joho/godotenv" + log "github.com/sirupsen/logrus" + "github.com/stretchr/testify/suite" + "go.temporal.io/sdk/testsuite" +) + +const snowflakeSuffix = "snowflake" + +type PeerFlowE2ETestSuiteSF struct { + suite.Suite + testsuite.WorkflowTestSuite + + pool *pgxpool.Pool + sfHelper *SnowflakeTestHelper +} + +func TestPeerFlowE2ETestSuiteSF(t *testing.T) { + suite.Run(t, new(PeerFlowE2ETestSuiteSF)) +} + +func (s *PeerFlowE2ETestSuiteSF) attachSchemaSuffix(tableName string) string { + return fmt.Sprintf("e2e_test_%s.%s", snowflakeSuffix, tableName) +} + +func (s *PeerFlowE2ETestSuiteSF) attachSuffix(input string) string { + return fmt.Sprintf("%s_%s", input, snowflakeSuffix) +} + +// setupSnowflake sets up the snowflake connection. +func (s *PeerFlowE2ETestSuiteSF) setupSnowflake() error { + sfHelper, err := NewSnowflakeTestHelper() + if err != nil { + return fmt.Errorf("failed to create snowflake helper: %w", err) + } + + s.sfHelper = sfHelper + + return nil +} + +func (s *PeerFlowE2ETestSuiteSF) SetupSuite() { + err := godotenv.Load() + if err != nil { + // it's okay if the .env file is not present + // we will use the default values + log.Infof("Unable to load .env file, using default values from env") + } + + log.SetReportCaller(true) + + pool, err := e2e.SetupPostgres(snowflakeSuffix) + if err != nil { + s.Fail("failed to setup postgres", err) + } + s.pool = pool + + err = s.setupSnowflake() + if err != nil { + s.Fail("failed to setup snowflake", err) + } +} + +// Implement TearDownAllSuite interface to tear down the test suite +func (s *PeerFlowE2ETestSuiteSF) TearDownSuite() { + err := e2e.TearDownPostgres(s.pool, snowflakeSuffix) + if err != nil { + s.Fail("failed to drop Postgres schema", err) + } + + if s.sfHelper != nil { + err = s.sfHelper.Cleanup() + if err != nil { + s.Fail("failed to clean up Snowflake", err) + } + } +} + +func (s *PeerFlowE2ETestSuiteSF) Test_Complete_Simple_Flow_SF() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_simple_flow_sf") + dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_simple_flow_sf") + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE %s ( + id SERIAL PRIMARY KEY, + key TEXT NOT NULL, + value TEXT NOT NULL + ); + `, srcTableName)) + s.NoError(err) + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_simple_flow"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.sfHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 2, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and then insert 10 rows into the source table + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + // insert 10 rows into the source table + for i := 0; i < 10; i++ { + testKey := fmt.Sprintf("test_key_%d", i) + testValue := fmt.Sprintf("test_value_%d", i) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s (key, value) VALUES ($1, $2) + `, srcTableName), testKey, testValue) + s.NoError(err) + } + fmt.Println("Inserted 10 rows into the source table") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + count, err := s.sfHelper.CountRows("test_simple_flow_sf") + s.NoError(err) + s.Equal(10, count) + + // TODO: verify that the data is correctly synced to the destination table + // on the bigquery side + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteSF) Test_Complete_Simple_Flow_SF_Avro_CDC() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_simple_flow_sf_avro_cdc") + dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_simple_flow_sf_avro_cdc") + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE %s ( + id SERIAL PRIMARY KEY, + key TEXT NOT NULL, + value TEXT NOT NULL + ); + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_simple_flow_avro"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.sfHelper.Peer, + CDCSyncMode: protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 2, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and then insert 10 rows into the source table + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + // insert 10 rows into the source table + for i := 0; i < 10; i++ { + testKey := fmt.Sprintf("test_key_%d", i) + testValue := fmt.Sprintf("test_value_%d", i) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s (key, value) VALUES ($1, $2) + `, srcTableName), testKey, testValue) + s.NoError(err) + } + fmt.Println("Inserted 10 rows into the source table") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + count, err := s.sfHelper.CountRows("test_simple_flow_sf_avro_cdc") + s.NoError(err) + s.Equal(10, count) + + // TODO: verify that the data is correctly synced to the destination table + // on the bigquery side + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteSF) Test_Toast_SF() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_toast_sf_1") + dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_1") + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE %s ( + id SERIAL PRIMARY KEY, + t1 text, + t2 text, + k int + );CREATE OR REPLACE FUNCTION random_string( int ) RETURNS TEXT as $$ + SELECT string_agg(substring('0123456789bcdfghjkmnpqrstvwxyz', + round(random() * 30)::integer, 1), '') FROM generate_series(1, $1); + $$ language sql; + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_toast_sf_1"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.sfHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 1, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and execute a transaction touching toast columns + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + /* + Executing a transaction which + 1. changes both toast column + 2. changes no toast column + 2. changes 1 toast column + */ + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + BEGIN; + INSERT INTO %s (t1,t2,k) SELECT random_string(9000),random_string(9000), + 1 FROM generate_series(1,2); + UPDATE %s SET k=102 WHERE id=1; + UPDATE %s SET t1='dummy' WHERE id=2; + END; + `, srcTableName, srcTableName, srcTableName)) + s.NoError(err) + fmt.Println("Executed a transaction touching toast columns") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + s.compareTableContentsSF("test_toast_sf_1", `id,t1,t2,k`, false) + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteSF) Test_Toast_Nochanges_SF() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_toast_sf_2") + dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_2") + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE %s ( + id SERIAL PRIMARY KEY, + t1 text, + t2 text, + k int + );CREATE OR REPLACE FUNCTION random_string( int ) RETURNS TEXT as $$ + SELECT string_agg(substring('0123456789bcdfghjkmnpqrstvwxyz', + round(random() * 30)::integer, 1), '') FROM generate_series(1, $1); + $$ language sql; + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_toast_sf_2"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.sfHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 1, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and execute a transaction touching toast columns + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + /* transaction updating no rows */ + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + BEGIN; + UPDATE %s SET k=102 WHERE id=1; + UPDATE %s SET t1='dummy' WHERE id=2; + END; + `, srcTableName, srcTableName)) + s.NoError(err) + fmt.Println("Executed a transaction touching toast columns") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + s.compareTableContentsSF("test_toast_sf_2", `id,t1,t2,k`, false) + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteSF) Test_Toast_Advance_1_SF() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_toast_sf_3") + dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_3") + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE %s ( + id SERIAL PRIMARY KEY, + t1 text, + t2 text, + k int + );CREATE OR REPLACE FUNCTION random_string( int ) RETURNS TEXT as $$ + SELECT string_agg(substring('0123456789bcdfghjkmnpqrstvwxyz', + round(random() * 30)::integer, 1), '') FROM generate_series(1, $1); + $$ language sql; + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_toast_sf_3"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.sfHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 2, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and execute a transaction touching toast columns + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + //complex transaction with random DMLs on a table with toast columns + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + BEGIN; + INSERT INTO %s (t1,t2,k) SELECT random_string(9000),random_string(9000), + 1 FROM generate_series(1,2); + UPDATE %s SET k=102 WHERE id=1; + UPDATE %s SET t1='dummy' WHERE id=2; + UPDATE %s SET t2='dummy' WHERE id=2; + DELETE FROM %s WHERE id=1; + INSERT INTO %s(t1,t2,k) SELECT random_string(9000),random_string(9000), + 1 FROM generate_series(1,2); + UPDATE %s SET k=1 WHERE id=1; + UPDATE %s SET t1='dummy1',t2='dummy2' WHERE id=1; + UPDATE %s SET t1='dummy3' WHERE id=3; + DELETE FROM %s WHERE id=2; + DELETE FROM %s WHERE id=3; + DELETE FROM %s WHERE id=2; + END; + `, srcTableName, srcTableName, srcTableName, srcTableName, srcTableName, srcTableName, + srcTableName, srcTableName, srcTableName, srcTableName, srcTableName, srcTableName)) + s.NoError(err) + fmt.Println("Executed a transaction touching toast columns") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + s.compareTableContentsSF("test_toast_sf_3", `id,t1,t2,k`, false) + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteSF) Test_Toast_Advance_2_SF() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_toast_sf_4") + dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_4") + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE %s ( + id SERIAL PRIMARY KEY, + t1 text, + k int + );CREATE OR REPLACE FUNCTION random_string( int ) RETURNS TEXT as $$ + SELECT string_agg(substring('0123456789bcdfghjkmnpqrstvwxyz', + round(random() * 30)::integer, 1), '') FROM generate_series(1, $1); + $$ language sql; + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_toast_sf_4"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.sfHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 1, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and execute a transaction touching toast columns + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + //complex transaction with random DMLs on a table with toast columns + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + BEGIN; + INSERT INTO %s (t1,k) SELECT random_string(9000), + 1 FROM generate_series(1,1); + UPDATE %s SET t1=sub.t1 FROM (SELECT random_string(9000) t1 + FROM generate_series(1,1) ) sub WHERE id=1; + UPDATE %s SET k=2 WHERE id=1; + UPDATE %s SET k=3 WHERE id=1; + UPDATE %s SET t1=sub.t1 FROM (SELECT random_string(9000) t1 + FROM generate_series(1,1)) sub WHERE id=1; + UPDATE %s SET k=4 WHERE id=1; + END; + `, srcTableName, srcTableName, srcTableName, srcTableName, srcTableName, srcTableName)) + s.NoError(err) + fmt.Println("Executed a transaction touching toast columns") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + s.compareTableContentsSF("test_toast_sf_4", `id,t1,k`, false) + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteSF) Test_Toast_Advance_3_SF() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_toast_sf_5") + dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_5") + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE %s ( + id SERIAL PRIMARY KEY, + t1 text, + t2 text, + k int + );CREATE OR REPLACE FUNCTION random_string( int ) RETURNS TEXT as $$ + SELECT string_agg(substring('0123456789bcdfghjkmnpqrstvwxyz', + round(random() * 30)::integer, 1), '') FROM generate_series(1, $1); + $$ language sql; + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_toast_sf_5"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.sfHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 1, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and execute a transaction touching toast columns + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + /* + transaction updating a single row + multiple times with changed/unchanged toast columns + */ + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + BEGIN; + INSERT INTO %s (t1,t2,k) SELECT random_string(9000),random_string(9000), + 1 FROM generate_series(1,1); + UPDATE %s SET k=102 WHERE id=1; + UPDATE %s SET t1='dummy' WHERE id=1; + UPDATE %s SET t2='dummy' WHERE id=1; + END; + `, srcTableName, srcTableName, srcTableName, srcTableName)) + s.NoError(err) + fmt.Println("Executed a transaction touching toast columns") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + s.compareTableContentsSF("test_toast_sf_5", `id,t1,t2,k`, false) + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteSF) Test_Types_SF() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_types_sf") + dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_types_sf") + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE %s (id serial PRIMARY KEY,c1 BIGINT,c2 BIT,c3 VARBIT,c4 BOOLEAN, + c6 BYTEA,c7 CHARACTER,c8 varchar,c9 CIDR,c11 DATE,c12 FLOAT,c13 DOUBLE PRECISION, + c14 INET,c15 INTEGER,c16 INTERVAL,c17 JSON,c18 JSONB,c21 MACADDR,c22 MONEY, + c23 NUMERIC,c24 OID,c28 REAL,c29 SMALLINT,c30 SMALLSERIAL,c31 SERIAL,c32 TEXT, + c33 TIMESTAMP,c34 TIMESTAMPTZ,c35 TIME, c36 TIMETZ,c37 TSQUERY,c38 TSVECTOR, + c39 TXID_SNAPSHOT,c40 UUID,c41 XML); + CREATE OR REPLACE FUNCTION random_bytea(bytea_length integer) + RETURNS bytea AS $body$ + SELECT decode(string_agg(lpad(to_hex(width_bucket(random(), 0, 1, 256)-1),2,'0') ,''), 'hex') + FROM generate_series(1, $1); + $body$ + LANGUAGE 'sql' + VOLATILE + SET search_path = 'pg_catalog'; + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_types_sf"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.sfHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 1, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and execute a transaction touching toast columns + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + /* test inserting various types*/ + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s SELECT 2,2,b'1',b'101', + true,random_bytea(32),'s','test','1.1.10.2'::cidr, + CURRENT_DATE,1.23,1.234,'192.168.1.5'::inet,1, + '5 years 2 months 29 days 1 minute 2 seconds 200 milliseconds 20000 microseconds'::interval, + '{"sai":1}'::json,'{"sai":1}'::jsonb,'08:00:2b:01:02:03'::macaddr, + 1.2,1.23,4::oid,1.23,1,1,1,'test',now(),now(),now()::time,now()::timetz, + 'fat & rat'::tsquery,'a fat cat sat on a mat and ate a fat rat'::tsvector, + txid_current_snapshot(), + '66073c38-b8df-4bdb-bbca-1c97596b8940'::uuid,xmlcomment('hello'); + `, srcTableName)) + s.NoError(err) + fmt.Println("Executed an insert with all types") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + noNulls, err := s.sfHelper.CheckNull("test_types_sf", []string{"c41", "c1", "c2", "c3", "c4", + "c6", "c39", "c40", "id", "c9", "c11", "c12", "c13", "c14", "c15", "c16", "c17", "c18", + "c21", "c22", "c23", "c24", "c28", "c29", "c30", "c31", "c33", "c34", "c35", "c36", + "c37", "c38", "c7", "c8", "c32"}) + if err != nil { + fmt.Println("error %w", err) + } + // Make sure that there are no nulls + s.Equal(noNulls, true) + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteSF) Test_Types_SF_Avro_CDC() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_types_sf_avro_cdc") + dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_types_sf_avro_cdc") + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE %s (id serial PRIMARY KEY,c1 BIGINT,c2 BIT,c3 VARBIT,c4 BOOLEAN, + c6 BYTEA,c7 CHARACTER,c8 varchar,c9 CIDR,c11 DATE,c12 FLOAT,c13 DOUBLE PRECISION, + c14 INET,c15 INTEGER,c16 INTERVAL,c17 JSON,c18 JSONB,c21 MACADDR,c22 MONEY, + c23 NUMERIC,c24 OID,c28 REAL,c29 SMALLINT,c30 SMALLSERIAL,c31 SERIAL,c32 TEXT, + c33 TIMESTAMP,c34 TIMESTAMPTZ,c35 TIME, c36 TIMETZ,c37 TSQUERY,c38 TSVECTOR, + c39 TXID_SNAPSHOT,c40 UUID,c41 XML); + CREATE OR REPLACE FUNCTION random_bytea(bytea_length integer) + RETURNS bytea AS $body$ + SELECT decode(string_agg(lpad(to_hex(width_bucket(random(), 0, 1, 256)-1),2,'0') ,''), 'hex') + FROM generate_series(1, $1); + $body$ + LANGUAGE 'sql' + VOLATILE + SET search_path = 'pg_catalog'; + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_types_sf"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.sfHelper.Peer, + CDCSyncMode: protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 1, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and execute a transaction touching toast columns + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + /* test inserting various types*/ + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s SELECT 2,2,b'1',b'101', + true,random_bytea(32),'s','test','1.1.10.2'::cidr, + CURRENT_DATE,1.23,1.234,'192.168.1.5'::inet,1, + '5 years 2 months 29 days 1 minute 2 seconds 200 milliseconds 20000 microseconds'::interval, + '{"sai":1}'::json,'{"sai":1}'::jsonb,'08:00:2b:01:02:03'::macaddr, + 1.2,1.23,4::oid,1.23,1,1,1,'test',now(),now(),now()::time,now()::timetz, + 'fat & rat'::tsquery,'a fat cat sat on a mat and ate a fat rat'::tsvector, + txid_current_snapshot(), + '66073c38-b8df-4bdb-bbca-1c97596b8940'::uuid,xmlcomment('hello'); + `, srcTableName)) + s.NoError(err) + fmt.Println("Executed an insert with all types") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + noNulls, err := s.sfHelper.CheckNull("test_types_sf_avro_cdc", []string{"c41", "c1", "c2", "c3", "c4", + "c6", "c39", "c40", "id", "c9", "c11", "c12", "c13", "c14", "c15", "c16", "c17", "c18", + "c21", "c22", "c23", "c24", "c28", "c29", "c30", "c31", "c33", "c34", "c35", "c36", + "c37", "c38", "c7", "c8", "c32"}) + if err != nil { + fmt.Println("error %w", err) + } + // Make sure that there are no nulls + s.Equal(noNulls, true) + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteSF) Test_Multi_Table_SF() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTable1Name := s.attachSchemaSuffix("test1_sf") + srcTable2Name := s.attachSchemaSuffix("test2_sf") + dstTable1Name := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test1_sf") + dstTable2Name := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test2_sf") + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE %s (id serial primary key, c1 int, c2 text); + CREATE TABLE %s (id serial primary key, c1 int, c2 text); + `, srcTable1Name, srcTable2Name)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_multi_table"), + TableNameMapping: map[string]string{srcTable1Name: dstTable1Name, srcTable2Name: dstTable2Name}, + PostgresPort: e2e.PostgresPort, + Destination: s.sfHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 1, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and execute a transaction touching toast columns + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + /* inserting across multiple tables*/ + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s (c1,c2) VALUES (1,'dummy_1'); + INSERT INTO %s (c1,c2) VALUES (-1,'dummy_-1'); + `, srcTable1Name, srcTable2Name)) + s.NoError(err) + fmt.Println("Executed an insert with all types") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + count1, err := s.sfHelper.CountRows("test1_sf") + s.NoError(err) + count2, err := s.sfHelper.CountRows("test2_sf") + s.NoError(err) + + s.Equal(1, count1) + s.Equal(1, count2) + + env.AssertExpectations(s.T()) +} diff --git a/flow/e2e/snowflake/qrep_flow_sf_test.go b/flow/e2e/snowflake/qrep_flow_sf_test.go new file mode 100644 index 0000000000..4d42adbcc3 --- /dev/null +++ b/flow/e2e/snowflake/qrep_flow_sf_test.go @@ -0,0 +1,232 @@ +package e2e_snowflake + +import ( + "context" + "fmt" + + connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" + "github.com/PeerDB-io/peer-flow/e2e" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/google/uuid" + "github.com/stretchr/testify/require" +) + +func (s *PeerFlowE2ETestSuiteSF) setupSourceTable(tableName string, rowCount int) { + err := e2e.CreateSourceTableQRep(s.pool, snowflakeSuffix, tableName) + s.NoError(err) + err = e2e.PopulateSourceTable(s.pool, snowflakeSuffix, tableName, rowCount) + s.NoError(err) +} + +func (s *PeerFlowE2ETestSuiteSF) setupSFDestinationTable(dstTable string) { + schema := e2e.GetOwnersSchema() + err := s.sfHelper.CreateTable(dstTable, schema) + + // fail if table creation fails + if err != nil { + s.FailNow("unable to create table on snowflake", err) + } + + fmt.Printf("created table on snowflake: %s.%s. %v\n", s.sfHelper.testSchemaName, dstTable, err) +} + +func (s *PeerFlowE2ETestSuiteSF) compareTableContentsSF(tableName string, selector string, caseSensitive bool) { + // read rows from source table + pgQueryExecutor := connpostgres.NewQRepQueryExecutor(s.pool, context.Background(), "testflow", "testpart") + pgQueryExecutor.SetTestEnv(true) + pgRows, err := pgQueryExecutor.ExecuteAndProcessQuery( + fmt.Sprintf("SELECT %s FROM e2e_test_%s.%s ORDER BY id", selector, snowflakeSuffix, tableName), + ) + require.NoError(s.T(), err) + + // read rows from destination table + qualifiedTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, tableName) + var sfSelQuery string + if caseSensitive { + sfSelQuery = fmt.Sprintf(`SELECT %s FROM %s ORDER BY "id"`, selector, qualifiedTableName) + } else { + sfSelQuery = fmt.Sprintf(`SELECT %s FROM %s ORDER BY id`, selector, qualifiedTableName) + } + fmt.Printf("running query on snowflake: %s\n", sfSelQuery) + + // sleep for 1 min for debugging + // time.Sleep(1 * time.Minute) + + sfRows, err := s.sfHelper.ExecuteAndProcessQuery(sfSelQuery) + require.NoError(s.T(), err) + + s.True(pgRows.Equals(sfRows), "rows from source and destination tables are not equal") +} + +func (s *PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + numRows := 10 + + tblName := "test_qrep_flow_avro_sf" + s.setupSourceTable(tblName, numRows) + s.setupSFDestinationTable(tblName) + + dstSchemaQualified := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, tblName) + + query := fmt.Sprintf("SELECT * FROM e2e_test_%s.%s WHERE updated_at BETWEEN {{.start}} AND {{.end}}", + snowflakeSuffix, tblName) + + qrepConfig, err := e2e.CreateQRepWorkflowConfig( + "test_qrep_flow_avro_sf", + fmt.Sprintf("e2e_test_%s.%s", snowflakeSuffix, tblName), + dstSchemaQualified, + query, + protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, + s.sfHelper.Peer, + "", + ) + s.NoError(err) + + e2e.RunQrepFlowWorkflow(env, qrepConfig) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + + // assert that error contains "invalid connection configs" + err = env.GetWorkflowError() + s.NoError(err) + + sel := e2e.GetOwnersSelectorString() + s.compareTableContentsSF(tblName, sel, true) + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF_Upsert_Simple() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + numRows := 10 + + tblName := "test_qrep_flow_avro_sf_ups" + s.setupSourceTable(tblName, numRows) + s.setupSFDestinationTable(tblName) + + dstSchemaQualified := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, tblName) + + query := fmt.Sprintf("SELECT * FROM e2e_test_%s.%s WHERE updated_at >= {{.start}} AND updated_at < {{.end}}", + snowflakeSuffix, tblName) + + qrepConfig, err := e2e.CreateQRepWorkflowConfig( + "test_qrep_flow_avro_sf", + fmt.Sprintf("e2e_test_%s.%s", snowflakeSuffix, tblName), + dstSchemaQualified, + query, + protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, + s.sfHelper.Peer, + "", + ) + qrepConfig.WriteMode = &protos.QRepWriteMode{ + WriteType: protos.QRepWriteType_QREP_WRITE_MODE_UPSERT, + UpsertKeyColumns: []string{"id"}, + } + s.NoError(err) + + e2e.RunQrepFlowWorkflow(env, qrepConfig) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + + // assert that error contains "invalid connection configs" + err = env.GetWorkflowError() + s.NoError(err) + + sel := e2e.GetOwnersSelectorString() + s.compareTableContentsSF(tblName, sel, true) + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF_S3() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + numRows := 10 + + tblName := "test_qrep_flow_avro_sf_s3" + s.setupSourceTable(tblName, numRows) + s.setupSFDestinationTable(tblName) + + dstSchemaQualified := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, tblName) + + query := fmt.Sprintf("SELECT * FROM e2e_test_%s.%s WHERE updated_at >= {{.start}} AND updated_at < {{.end}}", + snowflakeSuffix, tblName) + + qrepConfig, err := e2e.CreateQRepWorkflowConfig( + "test_qrep_flow_avro_sf", + s.attachSchemaSuffix(tblName), + dstSchemaQualified, + query, + protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, + s.sfHelper.Peer, + "", + ) + s.NoError(err) + qrepConfig.StagingPath = fmt.Sprintf("s3://peerdb-test-bucket/avro/%s", uuid.New()) + + e2e.RunQrepFlowWorkflow(env, qrepConfig) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + + // assert that error contains "invalid connection configs" + err = env.GetWorkflowError() + s.NoError(err) + + sel := e2e.GetOwnersSelectorString() + s.compareTableContentsSF(tblName, sel, true) + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF_S3_Integration() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + numRows := 10 + + tblName := "test_qrep_flow_avro_sf_s3_int" + s.setupSourceTable(tblName, numRows) + s.setupSFDestinationTable(tblName) + + dstSchemaQualified := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, tblName) + + query := fmt.Sprintf("SELECT * FROM e2e_test_%s.%s WHERE updated_at >= {{.start}} AND updated_at < {{.end}}", + snowflakeSuffix, tblName) + + sfPeer := s.sfHelper.Peer + sfPeer.GetSnowflakeConfig().S3Integration = "peerdb_s3_integration" + + qrepConfig, err := e2e.CreateQRepWorkflowConfig( + "test_qrep_flow_avro_sf_int", + s.attachSchemaSuffix(tblName), + dstSchemaQualified, + query, + protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO, + sfPeer, + "", + ) + s.NoError(err) + qrepConfig.StagingPath = fmt.Sprintf("s3://peerdb-test-bucket/avro/%s", uuid.New()) + + e2e.RunQrepFlowWorkflow(env, qrepConfig) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + + // assert that error contains "invalid connection configs" + err = env.GetWorkflowError() + s.NoError(err) + + sel := e2e.GetOwnersSelectorString() + s.compareTableContentsSF(tblName, sel, true) + + env.AssertExpectations(s.T()) +} diff --git a/flow/e2e/snowflake/snowflake_helper.go b/flow/e2e/snowflake/snowflake_helper.go new file mode 100644 index 0000000000..2a39daf93e --- /dev/null +++ b/flow/e2e/snowflake/snowflake_helper.go @@ -0,0 +1,131 @@ +package e2e_snowflake + +import ( + "context" + "encoding/json" + "fmt" + "os" + + connsnowflake "github.com/PeerDB-io/peer-flow/connectors/snowflake" + "github.com/PeerDB-io/peer-flow/e2e" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/model" + util "github.com/PeerDB-io/peer-flow/utils" +) + +type SnowflakeTestHelper struct { + // config is the Snowflake config. + Config *protos.SnowflakeConfig + // peer struct holder Snowflake + Peer *protos.Peer + // connection to another database, to manage the test database + adminClient *connsnowflake.SnowflakeClient + // connection to the test database + testClient *connsnowflake.SnowflakeClient + // testSchemaName is the schema to use for testing. + testSchemaName string + // dbName is the database used for testing. + testDatabaseName string +} + +func NewSnowflakeTestHelper() (*SnowflakeTestHelper, error) { + jsonPath := os.Getenv("TEST_SF_CREDS") + if jsonPath == "" { + return nil, fmt.Errorf("TEST_SF_CREDS env var not set") + } + + content, err := e2e.ReadFileToBytes(jsonPath) + if err != nil { + return nil, fmt.Errorf("failed to read file: %w", err) + } + + var config protos.SnowflakeConfig + err = json.Unmarshal(content, &config) + if err != nil { + return nil, fmt.Errorf("failed to unmarshal json: %w", err) + } + + peer := generateSFPeer(&config) + runID, err := util.RandomUInt64() + if err != nil { + return nil, fmt.Errorf("failed to generate random uint64: %w", err) + } + + testDatabaseName := fmt.Sprintf("e2e_test_%d", runID) + + adminClient, err := connsnowflake.NewSnowflakeClient(context.Background(), &config) + if err != nil { + return nil, fmt.Errorf("failed to create Snowflake client: %w", err) + } + err = adminClient.ExecuteQuery(fmt.Sprintf("CREATE DATABASE %s", testDatabaseName)) + if err != nil { + return nil, fmt.Errorf("failed to create Snowflake test database: %w", err) + } + + config.Database = testDatabaseName + testClient, err := connsnowflake.NewSnowflakeClient(context.Background(), &config) + if err != nil { + return nil, fmt.Errorf("failed to create Snowflake client: %w", err) + } + + return &SnowflakeTestHelper{ + Config: &config, + Peer: peer, + adminClient: adminClient, + testClient: testClient, + testSchemaName: "PUBLIC", + testDatabaseName: testDatabaseName, + }, nil +} + +func generateSFPeer(snowflakeConfig *protos.SnowflakeConfig) *protos.Peer { + ret := &protos.Peer{} + ret.Name = "test_sf_peer" + ret.Type = protos.DBType_SNOWFLAKE + + ret.Config = &protos.Peer_SnowflakeConfig{ + SnowflakeConfig: snowflakeConfig, + } + + return ret +} + +// Cleanup drops the database. +func (s *SnowflakeTestHelper) Cleanup() error { + err := s.testClient.Close() + if err != nil { + return err + } + err = s.adminClient.ExecuteQuery(fmt.Sprintf("DROP DATABASE %s", s.testDatabaseName)) + if err != nil { + return err + } + return s.adminClient.Close() +} + +// RunCommand runs the given command. +func (s *SnowflakeTestHelper) RunCommand(command string) error { + return s.testClient.ExecuteQuery(command) +} + +// CountRows(tableName) returns the number of rows in the given table. +func (s *SnowflakeTestHelper) CountRows(tableName string) (int, error) { + res, err := s.testClient.CountRows(s.testSchemaName, tableName) + if err != nil { + return 0, err + } + + return int(res), nil +} + +func (s *SnowflakeTestHelper) CheckNull(tableName string, colNames []string) (bool, error) { + return s.testClient.CheckNull(s.testSchemaName, tableName, colNames) +} + +func (s *SnowflakeTestHelper) ExecuteAndProcessQuery(query string) (*model.QRecordBatch, error) { + return s.testClient.ExecuteAndProcessQuery(query) +} + +func (s *SnowflakeTestHelper) CreateTable(tableName string, schema *model.QRecordSchema) error { + return s.testClient.CreateTable(schema, s.testSchemaName, tableName) +} diff --git a/flow/e2e/snowflake_helper.go b/flow/e2e/snowflake_helper.go deleted file mode 100644 index 090ddeccb7..0000000000 --- a/flow/e2e/snowflake_helper.go +++ /dev/null @@ -1,104 +0,0 @@ -package e2e - -import ( - "context" - "encoding/json" - "fmt" - "os" - - connsnowflake "github.com/PeerDB-io/peer-flow/connectors/snowflake" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/model" -) - -type SnowflakeTestHelper struct { - // config is the Snowflake config. - Config *protos.SnowflakeConfig - // peer struct holder Snowflake - Peer *protos.Peer - // connection to Snowflake - client *connsnowflake.SnowflakeClient - // testSchemaName is the schema to use for testing. - testSchemaName string -} - -func NewSnowflakeTestHelper(testSchemaName string) (*SnowflakeTestHelper, error) { - jsonPath := os.Getenv("TEST_SF_CREDS") - if jsonPath == "" { - return nil, fmt.Errorf("TEST_SF_CREDS env var not set") - } - - content, err := readFileToBytes(jsonPath) - if err != nil { - return nil, fmt.Errorf("failed to read file: %w", err) - } - - var config protos.SnowflakeConfig - err = json.Unmarshal(content, &config) - if err != nil { - return nil, fmt.Errorf("failed to unmarshal json: %w", err) - } - - peer := generateSFPeer(&config) - - client, err := connsnowflake.NewSnowflakeClient(context.Background(), &config) - if err != nil { - return nil, fmt.Errorf("failed to create Snowflake client: %w", err) - } - - return &SnowflakeTestHelper{ - Config: &config, - Peer: peer, - client: client, - testSchemaName: testSchemaName, - }, nil -} - -func generateSFPeer(snowflakeConfig *protos.SnowflakeConfig) *protos.Peer { - ret := &protos.Peer{} - ret.Name = "test_sf_peer" - ret.Type = protos.DBType_SNOWFLAKE - - ret.Config = &protos.Peer_SnowflakeConfig{ - SnowflakeConfig: snowflakeConfig, - } - - return ret -} - -// RecreateSchema recreates the schema, i.e., drops it if exists and creates it again. -func (s *SnowflakeTestHelper) RecreateSchema() error { - return s.client.RecreateSchema(s.testSchemaName) -} - -// DropSchema drops the schema. -func (s *SnowflakeTestHelper) DropSchema() error { - return s.client.DropSchema(s.testSchemaName) -} - -// RunCommand runs the given command. -func (s *SnowflakeTestHelper) RunCommand(command string) error { - return s.client.ExecuteQuery(command) -} - -// CountRows(tableName) returns the number of rows in the given table. -func (s *SnowflakeTestHelper) CountRows(tableName string) (int, error) { - res, err := s.client.CountRows(s.testSchemaName, tableName) - if err != nil { - return 0, err - } - - return int(res), nil -} - -func (s *SnowflakeTestHelper) CheckNull(tableName string, colNames []string) (bool, error) { - return s.client.CheckNull(s.testSchemaName, tableName, colNames) -} - -func (s *SnowflakeTestHelper) ExecuteAndProcessQuery(query string) (*model.QRecordBatch, error) { - return s.client.ExecuteAndProcessQuery(query) -} - -func (s *SnowflakeTestHelper) CreateTable(tableName string, schema *model.QRecordSchema) error { - return s.client.CreateTable(schema, s.testSchemaName, tableName) -} diff --git a/flow/e2e/qrep_flow_sqlserver_test.go b/flow/e2e/sqlserver/qrep_flow_sqlserver_test.go similarity index 53% rename from flow/e2e/qrep_flow_sqlserver_test.go rename to flow/e2e/sqlserver/qrep_flow_sqlserver_test.go index 80fe7e1e65..4a072f2011 100644 --- a/flow/e2e/qrep_flow_sqlserver_test.go +++ b/flow/e2e/sqlserver/qrep_flow_sqlserver_test.go @@ -1,23 +1,92 @@ -package e2e +package e2e_sqlserver import ( "context" "fmt" + "os" + "testing" "time" + "github.com/PeerDB-io/peer-flow/e2e" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" + "github.com/jackc/pgx/v5/pgxpool" + "github.com/joho/godotenv" + log "github.com/sirupsen/logrus" "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + "go.temporal.io/sdk/testsuite" ) -func (s *E2EPeerFlowTestSuite) setupSQLServerTable(tableName string) { +const sqlserverSuffix = "s3" + +type PeerFlowE2ETestSuiteSQLServer struct { + suite.Suite + testsuite.WorkflowTestSuite + + pool *pgxpool.Pool + sqlsHelper *SQLServerHelper +} + +func TestPeerFlowE2ETestSuiteSQLServer(t *testing.T) { + suite.Run(t, new(PeerFlowE2ETestSuiteSQLServer)) +} + +// setup sql server connection +func (s *PeerFlowE2ETestSuiteSQLServer) setupSQLServer() { + env := os.Getenv("ENABLE_SQLSERVER_TESTS") + if env != "true" { + s.sqlsHelper = nil + return + } + + sqlsHelper, err := NewSQLServerHelper("test_sqlserver_peer") + require.NoError(s.T(), err) + s.sqlsHelper = sqlsHelper +} + +func (s *PeerFlowE2ETestSuiteSQLServer) SetupSuite() { + err := godotenv.Load() + if err != nil { + // it's okay if the .env file is not present + // we will use the default values + log.Infof("Unable to load .env file, using default values from env") + } + + log.SetReportCaller(true) + + pool, err := e2e.SetupPostgres(sqlserverSuffix) + if err != nil { + s.Fail("failed to setup postgres", err) + } + s.pool = pool + + s.setupSQLServer() +} + +// Implement TearDownAllSuite interface to tear down the test suite +func (s *PeerFlowE2ETestSuiteSQLServer) TearDownSuite() { + err := e2e.TearDownPostgres(s.pool, sqlserverSuffix) + if err != nil { + s.Fail("failed to drop Postgres schema", err) + } + + if s.sqlsHelper != nil { + err = s.sqlsHelper.CleanUp() + if err != nil { + s.Fail("failed to clean up sqlserver", err) + } + } +} + +func (s *PeerFlowE2ETestSuiteSQLServer) setupSQLServerTable(tableName string) { schema := getSimpleTableSchema() err := s.sqlsHelper.CreateTable(schema, tableName) require.NoError(s.T(), err) } -func (s *E2EPeerFlowTestSuite) insertRowsIntoSQLServerTable(tableName string, numRows int) { +func (s *PeerFlowE2ETestSuiteSQLServer) insertRowsIntoSQLServerTable(tableName string, numRows int) { schemaQualified := fmt.Sprintf("%s.%s", s.sqlsHelper.SchemaName, tableName) for i := 0; i < numRows; i++ { params := make(map[string]interface{}) @@ -37,16 +106,15 @@ func (s *E2EPeerFlowTestSuite) insertRowsIntoSQLServerTable(tableName string, nu } } -func (s *E2EPeerFlowTestSuite) setupPGDestinationTable(schemaName, tableName string) { +func (s *PeerFlowE2ETestSuiteSQLServer) setupPGDestinationTable(tableName string) { ctx := context.Background() - _, err := s.pool.Exec(ctx, fmt.Sprintf("CREATE SCHEMA IF NOT EXISTS %s", schemaName)) - require.NoError(s.T(), err) - _, err = s.pool.Exec(ctx, fmt.Sprintf("DROP TABLE IF EXISTS %s.%s", schemaName, tableName)) + _, err := s.pool.Exec(ctx, fmt.Sprintf("DROP TABLE IF EXISTS e2e_test_%s.%s", sqlserverSuffix, tableName)) require.NoError(s.T(), err) - //nolint:lll - _, err = s.pool.Exec(ctx, fmt.Sprintf("CREATE TABLE %s.%s (id TEXT, card_id TEXT, v_from TIMESTAMP, price NUMERIC, status INT)", schemaName, tableName)) + _, err = s.pool.Exec(ctx, + fmt.Sprintf("CREATE TABLE e2e_test_%s.%s (id TEXT, card_id TEXT, v_from TIMESTAMP, price NUMERIC, status INT)", + sqlserverSuffix, tableName)) require.NoError(s.T(), err) } @@ -62,13 +130,13 @@ func getSimpleTableSchema() *model.QRecordSchema { } } -func (s *E2EPeerFlowTestSuite) Test_Complete_QRep_Flow_SqlServer_Append() { +func (s *PeerFlowE2ETestSuiteSQLServer) Test_Complete_QRep_Flow_SqlServer_Append() { if s.sqlsHelper == nil { s.T().Skip("Skipping SQL Server test") } env := s.NewTestWorkflowEnvironment() - registerWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env) numRows := 10 tblName := "test_qrep_flow_avro_ss_append" @@ -77,13 +145,13 @@ func (s *E2EPeerFlowTestSuite) Test_Complete_QRep_Flow_SqlServer_Append() { s.setupSQLServerTable(tblName) s.insertRowsIntoSQLServerTable(tblName, numRows) - s.setupPGDestinationTable(s.sqlsHelper.SchemaName, tblName) - dstTableName := fmt.Sprintf("%s.%s", s.sqlsHelper.SchemaName, tblName) + s.setupPGDestinationTable(tblName) + dstTableName := fmt.Sprintf("e2e_test_%s.%s", sqlserverSuffix, tblName) - //nolint:lll - query := fmt.Sprintf("SELECT * FROM %s.%s WHERE v_from BETWEEN {{.start}} AND {{.end}}", s.sqlsHelper.SchemaName, tblName) + query := fmt.Sprintf("SELECT * FROM %s.%s WHERE v_from BETWEEN {{.start}} AND {{.end}}", + s.sqlsHelper.SchemaName, tblName) - postgresPeer := GeneratePostgresPeer(postgresPort) + postgresPeer := e2e.GeneratePostgresPeer(e2e.PostgresPort) qrepConfig := &protos.QRepConfig{ FlowJobName: tblName, @@ -100,7 +168,7 @@ func (s *E2EPeerFlowTestSuite) Test_Complete_QRep_Flow_SqlServer_Append() { WaitBetweenBatchesSeconds: 5, } - runQrepFlowWorkflow(env, qrepConfig) + e2e.RunQrepFlowWorkflow(env, qrepConfig) // Verify workflow completes without error s.True(env.IsWorkflowCompleted()) diff --git a/flow/e2e/sqlserver_helper.go b/flow/e2e/sqlserver/sqlserver_helper.go similarity index 99% rename from flow/e2e/sqlserver_helper.go rename to flow/e2e/sqlserver/sqlserver_helper.go index 38933b35e1..1be94fc83e 100644 --- a/flow/e2e/sqlserver_helper.go +++ b/flow/e2e/sqlserver/sqlserver_helper.go @@ -1,4 +1,4 @@ -package e2e +package e2e_sqlserver import ( "context" diff --git a/flow/e2e/test_utils.go b/flow/e2e/test_utils.go index 293af9095f..761e0cbf51 100644 --- a/flow/e2e/test_utils.go +++ b/flow/e2e/test_utils.go @@ -1,13 +1,27 @@ package e2e import ( + "context" + "encoding/json" "fmt" "io" "os" + "strings" + "time" + + "github.com/PeerDB-io/peer-flow/activities" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/model" + "github.com/PeerDB-io/peer-flow/model/qvalue" + peerflow "github.com/PeerDB-io/peer-flow/workflows" + "github.com/google/uuid" + "github.com/jackc/pgx/v5/pgxpool" + log "github.com/sirupsen/logrus" + "go.temporal.io/sdk/testsuite" ) // readFileToBytes reads a file to a byte array. -func readFileToBytes(path string) ([]byte, error) { +func ReadFileToBytes(path string) ([]byte, error) { var ret []byte f, err := os.Open(path) @@ -24,3 +38,277 @@ func readFileToBytes(path string) ([]byte, error) { return ret, nil } + +func RegisterWorkflowsAndActivities(env *testsuite.TestWorkflowEnvironment) { + // set a 300 second timeout for the workflow to execute a few runs. + env.SetTestTimeout(300 * time.Second) + + env.RegisterWorkflow(peerflow.PeerFlowWorkflow) + env.RegisterWorkflow(peerflow.PeerFlowWorkflowWithConfig) + env.RegisterWorkflow(peerflow.SyncFlowWorkflow) + env.RegisterWorkflow(peerflow.SetupFlowWorkflow) + env.RegisterWorkflow(peerflow.SnapshotFlowWorkflow) + env.RegisterWorkflow(peerflow.NormalizeFlowWorkflow) + env.RegisterWorkflow(peerflow.QRepFlowWorkflow) + env.RegisterWorkflow(peerflow.QRepPartitionWorkflow) + env.RegisterActivity(&activities.FetchConfigActivity{}) + env.RegisterActivity(&activities.FlowableActivity{}) + env.RegisterActivity(&activities.SnapshotActivity{}) +} + +func SetupPeerFlowStatusQuery(env *testsuite.TestWorkflowEnvironment, + connectionGen FlowConnectionGenerationConfig) { + // wait for PeerFlowStatusQuery to finish setup + // sleep for 5 second to allow the workflow to start + time.Sleep(5 * time.Second) + for { + response, err := env.QueryWorkflow( + peerflow.PeerFlowStatusQuery, + connectionGen.FlowJobName, + ) + if err == nil { + var state peerflow.PeerFlowState + err = response.Get(&state) + if err != nil { + log.Errorln(err) + } + + if state.SetupComplete { + fmt.Println("query indicates setup is complete") + break + } + } else { + // log the error for informational purposes + log.Errorln(err) + } + time.Sleep(1 * time.Second) + } +} + +func CreateSourceTableQRep(pool *pgxpool.Pool, suffix string, tableName string) error { + tblFields := []string{ + "id UUID NOT NULL PRIMARY KEY", + "card_id UUID", + `"from" TIMESTAMP NOT NULL`, + "price NUMERIC", + "created_at TIMESTAMP NOT NULL", + "updated_at TIMESTAMP NOT NULL", + "transaction_hash BYTEA", + "ownerable_type VARCHAR", + "ownerable_id UUID", + "user_nonce INTEGER", + "transfer_type INTEGER DEFAULT 0 NOT NULL", + "blockchain INTEGER NOT NULL", + "deal_type VARCHAR", + "deal_id UUID", + "ethereum_transaction_id UUID", + "ignore_price BOOLEAN DEFAULT false", + "card_eth_value DOUBLE PRECISION", + "paid_eth_price DOUBLE PRECISION", + "card_bought_notified BOOLEAN DEFAULT false NOT NULL", + "address NUMERIC", + "account_id UUID", + "asset_id NUMERIC NOT NULL", + "status INTEGER", + "transaction_id UUID", + "settled_at TIMESTAMP", + "reference_id VARCHAR", + "settle_at TIMESTAMP", + "settlement_delay_reason INTEGER", + "f1 text[]", + "f2 bigint[]", + "f3 int[]", + "f4 varchar[]", + "f5 jsonb", + "f6 jsonb", + "f7 jsonb", + "f8 smallint", + } + + tblFieldStr := strings.Join(tblFields, ",") + + _, err := pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE e2e_test_%s.%s ( + %s + );`, suffix, tableName, tblFieldStr)) + if err != nil { + return err + } + + fmt.Printf("created table on postgres: e2e_test_%s.%s\n", suffix, tableName) + return nil +} + +func generate20MBJson() ([]byte, error) { + xn := make(map[string]interface{}) + for i := 0; i < 215000; i++ { + xn[uuid.New().String()] = uuid.New().String() + } + + v, err := json.Marshal(xn) + if err != nil { + return nil, err + } + + return v, nil +} + +func PopulateSourceTable(pool *pgxpool.Pool, suffix string, tableName string, rowCount int) error { + var ids []string + var rows []string + for i := 0; i < rowCount-1; i++ { + id := uuid.New().String() + ids = append(ids, id) + row := fmt.Sprintf(` + ( + '%s', '%s', CURRENT_TIMESTAMP, 3.86487206688919, CURRENT_TIMESTAMP, + CURRENT_TIMESTAMP, E'\\\\xDEADBEEF', 'type1', '%s', + 1, 0, 1, 'dealType1', + '%s', '%s', false, 1.2345, + 1.2345, false, 12345, '%s', + 12345, 1, '%s', CURRENT_TIMESTAMP, 'refID', + CURRENT_TIMESTAMP, 1, ARRAY['text1', 'text2'], ARRAY[123, 456], ARRAY[789, 012], + ARRAY['varchar1', 'varchar2'], '{"key": 8.5}', + '[{"key1": "value1", "key2": "value2", "key3": "value3"}]', + '{"key": "value"}', 15 + )`, + id, uuid.New().String(), uuid.New().String(), + uuid.New().String(), uuid.New().String(), uuid.New().String(), uuid.New().String()) + rows = append(rows, row) + } + + _, err := pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO e2e_test_%s.%s ( + id, card_id, "from", price, created_at, + updated_at, transaction_hash, ownerable_type, ownerable_id, + user_nonce, transfer_type, blockchain, deal_type, + deal_id, ethereum_transaction_id, ignore_price, card_eth_value, + paid_eth_price, card_bought_notified, address, account_id, + asset_id, status, transaction_id, settled_at, reference_id, + settle_at, settlement_delay_reason, f1, f2, f3, f4, f5, f6, f7, f8 + ) VALUES %s; + `, suffix, tableName, strings.Join(rows, ","))) + if err != nil { + return err + } + + // add a row where all the nullable fields are null + _, err = pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO e2e_test_%s.%s ( + id, "from", created_at, updated_at, + transfer_type, blockchain, card_bought_notified, asset_id + ) VALUES ( + '%s', CURRENT_TIMESTAMP, CURRENT_TIMESTAMP, CURRENT_TIMESTAMP, + 0, 1, false, 12345 + ); + `, suffix, tableName, uuid.New().String())) + if err != nil { + return err + } + + // generate a 20 MB json and update id[0]'s col f5 to it + v, err := generate20MBJson() + if err != nil { + return err + } + _, err = pool.Exec(context.Background(), fmt.Sprintf(` + UPDATE e2e_test_%s.%s SET f5 = $1 WHERE id = $2; + `, suffix, tableName), v, ids[0]) + if err != nil { + return err + } + + return nil +} + +func CreateQRepWorkflowConfig( + flowJobName string, + sourceTable string, + dstTable string, + query string, + syncMode protos.QRepSyncMode, + dest *protos.Peer, + stagingPath string, +) (*protos.QRepConfig, error) { + connectionGen := QRepFlowConnectionGenerationConfig{ + FlowJobName: flowJobName, + WatermarkTable: sourceTable, + DestinationTableIdentifier: dstTable, + PostgresPort: PostgresPort, + Destination: dest, + StagingPath: stagingPath, + } + + watermark := "updated_at" + + qrepConfig, err := connectionGen.GenerateQRepConfig(query, watermark, syncMode) + if err != nil { + return nil, err + } + + qrepConfig.InitialCopyOnly = true + + return qrepConfig, nil +} + +func RunQrepFlowWorkflow(env *testsuite.TestWorkflowEnvironment, config *protos.QRepConfig) { + lastPartition := &protos.QRepPartition{ + PartitionId: "not-applicable-partition", + Range: nil, + } + numPartitionsProcessed := 0 + env.ExecuteWorkflow(peerflow.QRepFlowWorkflow, config, lastPartition, numPartitionsProcessed) +} + +func GetOwnersSchema() *model.QRecordSchema { + return &model.QRecordSchema{ + Fields: []*model.QField{ + {Name: "id", Type: qvalue.QValueKindString, Nullable: true}, + {Name: "card_id", Type: qvalue.QValueKindString, Nullable: true}, + {Name: "from", Type: qvalue.QValueKindTimestamp, Nullable: true}, + {Name: "price", Type: qvalue.QValueKindNumeric, Nullable: true}, + {Name: "created_at", Type: qvalue.QValueKindTimestamp, Nullable: true}, + {Name: "updated_at", Type: qvalue.QValueKindTimestamp, Nullable: true}, + {Name: "transaction_hash", Type: qvalue.QValueKindBytes, Nullable: true}, + {Name: "ownerable_type", Type: qvalue.QValueKindString, Nullable: true}, + {Name: "ownerable_id", Type: qvalue.QValueKindString, Nullable: true}, + {Name: "user_nonce", Type: qvalue.QValueKindInt64, Nullable: true}, + {Name: "transfer_type", Type: qvalue.QValueKindInt64, Nullable: true}, + {Name: "blockchain", Type: qvalue.QValueKindInt64, Nullable: true}, + {Name: "deal_type", Type: qvalue.QValueKindString, Nullable: true}, + {Name: "deal_id", Type: qvalue.QValueKindString, Nullable: true}, + {Name: "ethereum_transaction_id", Type: qvalue.QValueKindString, Nullable: true}, + {Name: "ignore_price", Type: qvalue.QValueKindBoolean, Nullable: true}, + {Name: "card_eth_value", Type: qvalue.QValueKindFloat64, Nullable: true}, + {Name: "paid_eth_price", Type: qvalue.QValueKindFloat64, Nullable: true}, + {Name: "card_bought_notified", Type: qvalue.QValueKindBoolean, Nullable: true}, + {Name: "address", Type: qvalue.QValueKindNumeric, Nullable: true}, + {Name: "account_id", Type: qvalue.QValueKindString, Nullable: true}, + {Name: "asset_id", Type: qvalue.QValueKindNumeric, Nullable: true}, + {Name: "status", Type: qvalue.QValueKindInt64, Nullable: true}, + {Name: "transaction_id", Type: qvalue.QValueKindString, Nullable: true}, + {Name: "settled_at", Type: qvalue.QValueKindTimestamp, Nullable: true}, + {Name: "reference_id", Type: qvalue.QValueKindString, Nullable: true}, + {Name: "settle_at", Type: qvalue.QValueKindTimestamp, Nullable: true}, + {Name: "settlement_delay_reason", Type: qvalue.QValueKindInt64, Nullable: true}, + {Name: "f1", Type: qvalue.QValueKindArrayString, Nullable: true}, + {Name: "f2", Type: qvalue.QValueKindArrayInt64, Nullable: true}, + {Name: "f3", Type: qvalue.QValueKindArrayInt32, Nullable: true}, + {Name: "f4", Type: qvalue.QValueKindArrayString, Nullable: true}, + {Name: "f5", Type: qvalue.QValueKindJSON, Nullable: true}, + {Name: "f6", Type: qvalue.QValueKindJSON, Nullable: true}, + {Name: "f7", Type: qvalue.QValueKindJSON, Nullable: true}, + {Name: "f8", Type: qvalue.QValueKindInt16, Nullable: true}, + }, + } +} + +func GetOwnersSelectorString() string { + schema := GetOwnersSchema() + var fields []string + for _, field := range schema.Fields { + // append quoted field name + fields = append(fields, fmt.Sprintf(`"%s"`, field.Name)) + } + return strings.Join(fields, ",") +} From 86083fedff48df32bb1440bdbfcdd58b49eae6c8 Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Thu, 31 Aug 2023 07:39:14 -0700 Subject: [PATCH 09/12] improve avro writer heart beats (#366) --- flow/connectors/utils/avro/avro_writer.go | 37 ++++++++++++----------- 1 file changed, 19 insertions(+), 18 deletions(-) diff --git a/flow/connectors/utils/avro/avro_writer.go b/flow/connectors/utils/avro/avro_writer.go index 38c80d510f..6fd0d22bd3 100644 --- a/flow/connectors/utils/avro/avro_writer.go +++ b/flow/connectors/utils/avro/avro_writer.go @@ -5,16 +5,17 @@ import ( "fmt" "io" "os" + "time" "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" - "go.temporal.io/sdk/activity" "github.com/aws/aws-sdk-go/aws" "github.com/aws/aws-sdk-go/service/s3/s3manager" "github.com/linkedin/goavro/v2" log "github.com/sirupsen/logrus" + uber_atomic "go.uber.org/atomic" ) type PeerDBOCFWriter struct { @@ -54,8 +55,21 @@ func (p *PeerDBOCFWriter) writeRecordsToOCFWriter(ocfWriter *goavro.OCFWriter) ( } colNames := schema.GetColumnNames() - numRows := 0 - const heartBeatNumRows = 10000 + + var numRows uber_atomic.Uint32 + numRows.Store(0) + + if p.ctx != nil { + shutdown := utils.HeartbeatRoutine(p.ctx, 30*time.Second, func() string { + written := numRows.Load() + return fmt.Sprintf("[avro] written %d rows to OCF", written) + }) + + defer func() { + shutdown <- true + }() + } + for qRecordOrErr := range p.stream.Records { if qRecordOrErr.Err != nil { log.Errorf("[avro] failed to get record from stream: %v", qRecordOrErr.Err) @@ -82,23 +96,10 @@ func (p *PeerDBOCFWriter) writeRecordsToOCFWriter(ocfWriter *goavro.OCFWriter) ( return 0, fmt.Errorf("failed to write record to OCF: %w", err) } - if numRows%heartBeatNumRows == 0 { - log.Infof("written %d rows to OCF", numRows) - msg := fmt.Sprintf("written %d rows to OCF", numRows) - if p.ctx != nil { - activity.RecordHeartbeat(p.ctx, msg) - } - } - - numRows++ + numRows.Inc() } - if p.ctx != nil { - msg := fmt.Sprintf("written all: %d rows to OCF", numRows) - activity.RecordHeartbeat(p.ctx, msg) - } - - return numRows, nil + return int(numRows.Load()), nil } func (p *PeerDBOCFWriter) WriteOCF(w io.Writer) (int, error) { From 23ec4b2acbcea02f475f871d0449aa3339de9eb9 Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Fri, 1 Sep 2023 05:17:14 -0700 Subject: [PATCH 10/12] Send last processed client log position (#367) --- flow/connectors/postgres/cdc.go | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/flow/connectors/postgres/cdc.go b/flow/connectors/postgres/cdc.go index 0ea09f923d..f89ce4419f 100644 --- a/flow/connectors/postgres/cdc.go +++ b/flow/connectors/postgres/cdc.go @@ -131,8 +131,16 @@ func (p *PostgresCDCSource) consumeStream( for { if time.Now().After(nextStandbyMessageDeadline) { + // update the WALWritePosition to be clientXLogPos - 1 + // as the clientXLogPos is the last checkpoint id + 1 + // and we want to send the last checkpoint id as the last + // checkpoint id that we have processed. + lastProcessedXLogPos := clientXLogPos + if clientXLogPos > 0 { + lastProcessedXLogPos = clientXLogPos - 1 + } err := pglogrepl.SendStandbyStatusUpdate(p.ctx, conn, - pglogrepl.StandbyStatusUpdate{WALWritePosition: clientXLogPos}) + pglogrepl.StandbyStatusUpdate{WALWritePosition: lastProcessedXLogPos}) if err != nil { return nil, fmt.Errorf("SendStandbyStatusUpdate failed: %w", err) } @@ -239,8 +247,8 @@ func (p *PostgresCDCSource) consumeStream( } } - clientXLogPos = xld.WALStart + pglogrepl.LSN(len(xld.WALData)) - result.LastCheckPointID = int64(clientXLogPos) + currentPos := xld.WALStart + pglogrepl.LSN(len(xld.WALData)) + result.LastCheckPointID = int64(currentPos) if result.Records != nil && len(result.Records) == int(req.MaxBatchSize) { return result, nil From 09f26f3fb749c7fc2f80937e35a64083de44d7ab Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Fri, 1 Sep 2023 19:34:00 +0530 Subject: [PATCH 11/12] Ignore error for catalog monitor for EH (#369) CDC to Eventhub errors in SyncFlow with 'GetLastSyncBatchID not supported for Eventhub', because in `flowable.go`'s `StartFlow` this method is being called for catalog metrics. Eventhub does not have this method implemented so we can just ignore the error in this section and have sync ID in the metrics start from 0: ```Go if a.CatalogMirrorMonitor.IsActive() && len(records.Records) > 0 { syncBatchID, err := dest.GetLastSyncBatchID(input.FlowConnectionConfigs.FlowJobName) if err != nil && conn.Destination.Type != protos.DBType_EVENTHUB { return nil, err } .... } ``` --- flow/activities/flowable.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index 49e2f6a27a..6219069d91 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -205,7 +205,7 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, input *protos.StartFlo } if a.CatalogMirrorMonitor.IsActive() && len(records.Records) > 0 { syncBatchID, err := dest.GetLastSyncBatchID(input.FlowConnectionConfigs.FlowJobName) - if err != nil { + if err != nil && conn.Destination.Type != protos.DBType_EVENTHUB { return nil, err } From 3f48101348a8570e15a3d2985631c3fad65abf99 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Fri, 1 Sep 2023 21:19:29 +0530 Subject: [PATCH 12/12] Eventhub Normalize Fix (#370) `res` (result of normalize flow) in the below call is nil in case of Eventhub so we make it a non-nil: ```Go err = a.CatalogMirrorMonitor.UpdateEndTimeForCDCBatch(ctx, input.FlowConnectionConfigs.FlowJobName, res.EndBatchID) ``` --- flow/connectors/eventhub/eventhub.go | 6 +++++- flow/connectors/postgres/cdc.go | 2 +- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/flow/connectors/eventhub/eventhub.go b/flow/connectors/eventhub/eventhub.go index f94b5b0d3d..e674da2f67 100644 --- a/flow/connectors/eventhub/eventhub.go +++ b/flow/connectors/eventhub/eventhub.go @@ -310,7 +310,11 @@ func (c *EventHubConnector) SetupNormalizedTables( func (c *EventHubConnector) NormalizeRecords(req *model.NormalizeRecordsRequest) (*model.NormalizeResponse, error) { log.Infof("normalization for event hub is a no-op") - return nil, nil + return &model.NormalizeResponse{ + EndBatchID: 0, + StartBatchID: 0, + Done: true, + }, nil } // cleanup diff --git a/flow/connectors/postgres/cdc.go b/flow/connectors/postgres/cdc.go index f89ce4419f..a3e45b6ed4 100644 --- a/flow/connectors/postgres/cdc.go +++ b/flow/connectors/postgres/cdc.go @@ -144,7 +144,7 @@ func (p *PostgresCDCSource) consumeStream( if err != nil { return nil, fmt.Errorf("SendStandbyStatusUpdate failed: %w", err) } - log.Debugf("Sent Standby status message") + log.Infof("Sent Standby status message") nextStandbyMessageDeadline = time.Now().Add(standbyMessageTimeout) }