diff --git a/.github/workflows/flow.yml b/.github/workflows/flow.yml index 0a398c8de4..5f6899b15b 100644 --- a/.github/workflows/flow.yml +++ b/.github/workflows/flow.yml @@ -12,22 +12,17 @@ jobs: matrix: runner: [ubicloud-standard-16-ubuntu-2204-arm] runs-on: ${{ matrix.runner }} - timeout-minutes: 30 + timeout-minutes: 40 services: - pg_cdc: + catalog: image: imresamu/postgis:15-3.4-alpine ports: - - 7132:5432 + - 5432:5432 env: - POSTGRES_USER: postgres + PGUSER: postgres POSTGRES_PASSWORD: postgres POSTGRES_DB: postgres - options: >- - --name pg_cdc - --health-cmd pg_isready - --health-interval 10s - --health-timeout 5s - --health-retries 5 + steps: - name: checkout sources uses: actions/checkout@v4 @@ -43,18 +38,12 @@ jobs: go-version: "1.22" cache-dependency-path: flow/go.sum - - name: install gotestsum - run: | - go install gotest.tools/gotestsum@latest - - name: install lib-geos run: | sudo apt-get update sudo apt-get install libgeos-dev - - name: download go modules - run: | - go mod download + - run: go mod download working-directory: ./flow - name: setup gcp service account @@ -87,21 +76,27 @@ jobs: - name: create hstore extension, increase logical replication limits, and setup catalog database run: > - docker exec pg_cdc psql -h localhost -p 5432 -U postgres -c "CREATE EXTENSION hstore;" + docker exec "${{ job.services.catalog.id }}" psql -U postgres -c "CREATE EXTENSION hstore;" -c "ALTER SYSTEM SET wal_level=logical;" -c "ALTER SYSTEM SET max_replication_slots=192;" -c "ALTER SYSTEM SET max_wal_senders=256;" -c "ALTER SYSTEM SET max_connections=2048;" && - (cat ../nexus/catalog/migrations/V{?,??}__* | docker exec -i pg_cdc psql -h localhost -p 5432 -U postgres) && - docker restart pg_cdc - working-directory: ./flow + (cat ./nexus/catalog/migrations/V{?,??}__* | docker exec -i "${{ job.services.catalog.id }}" psql -U postgres) && + docker restart "${{ job.services.catalog.id }}" env: - PG_CDC: empty PGPASSWORD: postgres + - name: Install Temporal CLI + uses: temporalio/setup-temporal@v0 + - name: run tests run: | - gotestsum --format testname -- -p 24 ./... -timeout 1200s + temporal server start-dev --namespace default --headless & + go build -ldflags="-s -w" -o peer-flow + temporal operator search-attribute create --name MirrorName --type Text --namespace default + ./peer-flow worker & + ./peer-flow snapshot-worker & + go test -p 32 ./... -timeout 1200s working-directory: ./flow env: AWS_ACCESS_KEY_ID: ${{ secrets.AWS_ACCESS_KEY_ID }} @@ -122,7 +117,7 @@ jobs: SQLSERVER_PASSWORD: ${{ secrets.SQLSERVER_PASSWORD }} SQLSERVER_DB: ${{ secrets.SQLSERVER_DB }} PEERDB_CATALOG_HOST: localhost - PEERDB_CATALOG_PORT: 7132 + PEERDB_CATALOG_PORT: 5432 PEERDB_CATALOG_USER: postgres PEERDB_CATALOG_PASSWORD: postgres PEERDB_CATALOG_DATABASE: postgres diff --git a/flow/alerting/alerting.go b/flow/alerting/alerting.go index 4e95f1d5d1..a837f58889 100644 --- a/flow/alerting/alerting.go +++ b/flow/alerting/alerting.go @@ -3,7 +3,6 @@ package alerting import ( "context" "encoding/json" - "errors" "fmt" "log/slog" "time" @@ -51,14 +50,14 @@ func (a *Alerter) registerSendersFromPool(ctx context.Context) ([]*slackAlertSen } // doesn't take care of closing pool, needs to be done externally. -func NewAlerter(catalogPool *pgxpool.Pool) (*Alerter, error) { +func NewAlerter(catalogPool *pgxpool.Pool) *Alerter { if catalogPool == nil { - return nil, errors.New("catalog pool is nil for Alerter") + panic("catalog pool is nil for Alerter") } return &Alerter{ catalogPool: catalogPool, - }, nil + } } func (a *Alerter) AlertIfSlotLag(ctx context.Context, peerName string, slotInfo *protos.SlotInfo) { diff --git a/flow/cmd/api.go b/flow/cmd/api.go index f24178efda..5803d42317 100644 --- a/flow/cmd/api.go +++ b/flow/cmd/api.go @@ -1,4 +1,4 @@ -package main +package cmd import ( "context" diff --git a/flow/cmd/cert.go b/flow/cmd/cert.go index c750378926..9031d55b6c 100644 --- a/flow/cmd/cert.go +++ b/flow/cmd/cert.go @@ -1,4 +1,4 @@ -package main +package cmd import ( "crypto/tls" diff --git a/flow/cmd/handler.go b/flow/cmd/handler.go index 6519373181..f4a7a7cc62 100644 --- a/flow/cmd/handler.go +++ b/flow/cmd/handler.go @@ -1,4 +1,4 @@ -package main +package cmd import ( "context" diff --git a/flow/cmd/mirror_status.go b/flow/cmd/mirror_status.go index 70dee3c6eb..e8160277f5 100644 --- a/flow/cmd/mirror_status.go +++ b/flow/cmd/mirror_status.go @@ -1,4 +1,4 @@ -package main +package cmd import ( "context" diff --git a/flow/cmd/peer_data.go b/flow/cmd/peer_data.go index eb45003b41..f4846280ed 100644 --- a/flow/cmd/peer_data.go +++ b/flow/cmd/peer_data.go @@ -1,4 +1,4 @@ -package main +package cmd import ( "context" diff --git a/flow/cmd/snapshot_worker.go b/flow/cmd/snapshot_worker.go index 07cf5a19dc..eb9021de1a 100644 --- a/flow/cmd/snapshot_worker.go +++ b/flow/cmd/snapshot_worker.go @@ -1,4 +1,4 @@ -package main +package cmd import ( "context" @@ -25,7 +25,7 @@ type SnapshotWorkerOptions struct { TemporalKey string } -func SnapshotWorkerMain(opts *SnapshotWorkerOptions) error { +func SnapshotWorkerMain(opts *SnapshotWorkerOptions) (client.Client, worker.Worker, error) { clientOptions := client.Options{ HostPort: opts.TemporalHostPort, Namespace: opts.TemporalNamespace, @@ -35,7 +35,7 @@ func SnapshotWorkerMain(opts *SnapshotWorkerOptions) error { if opts.TemporalCert != "" && opts.TemporalKey != "" { certs, err := Base64DecodeCertAndKey(opts.TemporalCert, opts.TemporalKey) if err != nil { - return fmt.Errorf("unable to process certificate and key: %w", err) + return nil, nil, fmt.Errorf("unable to process certificate and key: %w", err) } connOptions := client.ConnectionOptions{ @@ -47,37 +47,29 @@ func SnapshotWorkerMain(opts *SnapshotWorkerOptions) error { clientOptions.ConnectionOptions = connOptions } + conn, err := utils.GetCatalogConnectionPoolFromEnv(context.Background()) + if err != nil { + return nil, nil, fmt.Errorf("unable to create catalog connection pool: %w", err) + } + c, err := client.Dial(clientOptions) if err != nil { - return fmt.Errorf("unable to create Temporal client: %w", err) + return nil, nil, fmt.Errorf("unable to create Temporal client: %w", err) } - defer c.Close() taskQueue := shared.GetPeerFlowTaskQueueName(shared.SnapshotFlowTaskQueue) w := worker.New(c, taskQueue, worker.Options{ EnableSessionWorker: true, + OnFatalError: func(err error) { + slog.Error("Snapshot Worker failed", slog.Any("error", err)) + }, }) - conn, err := utils.GetCatalogConnectionPoolFromEnv(context.Background()) - if err != nil { - return fmt.Errorf("unable to create catalog connection pool: %w", err) - } - - alerter, err := alerting.NewAlerter(conn) - if err != nil { - return fmt.Errorf("unable to create alerter: %w", err) - } - w.RegisterWorkflow(peerflow.SnapshotFlowWorkflow) w.RegisterActivity(&activities.SnapshotActivity{ SnapshotConnections: make(map[string]activities.SlotSnapshotSignal), - Alerter: alerter, + Alerter: alerting.NewAlerter(conn), }) - err = w.Run(worker.InterruptCh()) - if err != nil { - return fmt.Errorf("worker run error: %w", err) - } - - return nil + return c, w, nil } diff --git a/flow/cmd/validate_mirror.go b/flow/cmd/validate_mirror.go index 0ef515a14e..f3169d927d 100644 --- a/flow/cmd/validate_mirror.go +++ b/flow/cmd/validate_mirror.go @@ -1,4 +1,4 @@ -package main +package cmd import ( "context" diff --git a/flow/cmd/validate_peer.go b/flow/cmd/validate_peer.go index 2ef7d6c063..5bbc1cfb2c 100644 --- a/flow/cmd/validate_peer.go +++ b/flow/cmd/validate_peer.go @@ -1,4 +1,4 @@ -package main +package cmd import ( "context" diff --git a/flow/cmd/version.go b/flow/cmd/version.go index 3338a20e6a..577ab44994 100644 --- a/flow/cmd/version.go +++ b/flow/cmd/version.go @@ -1,4 +1,4 @@ -package main +package cmd import ( "context" diff --git a/flow/cmd/worker.go b/flow/cmd/worker.go index 476a274de6..4014d47596 100644 --- a/flow/cmd/worker.go +++ b/flow/cmd/worker.go @@ -1,4 +1,4 @@ -package main +package cmd import ( "context" @@ -7,9 +7,7 @@ import ( "log" "log/slog" "os" - "os/signal" "runtime" - "syscall" "github.com/grafana/pyroscope-go" "go.temporal.io/sdk/client" @@ -74,22 +72,11 @@ func setupPyroscope(opts *WorkerOptions) { } } -func WorkerMain(opts *WorkerOptions) error { +func WorkerMain(opts *WorkerOptions) (client.Client, worker.Worker, error) { if opts.EnableProfiling { setupPyroscope(opts) } - go func() { - sigs := make(chan os.Signal, 1) - signal.Notify(sigs, syscall.SIGQUIT) - buf := make([]byte, 1<<20) - for { - <-sigs - stacklen := runtime.Stack(buf, true) - log.Printf("=== received SIGQUIT ===\n*** goroutine dump...\n%s\n*** end\n", buf[:stacklen]) - } - }() - clientOptions := client.Options{ HostPort: opts.TemporalHostPort, Namespace: opts.TemporalNamespace, @@ -100,7 +87,7 @@ func WorkerMain(opts *WorkerOptions) error { slog.Info("Using temporal certificate/key for authentication") certs, err := Base64DecodeCertAndKey(opts.TemporalCert, opts.TemporalKey) if err != nil { - return fmt.Errorf("unable to process certificate and key: %w", err) + return nil, nil, fmt.Errorf("unable to process certificate and key: %w", err) } connOptions := client.ConnectionOptions{ TLS: &tls.Config{ @@ -113,37 +100,29 @@ func WorkerMain(opts *WorkerOptions) error { conn, err := utils.GetCatalogConnectionPoolFromEnv(context.Background()) if err != nil { - return fmt.Errorf("unable to create catalog connection pool: %w", err) + return nil, nil, fmt.Errorf("unable to create catalog connection pool: %w", err) } c, err := client.Dial(clientOptions) if err != nil { - return fmt.Errorf("unable to create Temporal client: %w", err) + return nil, nil, fmt.Errorf("unable to create Temporal client: %w", err) } slog.Info("Created temporal client") - defer c.Close() taskQueue := shared.GetPeerFlowTaskQueueName(shared.PeerFlowTaskQueue) w := worker.New(c, taskQueue, worker.Options{ EnableSessionWorker: true, + OnFatalError: func(err error) { + slog.Error("Peerflow Worker failed", slog.Any("error", err)) + }, }) peerflow.RegisterFlowWorkerWorkflows(w) - alerter, err := alerting.NewAlerter(conn) - if err != nil { - return fmt.Errorf("unable to create alerter: %w", err) - } - w.RegisterActivity(&activities.FlowableActivity{ CatalogPool: conn, - Alerter: alerter, + Alerter: alerting.NewAlerter(conn), CdcCache: make(map[string]connectors.CDCPullConnector), }) - err = w.Run(worker.InterruptCh()) - if err != nil { - return fmt.Errorf("worker run error: %w", err) - } - - return nil + return c, w, nil } diff --git a/flow/connectors/postgres/postgres_schema_delta_test.go b/flow/connectors/postgres/postgres_schema_delta_test.go index 6196cbc91f..c059c36836 100644 --- a/flow/connectors/postgres/postgres_schema_delta_test.go +++ b/flow/connectors/postgres/postgres_schema_delta_test.go @@ -9,6 +9,7 @@ import ( "github.com/jackc/pgx/v5" "github.com/stretchr/testify/require" + "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" "github.com/PeerDB-io/peer-flow/e2eshared" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model/qvalue" @@ -24,13 +25,7 @@ type PostgresSchemaDeltaTestSuite struct { func SetupSuite(t *testing.T) PostgresSchemaDeltaTestSuite { t.Helper() - connector, err := NewPostgresConnector(context.Background(), &protos.PostgresConfig{ - Host: "localhost", - Port: 7132, - User: "postgres", - Password: "postgres", - Database: "postgres", - }) + connector, err := NewPostgresConnector(context.Background(), utils.GetCatalogPostgresConfigFromEnv()) require.NoError(t, err) setupTx, err := connector.conn.Begin(context.Background()) diff --git a/flow/connectors/postgres/qrep_bench_test.go b/flow/connectors/postgres/qrep_bench_test.go index 406d74a00b..f5882bf299 100644 --- a/flow/connectors/postgres/qrep_bench_test.go +++ b/flow/connectors/postgres/qrep_bench_test.go @@ -4,21 +4,14 @@ import ( "context" "testing" - "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" ) func BenchmarkQRepQueryExecutor(b *testing.B) { query := "SELECT * FROM bench.large_table" ctx := context.Background() - connector, err := NewPostgresConnector(ctx, - &protos.PostgresConfig{ - Host: "localhost", - Port: 7132, - User: "postgres", - Password: "postgres", - Database: "postgres", - }) + connector, err := NewPostgresConnector(ctx, utils.GetCatalogPostgresConfigFromEnv()) if err != nil { b.Fatalf("failed to create connection: %v", err) } diff --git a/flow/connectors/postgres/qrep_partition_test.go b/flow/connectors/postgres/qrep_partition_test.go index 9ed69968a4..0512c68415 100644 --- a/flow/connectors/postgres/qrep_partition_test.go +++ b/flow/connectors/postgres/qrep_partition_test.go @@ -11,6 +11,7 @@ import ( "github.com/stretchr/testify/assert" "go.temporal.io/sdk/log" + "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/shared" ) @@ -63,7 +64,7 @@ func newTestCaseForCTID(schema string, name string, rows uint32, expectedNum int } func TestGetQRepPartitions(t *testing.T) { - const connStr = "postgres://postgres:postgres@localhost:7132/postgres" + connStr := utils.GetCatalogConnectionStringFromEnv() // Setup the DB config, err := pgx.ParseConfig(connStr) diff --git a/flow/connectors/postgres/qrep_query_executor_test.go b/flow/connectors/postgres/qrep_query_executor_test.go index 24eb88a0a9..c8ceaee9a2 100644 --- a/flow/connectors/postgres/qrep_query_executor_test.go +++ b/flow/connectors/postgres/qrep_query_executor_test.go @@ -11,20 +11,13 @@ import ( "github.com/google/uuid" "github.com/jackc/pgx/v5" - "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" ) func setupDB(t *testing.T) (*PostgresConnector, string) { t.Helper() - connector, err := NewPostgresConnector(context.Background(), - &protos.PostgresConfig{ - Host: "localhost", - Port: 7132, - User: "postgres", - Password: "postgres", - Database: "postgres", - }) + connector, err := NewPostgresConnector(context.Background(), utils.GetCatalogPostgresConfigFromEnv()) if err != nil { t.Fatalf("unable to create connector: %v", err) } diff --git a/flow/e2e/bigquery/peer_flow_bq_test.go b/flow/e2e/bigquery/peer_flow_bq_test.go index 74a1949ff7..14e95deda2 100644 --- a/flow/e2e/bigquery/peer_flow_bq_test.go +++ b/flow/e2e/bigquery/peer_flow_bq_test.go @@ -194,20 +194,18 @@ func setupSuite(t *testing.T) PeerFlowE2ETestSuiteBQ { } func (s PeerFlowE2ETestSuiteBQ) Test_Invalid_Connection_Config() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, nil, nil) - - // Verify workflow completes - require.True(s.t, env.IsWorkflowCompleted()) - err := env.GetWorkflowError() + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, nil, nil) + e2e.EnvWaitForFinished(s.t, env, 3*time.Minute) + require.Error(s.t, env.Error()) // assert that error contains "invalid connection configs" - require.Contains(s.t, err.Error(), "invalid connection configs") + require.Contains(s.t, env.Error().Error(), "invalid connection configs") } func (s PeerFlowE2ETestSuiteBQ) Test_Complete_Flow_No_Data() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcTableName := s.attachSchemaSuffix("test_no_data") dstTableName := "test_no_data" @@ -231,18 +229,15 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Complete_Flow_No_Data() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 1 - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - e2e.EnvWaitForEqualTables(env, s, "create table", dstTableName, "id,key,value") - env.CancelWorkflow() - }() - - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + e2e.EnvWaitForEqualTables(env, s, "create table", dstTableName, "id,key,value") + env.Cancel() e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuiteBQ) Test_Char_ColType_Error() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcTableName := s.attachSchemaSuffix("test_char_coltype") dstTableName := "test_char_coltype" @@ -266,13 +261,10 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Char_ColType_Error() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 1 - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - e2e.EnvWaitForEqualTables(env, s, "create table", dstTableName, "id,key,value") - env.CancelWorkflow() - }() - - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + e2e.EnvWaitForEqualTables(env, s, "create table", dstTableName, "id,key,value") + env.Cancel() e2e.RequireEnvCanceled(s.t, env) } @@ -280,7 +272,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Char_ColType_Error() { // 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 := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcTableName := s.attachSchemaSuffix("test_simple_flow_bq") dstTableName := "test_simple_flow_bq" @@ -304,29 +296,28 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Complete_Simple_Flow_BQ() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - // insert 10 rows into the source table - for i := range 10 { - testKey := fmt.Sprintf("test_key_%d", i) - testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + + // insert 10 rows into the source table + for i := range 10 { + testKey := fmt.Sprintf("test_key_%d", i) + testValue := fmt.Sprintf("test_value_%d", i) + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(key, value) VALUES ($1, $2) `, srcTableName), testKey, testValue) - e2e.EnvNoError(s.t, env, err) - } - s.t.Log("Inserted 10 rows into the source table") + e2e.EnvNoError(s.t, env, err) + } + s.t.Log("Inserted 10 rows into the source table") - e2e.EnvWaitForEqualTables(env, s, "normalize inserts", dstTableName, "id,key,value") - env.CancelWorkflow() - }() + e2e.EnvWaitForEqualTables(env, s, "normalize inserts", dstTableName, "id,key,value") - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) + env.Cancel() e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuiteBQ) Test_Toast_BQ() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcTableName := s.attachSchemaSuffix("test_toast_bq_1") dstTableName := "test_toast_bq_1" @@ -351,17 +342,17 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Toast_BQ() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and execute a transaction touching toast columns - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - /* - Executing a transaction which - 1. changes both toast column - 2. changes no toast column - 2. changes 1 toast column - */ - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + /* + Executing a transaction which + 1. changes both toast column + 2. changes no toast column + 2. changes 1 toast column + */ + _, err = s.Conn().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); @@ -369,19 +360,16 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Toast_BQ() { UPDATE %s SET t1='dummy' WHERE id=2; END; `, srcTableName, srcTableName, srcTableName)) - e2e.EnvNoError(s.t, env, err) - s.t.Log("Executed a transaction touching toast columns") - - e2e.EnvWaitForEqualTables(env, s, "normalize tx", dstTableName, "id,t1,t2,k") - env.CancelWorkflow() - }() + e2e.EnvNoError(s.t, env, err) + s.t.Log("Executed a transaction touching toast columns") - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.EnvWaitForEqualTables(env, s, "normalize tx", dstTableName, "id,t1,t2,k") + env.Cancel() e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_1_BQ() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcTableName := s.attachSchemaSuffix("test_toast_bq_3") dstTableName := "test_toast_bq_3" @@ -406,12 +394,12 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_1_BQ() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and execute a transaction touching toast columns - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - // complex transaction with random DMLs on a table with toast columns - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + // complex transaction with random DMLs on a table with toast columns + _, err = s.Conn().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); @@ -429,20 +417,17 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_1_BQ() { DELETE FROM %s WHERE id=2; END; `, srcTableName, srcTableName, srcTableName, srcTableName, srcTableName, srcTableName, - srcTableName, srcTableName, srcTableName, srcTableName, srcTableName, srcTableName)) - e2e.EnvNoError(s.t, env, err) - s.t.Log("Executed a transaction touching toast columns") + srcTableName, srcTableName, srcTableName, srcTableName, srcTableName, srcTableName)) + e2e.EnvNoError(s.t, env, err) + s.t.Log("Executed a transaction touching toast columns") - e2e.EnvWaitForEqualTables(env, s, "normalizing tx", dstTableName, "id,t1,t2,k") - env.CancelWorkflow() - }() - - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.EnvWaitForEqualTables(env, s, "normalizing tx", dstTableName, "id,t1,t2,k") + env.Cancel() e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_2_BQ() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcTableName := s.attachSchemaSuffix("test_toast_bq_4") dstTableName := "test_toast_bq_4" @@ -466,12 +451,12 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_2_BQ() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and execute a transaction touching toast columns - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - // complex transaction with random DMLs on a table with toast columns - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + // complex transaction with random DMLs on a table with toast columns + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` BEGIN; INSERT INTO %s(t1,k) SELECT random_string(9000), 1 FROM generate_series(1,1); @@ -484,18 +469,15 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_2_BQ() { UPDATE %s SET k=4 WHERE id=1; END; `, srcTableName, srcTableName, srcTableName, srcTableName, srcTableName, srcTableName)) - e2e.EnvNoError(s.t, env, err) - s.t.Log("Executed a transaction touching toast columns") - e2e.EnvWaitForEqualTables(env, s, "normalizing tx", dstTableName, "id,t1,k") - env.CancelWorkflow() - }() - - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.EnvNoError(s.t, env, err) + s.t.Log("Executed a transaction touching toast columns") + e2e.EnvWaitForEqualTables(env, s, "normalizing tx", dstTableName, "id,t1,k") + env.Cancel() e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_3_BQ() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcTableName := s.attachSchemaSuffix("test_toast_bq_5") dstTableName := "test_toast_bq_5" @@ -520,15 +502,15 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_3_BQ() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and execute a transaction touching toast columns - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - /* - transaction updating a single row - multiple times with changed/unchanged toast columns - */ - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + /* + transaction updating a single row + multiple times with changed/unchanged toast columns + */ + _, err = s.Conn().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); @@ -537,18 +519,15 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_3_BQ() { UPDATE %s SET t2='dummy' WHERE id=1; END; `, srcTableName, srcTableName, srcTableName, srcTableName)) - e2e.EnvNoError(s.t, env, err) - s.t.Log("Executed a transaction touching toast columns") - e2e.EnvWaitForEqualTables(env, s, "normalizing tx", dstTableName, "id,t1,t2,k") - env.CancelWorkflow() - }() - - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.EnvNoError(s.t, env, err) + s.t.Log("Executed a transaction touching toast columns") + e2e.EnvWaitForEqualTables(env, s, "normalizing tx", dstTableName, "id,t1,t2,k") + env.Cancel() e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuiteBQ) Test_Types_BQ() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcTableName := s.attachSchemaSuffix("test_types_bq") dstTableName := "test_types_bq" @@ -580,12 +559,12 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Types_BQ() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and execute a transaction touching toast columns - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - /* test inserting various types*/ - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + /* test inserting various types*/ + _, err = s.Conn().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,'10.0.0.0/32'::inet,1, @@ -605,46 +584,43 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Types_BQ() { '{true, false}'::boolean[], '{1, 2}'::smallint[]; `, srcTableName)) - e2e.EnvNoError(s.t, env, err) - - e2e.EnvWaitFor(s.t, env, 2*time.Minute, "normalize types", func() bool { - 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", "c45", "c46", "c47", "c48", - "c49", "c50", "c51", - }) - if err != nil { - s.t.Log(err) - return false - } - - // check if JSON on bigquery side is a good JSON - if err := s.checkJSONValue(dstTableName, "c17", "sai", "-8.021390374331551"); err != nil { - return false - } + e2e.EnvNoError(s.t, env, err) + + e2e.EnvWaitFor(s.t, env, 2*time.Minute, "normalize types", func() bool { + 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", "c45", "c46", "c47", "c48", + "c49", "c50", "c51", + }) + if err != nil { + s.t.Log(err) + return false + } - // check if HSTORE on bigquery side is a good JSON - if err := s.checkJSONValue(dstTableName, "c46", "key1", "\"value1\""); err != nil { - return false - } - if err := s.checkJSONValue(dstTableName, "c46", "key2", "null"); err != nil { - return false - } + // check if JSON on bigquery side is a good JSON + if err := s.checkJSONValue(dstTableName, "c17", "sai", "-8.021390374331551"); err != nil { + return false + } - return noNulls - }) + // check if HSTORE on bigquery side is a good JSON + if err := s.checkJSONValue(dstTableName, "c46", "key1", "\"value1\""); err != nil { + return false + } + if err := s.checkJSONValue(dstTableName, "c46", "key2", "null"); err != nil { + return false + } - env.CancelWorkflow() - }() + return noNulls + }) - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) + env.Cancel() e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuiteBQ) Test_NaN_Doubles_BQ() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcTableName := s.attachSchemaSuffix("test_nans_bq") dstTableName := "test_nans_bq" @@ -663,29 +639,28 @@ func (s PeerFlowE2ETestSuiteBQ) Test_NaN_Doubles_BQ() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and execute a transaction touching toast columns - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - /* test inserting various types*/ - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + + // test inserting various types + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s SELECT 2, 'NaN'::double precision, '{NaN, Infinity, -Infinity}'; `, srcTableName)) - e2e.EnvNoError(s.t, env, err) + e2e.EnvNoError(s.t, env, err) - e2e.EnvWaitFor(s.t, env, 2*time.Minute, "normalize weird floats", func() bool { - good, err := s.bqHelper.CheckDoubleValues(dstTableName, "c1", "c2") - return err == nil && good - }) - env.CancelWorkflow() - }() + e2e.EnvWaitFor(s.t, env, 2*time.Minute, "normalize weird floats", func() bool { + good, err := s.bqHelper.CheckDoubleValues(dstTableName, "c1", "c2") + return err == nil && good + }) - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) + env.Cancel() e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuiteBQ) Test_Invalid_Geo_BQ_Avro_CDC() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcTableName := s.attachSchemaSuffix("test_invalid_geo_bq_avro_cdc") dstTableName := "test_invalid_geo_bq_avro_cdc" @@ -709,64 +684,61 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Invalid_Geo_BQ_Avro_CDC() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and then insert 10 rows into the source table - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - // insert 4 invalid shapes and 6 valid shapes into the source table - for range 4 { - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + // insert 4 invalid shapes and 6 valid shapes into the source table + for range 4 { + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s (line,"polyPoly") VALUES ($1,$2) `, srcTableName), "010200000001000000000000000000F03F0000000000000040", - "0103000020e6100000010000000c0000001a8361d35dc64140afdb8d2b1bc3c9bf1b8ed4685fc641405ba64c"+ - "579dc2c9bf6a6ad95a5fc64140cd82767449c2c9bf9570fbf85ec641408a07944db9c2c9bf729a18a55ec6414021b8b748c7c2c9bfba46de4c"+ - "5fc64140f2567052abc2c9bf2df9c5925fc641409394e16573c2c9bf2df9c5925fc6414049eceda9afc1c9bfdd1cc1a05fc64140fe43faedebc0"+ - "c9bf4694f6065fc64140fe43faedebc0c9bfffe7305f5ec641406693d6f2ddc0c9bf1a8361d35dc64140afdb8d2b1bc3c9bf", - ) - e2e.EnvNoError(s.t, env, err) - } - s.t.Log("Inserted 4 invalid geography rows into the source table") - for range 6 { - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + "0103000020e6100000010000000c0000001a8361d35dc64140afdb8d2b1bc3c9bf1b8ed4685fc641405ba64c"+ + "579dc2c9bf6a6ad95a5fc64140cd82767449c2c9bf9570fbf85ec641408a07944db9c2c9bf729a18a55ec6414021b8b748c7c2c9bfba46de4c"+ + "5fc64140f2567052abc2c9bf2df9c5925fc641409394e16573c2c9bf2df9c5925fc6414049eceda9afc1c9bfdd1cc1a05fc64140fe43faedebc0"+ + "c9bf4694f6065fc64140fe43faedebc0c9bfffe7305f5ec641406693d6f2ddc0c9bf1a8361d35dc64140afdb8d2b1bc3c9bf", + ) + e2e.EnvNoError(s.t, env, err) + } + s.t.Log("Inserted 4 invalid geography rows into the source table") + for range 6 { + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s (line,"polyPoly") VALUES ($1,$2) `, srcTableName), "010200000002000000000000000000F03F000000000000004000000000000008400000000000001040", - "010300000001000000050000000000000000000000000000000000000000000000"+ - "00000000000000000000f03f000000000000f03f000000000000f03f0000000000"+ - "00f03f000000000000000000000000000000000000000000000000") - e2e.EnvNoError(s.t, env, err) - } - s.t.Log("Inserted 6 valid geography rows and 10 total rows into source") - - e2e.EnvWaitFor(s.t, env, 2*time.Minute, "normalize shapes", func() bool { - // We inserted 4 invalid shapes in each, - // which should be filtered out as null on destination. - lineCount, err := s.bqHelper.countRowsWithDataset(s.bqHelper.Config.DatasetId, dstTableName, "line") - if err != nil { - return false - } + "010300000001000000050000000000000000000000000000000000000000000000"+ + "00000000000000000000f03f000000000000f03f000000000000f03f0000000000"+ + "00f03f000000000000000000000000000000000000000000000000") + e2e.EnvNoError(s.t, env, err) + } + s.t.Log("Inserted 6 valid geography rows and 10 total rows into source") - polyCount, err := s.bqHelper.countRowsWithDataset(s.bqHelper.Config.DatasetId, dstTableName, "`polyPoly`") - if err != nil { - return false - } + e2e.EnvWaitFor(s.t, env, 2*time.Minute, "normalize shapes", func() bool { + // We inserted 4 invalid shapes in each, + // which should be filtered out as null on destination. + lineCount, err := s.bqHelper.countRowsWithDataset(s.bqHelper.Config.DatasetId, dstTableName, "line") + if err != nil { + return false + } - if lineCount != 6 || polyCount != 6 { - s.t.Logf("wrong counts, expect 6 lines 6 polies, not %d lines %d polies", lineCount, polyCount) - return false - } else { - return true - } - }) + polyCount, err := s.bqHelper.countRowsWithDataset(s.bqHelper.Config.DatasetId, dstTableName, "`polyPoly`") + if err != nil { + return false + } - env.CancelWorkflow() - }() + if lineCount != 6 || polyCount != 6 { + s.t.Logf("wrong counts, expect 6 lines 6 polies, not %d lines %d polies", lineCount, polyCount) + return false + } else { + return true + } + }) - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) + env.Cancel() e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuiteBQ) Test_Multi_Table_BQ() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcTable1Name := s.attachSchemaSuffix("test1_bq") dstTable1Name := "test1_bq" @@ -789,41 +761,38 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Multi_Table_BQ() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and execute a transaction touching toast columns - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - /* inserting across multiple tables*/ - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + /* inserting across multiple tables*/ + _, err = s.Conn().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)) - e2e.EnvNoError(s.t, env, err) - s.t.Log("Executed an insert on two tables") - - e2e.EnvWaitFor(s.t, env, 2*time.Minute, "normalize both tables", func() bool { - count1, err := s.bqHelper.countRows(dstTable1Name) - if err != nil { - return false - } - count2, err := s.bqHelper.countRows(dstTable2Name) - if err != nil { - return false - } + e2e.EnvNoError(s.t, env, err) + s.t.Log("Executed an insert on two tables") - return count1 == 1 && count2 == 1 - }) + e2e.EnvWaitFor(s.t, env, 2*time.Minute, "normalize both tables", func() bool { + count1, err := s.bqHelper.countRows(dstTable1Name) + if err != nil { + return false + } + count2, err := s.bqHelper.countRows(dstTable2Name) + if err != nil { + return false + } - env.CancelWorkflow() - }() + return count1 == 1 && count2 == 1 + }) - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) + env.Cancel() e2e.RequireEnvCanceled(s.t, env) } // TODO: not checking schema exactly, add later func (s PeerFlowE2ETestSuiteBQ) Test_Simple_Schema_Changes_BQ() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) tableName := "test_simple_schema_changes" srcTableName := s.attachSchemaSuffix(tableName) @@ -846,66 +815,63 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Simple_Schema_Changes_BQ() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and then insert and mutate schema repeatedly. - go func() { - // insert first row. - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + // insert first row. + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c1) VALUES (1)`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - s.t.Log("Inserted initial row in the source table") + e2e.EnvNoError(s.t, env, err) + s.t.Log("Inserted initial row in the source table") - e2e.EnvWaitForEqualTables(env, s, "normalize insert", tableName, "id,c1") + e2e.EnvWaitForEqualTables(env, s, "normalize insert", tableName, "id,c1") - // alter source table, add column c2 and insert another row. - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + // alter source table, add column c2 and insert another row. + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` ALTER TABLE %s ADD COLUMN c2 BIGINT`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - s.t.Log("Altered source table, added column c2") - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + s.t.Log("Altered source table, added column c2") + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c1,c2) VALUES (2,2)`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - s.t.Log("Inserted row with added c2 in the source table") + e2e.EnvNoError(s.t, env, err) + s.t.Log("Inserted row with added c2 in the source table") - // verify we got our two rows, if schema did not match up it will error. - e2e.EnvWaitForEqualTables(env, s, "normalize altered row", tableName, "id,c1,c2") + // verify we got our two rows, if schema did not match up it will error. + e2e.EnvWaitForEqualTables(env, s, "normalize altered row", tableName, "id,c1,c2") - // alter source table, add column c3, drop column c2 and insert another row. - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + // alter source table, add column c3, drop column c2 and insert another row. + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` ALTER TABLE %s DROP COLUMN c2, ADD COLUMN c3 BIGINT`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - s.t.Log("Altered source table, dropped column c2 and added column c3") - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + s.t.Log("Altered source table, dropped column c2 and added column c3") + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c1,c3) VALUES (3,3)`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - s.t.Log("Inserted row with added c3 in the source table") + e2e.EnvNoError(s.t, env, err) + s.t.Log("Inserted row with added c3 in the source table") - // verify we got our two rows, if schema did not match up it will error. - e2e.EnvWaitForEqualTables(env, s, "normalize altered row", tableName, "id,c1,c3") + // verify we got our two rows, if schema did not match up it will error. + e2e.EnvWaitForEqualTables(env, s, "normalize altered row", tableName, "id,c1,c3") - // alter source table, drop column c3 and insert another row. - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + // alter source table, drop column c3 and insert another row. + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` ALTER TABLE %s DROP COLUMN c3`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - s.t.Log("Altered source table, dropped column c3") - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + s.t.Log("Altered source table, dropped column c3") + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c1) VALUES (4)`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - s.t.Log("Inserted row after dropping all columns in the source table") + e2e.EnvNoError(s.t, env, err) + s.t.Log("Inserted row after dropping all columns in the source table") - // verify we got our two rows, if schema did not match up it will error. - e2e.EnvWaitForEqualTables(env, s, "normalize drop column", tableName, "id,c1") + // verify we got our two rows, if schema did not match up it will error. + e2e.EnvWaitForEqualTables(env, s, "normalize drop column", tableName, "id,c1") - env.CancelWorkflow() - }() - - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) + env.Cancel() e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuiteBQ) Test_Composite_PKey_BQ() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) tableName := "test_simple_cpkey" srcTableName := s.attachSchemaSuffix("test_simple_cpkey") @@ -931,40 +897,37 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Composite_PKey_BQ() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and then insert, update and delete rows in the table. - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - // insert 10 rows into the source table - for i := range 10 { - testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + // insert 10 rows into the source table + for i := range 10 { + testValue := fmt.Sprintf("test_value_%d", i) + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c2,t) VALUES ($1,$2) `, srcTableName), i, testValue) - e2e.EnvNoError(s.t, env, err) - } - s.t.Log("Inserted 10 rows into the source table") - - // verify we got our 10 rows - e2e.EnvWaitForEqualTables(env, s, "normalize table", tableName, "id,c1,c2,t") - - _, err := s.Conn().Exec(context.Background(), - fmt.Sprintf(`UPDATE %s SET c1=c1+1 WHERE MOD(c2,2)=$1`, srcTableName), 1) - e2e.EnvNoError(s.t, env, err) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=$1`, srcTableName), 0) e2e.EnvNoError(s.t, env, err) + } + s.t.Log("Inserted 10 rows into the source table") - e2e.EnvWaitForEqualTables(env, s, "normalize update", tableName, "id,c1,c2,t") + // verify we got our 10 rows + e2e.EnvWaitForEqualTables(env, s, "normalize table", tableName, "id,c1,c2,t") - env.CancelWorkflow() - }() + _, err = s.Conn().Exec(context.Background(), + fmt.Sprintf(`UPDATE %s SET c1=c1+1 WHERE MOD(c2,2)=$1`, srcTableName), 1) + e2e.EnvNoError(s.t, env, err) + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=$1`, srcTableName), 0) + e2e.EnvNoError(s.t, env, err) - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.EnvWaitForEqualTables(env, s, "normalize update", tableName, "id,c1,c2,t") + + env.Cancel() e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuiteBQ) Test_Composite_PKey_Toast_1_BQ() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcTableName := s.attachSchemaSuffix("test_cpkey_toast1") dstTableName := "test_cpkey_toast1" @@ -991,41 +954,39 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Composite_PKey_Toast_1_BQ() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and then insert, update and delete rows in the table. - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - rowsTx, err := s.Conn().Begin(context.Background()) - e2e.EnvNoError(s.t, env, err) - - // insert 10 rows into the source table - for i := range 10 { - testValue := fmt.Sprintf("test_value_%d", i) - _, err = rowsTx.Exec(context.Background(), fmt.Sprintf(` + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + rowsTx, err := s.Conn().Begin(context.Background()) + e2e.EnvNoError(s.t, env, err) + + // insert 10 rows into the source table + for i := range 10 { + testValue := fmt.Sprintf("test_value_%d", i) + _, err = rowsTx.Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c2,t,t2) VALUES ($1,$2,random_string(9000)) `, srcTableName), i, testValue) - e2e.EnvNoError(s.t, env, err) - } - s.t.Log("Inserted 10 rows into the source table") - - _, err = rowsTx.Exec(context.Background(), - fmt.Sprintf(`UPDATE %s SET c1=c1+1 WHERE MOD(c2,2)=$1`, srcTableName), 1) - e2e.EnvNoError(s.t, env, err) - _, err = rowsTx.Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=$1`, srcTableName), 0) e2e.EnvNoError(s.t, env, err) + } + s.t.Log("Inserted 10 rows into the source table") - err = rowsTx.Commit(context.Background()) - e2e.EnvNoError(s.t, env, err) - e2e.EnvWaitForEqualTables(env, s, "normalize tx", dstTableName, "id,c1,c2,t,t2") - env.CancelWorkflow() - }() + _, err = rowsTx.Exec(context.Background(), + fmt.Sprintf(`UPDATE %s SET c1=c1+1 WHERE MOD(c2,2)=$1`, srcTableName), 1) + e2e.EnvNoError(s.t, env, err) + _, err = rowsTx.Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=$1`, srcTableName), 0) + e2e.EnvNoError(s.t, env, err) + + err = rowsTx.Commit(context.Background()) + e2e.EnvNoError(s.t, env, err) + e2e.EnvWaitForEqualTables(env, s, "normalize tx", dstTableName, "id,c1,c2,t,t2") - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) + env.Cancel() e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuiteBQ) Test_Composite_PKey_Toast_2_BQ() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) tableName := "test_cpkey_toast2" srcTableName := s.attachSchemaSuffix("test_cpkey_toast2") @@ -1052,38 +1013,35 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Composite_PKey_Toast_2_BQ() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and then insert, update and delete rows in the table. - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - // insert 10 rows into the source table - for i := range 10 { - testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + // insert 10 rows into the source table + for i := range 10 { + testValue := fmt.Sprintf("test_value_%d", i) + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c2,t,t2) VALUES ($1,$2,random_string(9000)) `, srcTableName), i, testValue) - e2e.EnvNoError(s.t, env, err) - } - s.t.Log("Inserted 10 rows into the source table") - - e2e.EnvWaitForEqualTables(env, s, "normalize table", tableName, "id,c2,t,t2") - _, err = s.Conn().Exec(context.Background(), - fmt.Sprintf(`UPDATE %s SET c1=c1+1 WHERE MOD(c2,2)=$1`, srcTableName), 1) - e2e.EnvNoError(s.t, env, err) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=$1`, srcTableName), 0) e2e.EnvNoError(s.t, env, err) - e2e.EnvWaitForEqualTables(env, s, "normalize update", tableName, "id,c2,t,t2") + } + s.t.Log("Inserted 10 rows into the source table") - env.CancelWorkflow() - }() + e2e.EnvWaitForEqualTables(env, s, "normalize table", tableName, "id,c2,t,t2") + _, err = s.Conn().Exec(context.Background(), + fmt.Sprintf(`UPDATE %s SET c1=c1+1 WHERE MOD(c2,2)=$1`, srcTableName), 1) + e2e.EnvNoError(s.t, env, err) + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=$1`, srcTableName), 0) + e2e.EnvNoError(s.t, env, err) + e2e.EnvWaitForEqualTables(env, s, "normalize update", tableName, "id,c2,t,t2") - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) + env.Cancel() e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuiteBQ) Test_Columns_BQ() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcTableName := s.attachSchemaSuffix("test_peerdb_cols") dstTableName := "test_peerdb_cols_dst" @@ -1106,34 +1064,31 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Columns_BQ() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - // insert 1 row into the source table - testKey := fmt.Sprintf("test_key_%d", 1) - testValue := fmt.Sprintf("test_value_%d", 1) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + // insert 1 row into the source table + testKey := fmt.Sprintf("test_key_%d", 1) + testValue := fmt.Sprintf("test_value_%d", 1) + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(key, value) VALUES ($1, $2) `, srcTableName), testKey, testValue) - e2e.EnvNoError(s.t, env, err) + e2e.EnvNoError(s.t, env, err) - // delete that row - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + // delete that row + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` DELETE FROM %s WHERE id=1 `, srcTableName)) - e2e.EnvNoError(s.t, env, err) - - e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize insert/delete", func() bool { - return s.checkPeerdbColumns(dstTableName, true) == nil - }) - env.CancelWorkflow() - }() + e2e.EnvNoError(s.t, env, err) - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize insert/delete", func() bool { + return s.checkPeerdbColumns(dstTableName, true) == nil + }) + env.Cancel() e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuiteBQ) Test_Multi_Table_Multi_Dataset_BQ() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcTable1Name := s.attachSchemaSuffix("test1_bq") dstTable1Name := "test1_bq" @@ -1160,41 +1115,39 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Multi_Table_Multi_Dataset_BQ() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and execute a transaction touching toast columns - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - /* inserting across multiple tables*/ - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + /* inserting across multiple tables*/ + _, err = s.Conn().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)) - e2e.EnvNoError(s.t, env, err) - s.t.Log("Executed an insert on two tables") + e2e.EnvNoError(s.t, env, err) + s.t.Log("Executed an insert on two tables") - e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize multi dataset", func() bool { - count1, err := s.bqHelper.countRows(dstTable1Name) - if err != nil { - return false - } - count2, err := s.bqHelper.countRowsWithDataset(secondDataset, dstTable2Name, "") - if err != nil { - return false - } + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize multi dataset", func() bool { + count1, err := s.bqHelper.countRows(dstTable1Name) + if err != nil { + return false + } + count2, err := s.bqHelper.countRowsWithDataset(secondDataset, dstTable2Name, "") + if err != nil { + return false + } - return count1 == 1 && count2 == 1 - }) - env.CancelWorkflow() - }() + return count1 == 1 && count2 == 1 + }) - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) + env.Cancel() e2e.RequireEnvCanceled(s.t, env) require.NoError(s.t, s.bqHelper.DropDataset(secondDataset)) } func (s PeerFlowE2ETestSuiteBQ) Test_Soft_Delete_Basic() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) tableName := "test_softdel" srcName := "test_softdel_src" @@ -1231,38 +1184,35 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Soft_Delete_Basic() { MaxBatchSize: 100, } - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and then insert, update and delete rows in the table. - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, config, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c1,c2,t) VALUES (1,2,random_string(9000))`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - e2e.EnvWaitForEqualTablesWithNames(env, s, "normalize insert", srcName, tableName, "id,c1,c2,t") - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + e2e.EnvWaitForEqualTablesWithNames(env, s, "normalize insert", srcName, tableName, "id,c1,c2,t") + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` UPDATE %s SET c1=c1+4 WHERE id=1`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - e2e.EnvWaitForEqualTablesWithNames(env, s, "normalize update", srcName, tableName, "id,c1,c2,t") - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + e2e.EnvWaitForEqualTablesWithNames(env, s, "normalize update", srcName, tableName, "id,c1,c2,t") + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` DELETE FROM %s WHERE id=1`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize delete", func() bool { - pgRows, err := e2e.GetPgRows(s.conn, s.bqSuffix, srcName, "id,c1,c2,t") - if err != nil { - return false - } - rows, err := s.GetRowsWhere(tableName, "id,c1,c2,t", "NOT _PEERDB_IS_DELETED") - if err != nil { - return false - } - return e2eshared.CheckEqualRecordBatches(s.t, pgRows, rows) - }) - - env.CancelWorkflow() - }() + e2e.EnvNoError(s.t, env, err) + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize delete", func() bool { + pgRows, err := e2e.GetPgRows(s.conn, s.bqSuffix, srcName, "id,c1,c2,t") + if err != nil { + return false + } + rows, err := s.GetRowsWhere(tableName, "id,c1,c2,t", "NOT _PEERDB_IS_DELETED") + if err != nil { + return false + } + return e2eshared.CheckEqualRecordBatches(s.t, pgRows, rows) + }) - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, config, nil) + env.Cancel() e2e.RequireEnvCanceled(s.t, env) newerSyncedAtQuery := fmt.Sprintf( @@ -1274,7 +1224,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Soft_Delete_Basic() { } func (s PeerFlowE2ETestSuiteBQ) Test_Soft_Delete_IUD_Same_Batch() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) cmpTableName := s.attachSchemaSuffix("test_softdel_iud") srcTableName := cmpTableName + "_src" @@ -1311,48 +1261,45 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Soft_Delete_IUD_Same_Batch() { MaxBatchSize: 100, } - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and then insert, update and delete rows in the table. - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, config, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - insertTx, err := s.Conn().Begin(context.Background()) - e2e.EnvNoError(s.t, env, err) + insertTx, err := s.Conn().Begin(context.Background()) + e2e.EnvNoError(s.t, env, err) - _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` + _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c1,c2,t) VALUES (1,2,random_string(9000))`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` UPDATE %s SET c1=c1+4 WHERE id=1`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - // since we delete stuff, create another table to compare with - _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + // since we delete stuff, create another table to compare with + _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE %s AS SELECT * FROM %s`, cmpTableName, srcTableName)) - e2e.EnvNoError(s.t, env, err) - _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` DELETE FROM %s WHERE id=1`, srcTableName)) + e2e.EnvNoError(s.t, env, err) + + e2e.EnvNoError(s.t, env, insertTx.Commit(context.Background())) + e2e.EnvWaitForEqualTables(env, s, "normalizing tx", "test_softdel_iud", "id,c1,c2,t") + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "checking soft delete", func() bool { + newerSyncedAtQuery := fmt.Sprintf( + "SELECT COUNT(*) FROM `%s.%s` WHERE _PEERDB_IS_DELETED", + s.bqHelper.Config.DatasetId, dstTableName) + numNewRows, err := s.bqHelper.RunInt64Query(newerSyncedAtQuery) e2e.EnvNoError(s.t, env, err) + return numNewRows == 1 + }) - e2e.EnvNoError(s.t, env, insertTx.Commit(context.Background())) - e2e.EnvWaitForEqualTables(env, s, "normalizing tx", "test_softdel_iud", "id,c1,c2,t") - e2e.EnvWaitFor(s.t, env, 3*time.Minute, "checking soft delete", func() bool { - newerSyncedAtQuery := fmt.Sprintf( - "SELECT COUNT(*) FROM `%s.%s` WHERE _PEERDB_IS_DELETED", - s.bqHelper.Config.DatasetId, dstTableName) - numNewRows, err := s.bqHelper.RunInt64Query(newerSyncedAtQuery) - e2e.EnvNoError(s.t, env, err) - return numNewRows == 1 - }) - - env.CancelWorkflow() - }() - - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, config, nil) + env.Cancel() e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuiteBQ) Test_Soft_Delete_UD_Same_Batch() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcName := "test_softdel_ud_src" srcTableName := s.attachSchemaSuffix(srcName) @@ -1389,43 +1336,40 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Soft_Delete_UD_Same_Batch() { MaxBatchSize: 100, } - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and then insert, update and delete rows in the table. - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, config, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c1,c2,t) VALUES (1,2,random_string(9000))`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - e2e.EnvWaitForEqualTablesWithNames(env, s, "normalize insert", srcName, dstName, "id,c1,c2,t") + e2e.EnvNoError(s.t, env, err) + e2e.EnvWaitForEqualTablesWithNames(env, s, "normalize insert", srcName, dstName, "id,c1,c2,t") - insertTx, err := s.Conn().Begin(context.Background()) - e2e.EnvNoError(s.t, env, err) - _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` + insertTx, err := s.Conn().Begin(context.Background()) + e2e.EnvNoError(s.t, env, err) + _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` UPDATE %s SET t=random_string(10000) WHERE id=1`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` UPDATE %s SET c1=c1+4 WHERE id=1`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` DELETE FROM %s WHERE id=1`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - e2e.EnvNoError(s.t, env, insertTx.Commit(context.Background())) - - e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize transaction", func() bool { - pgRows, err := e2e.GetPgRows(s.conn, s.bqSuffix, srcName, "id,c1,c2,t") - e2e.EnvNoError(s.t, env, err) - rows, err := s.GetRowsWhere(dstName, "id,c1,c2,t", "NOT _PEERDB_IS_DELETED") - if err != nil { - return false - } - return e2eshared.CheckEqualRecordBatches(s.t, pgRows, rows) - }) + e2e.EnvNoError(s.t, env, err) + e2e.EnvNoError(s.t, env, insertTx.Commit(context.Background())) - env.CancelWorkflow() - }() + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize transaction", func() bool { + pgRows, err := e2e.GetPgRows(s.conn, s.bqSuffix, srcName, "id,c1,c2,t") + e2e.EnvNoError(s.t, env, err) + rows, err := s.GetRowsWhere(dstName, "id,c1,c2,t", "NOT _PEERDB_IS_DELETED") + if err != nil { + return false + } + return e2eshared.CheckEqualRecordBatches(s.t, pgRows, rows) + }) - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, config, nil) + env.Cancel() e2e.RequireEnvCanceled(s.t, env) newerSyncedAtQuery := fmt.Sprintf( @@ -1437,7 +1381,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Soft_Delete_UD_Same_Batch() { } func (s PeerFlowE2ETestSuiteBQ) Test_Soft_Delete_Insert_After_Delete() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) tableName := "test_softdel_iad" srcTableName := s.attachSchemaSuffix(tableName) @@ -1473,38 +1417,35 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Soft_Delete_Insert_After_Delete() { MaxBatchSize: 100, } - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and then insert and delete rows in the table. - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, config, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c1,c2,t) VALUES (1,2,random_string(9000))`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - e2e.EnvWaitForEqualTables(env, s, "normalize insert", tableName, "id,c1,c2,t") - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + e2e.EnvWaitForEqualTables(env, s, "normalize insert", tableName, "id,c1,c2,t") + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` DELETE FROM %s WHERE id=1`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize delete", func() bool { - pgRows, err := e2e.GetPgRows(s.conn, s.bqSuffix, tableName, "id,c1,c2,t") - if err != nil { - return false - } - rows, err := s.GetRowsWhere(tableName, "id,c1,c2,t", "NOT _PEERDB_IS_DELETED") - if err != nil { - return false - } - return e2eshared.CheckEqualRecordBatches(s.t, pgRows, rows) - }) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize delete", func() bool { + pgRows, err := e2e.GetPgRows(s.conn, s.bqSuffix, tableName, "id,c1,c2,t") + if err != nil { + return false + } + rows, err := s.GetRowsWhere(tableName, "id,c1,c2,t", "NOT _PEERDB_IS_DELETED") + if err != nil { + return false + } + return e2eshared.CheckEqualRecordBatches(s.t, pgRows, rows) + }) + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(id,c1,c2,t) VALUES (1,3,4,random_string(10000))`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - e2e.EnvWaitForEqualTables(env, s, "normalize reinsert", tableName, "id,c1,c2,t") - - env.CancelWorkflow() - }() + e2e.EnvNoError(s.t, env, err) + e2e.EnvWaitForEqualTables(env, s, "normalize reinsert", tableName, "id,c1,c2,t") - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, config, nil) + env.Cancel() e2e.RequireEnvCanceled(s.t, env) newerSyncedAtQuery := fmt.Sprintf( @@ -1516,8 +1457,6 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Soft_Delete_Insert_After_Delete() { } func (s PeerFlowE2ETestSuiteBQ) Test_JSON_PKey_BQ() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) - srcTableName := s.attachSchemaSuffix("test_json_pkey_bq") dstTableName := "test_json_pkey_bq" @@ -1546,24 +1485,24 @@ func (s PeerFlowE2ETestSuiteBQ) Test_JSON_PKey_BQ() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - // insert 10 rows into the source table - for i := range 10 { - testKey := fmt.Sprintf("test_key_%d", i) - testValue := fmt.Sprintf("test_value_%d", i) - testJson := `'{"name":"jack", "age":12, "spouse":null}'::json` - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` - INSERT INTO %s(key, value, j) VALUES ($1, $2, %s) - `, srcTableName, testJson), testKey, testValue) - e2e.EnvNoError(s.t, env, err) - } - s.t.Log("Inserted 10 rows into the source table") + tc := e2e.NewTemporalClient(s.t) + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + // insert 10 rows into the source table + for i := range 10 { + testKey := fmt.Sprintf("test_key_%d", i) + testValue := fmt.Sprintf("test_value_%d", i) + testJson := `'{"name":"jack", "age":12, "spouse":null}'::json` + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(key, value, j) VALUES ($1, $2, %s) + `, srcTableName, testJson), testKey, testValue) + e2e.EnvNoError(s.t, env, err) + } + s.t.Log("Inserted 10 rows into the source table") - e2e.EnvWaitForEqualTables(env, s, "normalize inserts", dstTableName, "id,key,value,j") - env.CancelWorkflow() - }() + e2e.EnvWaitForEqualTables(env, s, "normalize inserts", dstTableName, "id,key,value,j") - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) + env.Cancel() e2e.RequireEnvCanceled(s.t, env) } diff --git a/flow/e2e/bigquery/qrep_flow_bq_test.go b/flow/e2e/bigquery/qrep_flow_bq_test.go index 4fdfb52e95..c7f6a5c7f8 100644 --- a/flow/e2e/bigquery/qrep_flow_bq_test.go +++ b/flow/e2e/bigquery/qrep_flow_bq_test.go @@ -4,6 +4,7 @@ import ( "context" "fmt" "strings" + "time" "github.com/stretchr/testify/require" @@ -57,7 +58,7 @@ func (s PeerFlowE2ETestSuiteBQ) setupTimeTable(tableName string) { } func (s PeerFlowE2ETestSuiteBQ) Test_Complete_QRep_Flow_Avro() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) numRows := 10 @@ -76,20 +77,15 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Complete_QRep_Flow_Avro() { true, "") require.NoError(s.t, err) - e2e.RunQrepFlowWorkflow(env, qrepConfig) - - // Verify workflow completes without error - require.True(s.t, env.IsWorkflowCompleted()) - - err = env.GetWorkflowError() - require.NoError(s.t, err) + env := e2e.RunQrepFlowWorkflow(tc, qrepConfig) + e2e.EnvWaitForFinished(s.t, env, 3*time.Minute) + require.NoError(s.t, env.Error()) e2e.RequireEqualTables(s, tblName, "*") } func (s PeerFlowE2ETestSuiteBQ) Test_Invalid_Timestamps_And_Date_QRep() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) - + tc := e2e.NewTemporalClient(s.t) tblName := "test_invalid_time_bq" s.setupTimeTable(tblName) @@ -106,13 +102,9 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Invalid_Timestamps_And_Date_QRep() { "") qrepConfig.WatermarkColumn = "watermark_ts" require.NoError(s.t, err) - e2e.RunQrepFlowWorkflow(env, qrepConfig) - - // Verify workflow completes without error - require.True(s.t, env.IsWorkflowCompleted()) - - err = env.GetWorkflowError() - require.NoError(s.t, err) + env := e2e.RunQrepFlowWorkflow(tc, qrepConfig) + e2e.EnvWaitForFinished(s.t, env, 3*time.Minute) + require.NoError(s.t, env.Error()) goodValues := []string{"watermark_ts", "mydate", "medieval"} badValues := []string{"mytimestamp", "mytztimestamp", "mybaddate"} @@ -131,7 +123,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Invalid_Timestamps_And_Date_QRep() { } func (s PeerFlowE2ETestSuiteBQ) Test_PeerDB_Columns_QRep_BQ() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) numRows := 10 @@ -150,13 +142,9 @@ func (s PeerFlowE2ETestSuiteBQ) Test_PeerDB_Columns_QRep_BQ() { true, "_PEERDB_SYNCED_AT") require.NoError(s.t, err) - e2e.RunQrepFlowWorkflow(env, qrepConfig) - - // Verify workflow completes without error - require.True(s.t, env.IsWorkflowCompleted()) - - err = env.GetWorkflowError() - require.NoError(s.t, err) + env := e2e.RunQrepFlowWorkflow(tc, qrepConfig) + e2e.EnvWaitForFinished(s.t, env, 3*time.Minute) + require.NoError(s.t, env.Error()) err = s.checkPeerdbColumns(tblName, false) require.NoError(s.t, err) diff --git a/flow/e2e/postgres/peer_flow_pg_test.go b/flow/e2e/postgres/peer_flow_pg_test.go index 0ec31befe4..192ee85ba9 100644 --- a/flow/e2e/postgres/peer_flow_pg_test.go +++ b/flow/e2e/postgres/peer_flow_pg_test.go @@ -12,9 +12,6 @@ import ( "github.com/jackc/pgx/v5/pgtype" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - "go.temporal.io/sdk/testsuite" - "go.temporal.io/sdk/worker" - "go.temporal.io/sdk/workflow" "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/e2e" @@ -55,7 +52,7 @@ func (s PeerFlowE2ETestSuitePG) checkPeerdbColumns(dstSchemaQualified string, ro } func (s PeerFlowE2ETestSuitePG) WaitForSchema( - env *testsuite.TestWorkflowEnvironment, + env e2e.WorkflowRun, reason string, srcTableName string, dstTableName string, @@ -81,8 +78,6 @@ func (s PeerFlowE2ETestSuitePG) WaitForSchema( } func (s PeerFlowE2ETestSuitePG) Test_Simple_Flow_PG() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) - srcTableName := s.attachSchemaSuffix("test_simple_flow") dstTableName := s.attachSchemaSuffix("test_simple_flow_dst") @@ -105,34 +100,30 @@ func (s PeerFlowE2ETestSuitePG) Test_Simple_Flow_PG() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup - // and then insert 10 rows into the source table - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - // insert 10 rows into the source table - for i := range 10 { - testKey := fmt.Sprintf("test_key_%d", i) - testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` - INSERT INTO %s(key, value, myh) VALUES ($1, $2, '"a"=>"b"') - `, srcTableName), testKey, testValue) - e2e.EnvNoError(s.t, env, err) - } - s.t.Log("Inserted 10 rows into the source table") + tc := e2e.NewTemporalClient(s.t) + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) - e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize 10 rows", func() bool { - return s.comparePGTables(srcTableName, dstTableName, "id,key,value") == nil - }) - env.CancelWorkflow() - }() + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + // insert 10 rows into the source table + for i := range 10 { + testKey := fmt.Sprintf("test_key_%d", i) + testValue := fmt.Sprintf("test_value_%d", i) + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(key, value, myh) VALUES ($1, $2, '"a"=>"b"') + `, srcTableName), testKey, testValue) + e2e.EnvNoError(s.t, env, err) + } + s.t.Log("Inserted 10 rows into the source table") + + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize 10 rows", func() bool { + return s.comparePGTables(srcTableName, dstTableName, "id,key,value") == nil + }) + env.Cancel() - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuitePG) Test_Geospatial_PG() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) - srcTableName := s.attachSchemaSuffix("test_geospatial_pg") dstTableName := s.attachSchemaSuffix("test_geospatial_pg_dst") @@ -154,27 +145,27 @@ func (s PeerFlowE2ETestSuitePG) Test_Geospatial_PG() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - // insert 1 row into the source table - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` - INSERT INTO %s(gg, gm) VALUES ('POLYGON((0 0, 0 1, 1 1, 1 0, 0 0))','LINESTRING(0 0, 1 1, 2 2)') - `, srcTableName)) - e2e.EnvNoError(s.t, env, err) + tc := e2e.NewTemporalClient(s.t) + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) - s.t.Log("Inserted 1 row into the source table") - e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize shapes", func() bool { - return s.comparePGTables(srcTableName, dstTableName, "id,gg,gm") == nil - }) - env.CancelWorkflow() - }() + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + // insert 1 row into the source table + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(gg, gm) VALUES ('POLYGON((0 0, 0 1, 1 1, 1 0, 0 0))','LINESTRING(0 0, 1 1, 2 2)') + `, srcTableName)) + e2e.EnvNoError(s.t, env, err) + + s.t.Log("Inserted 1 row into the source table") + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize shapes", func() bool { + return s.comparePGTables(srcTableName, dstTableName, "id,gg,gm") == nil + }) - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) + env.Cancel() e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuitePG) Test_Types_PG() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcTableName := s.attachSchemaSuffix("test_types_pg") dstTableName := s.attachSchemaSuffix("test_types_pg_dst") @@ -199,9 +190,9 @@ func (s PeerFlowE2ETestSuitePG) Test_Types_PG() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s SELECT 2,2,b'1', true,'s','test','1.1.10.2'::cidr, CURRENT_DATE,1.23,1.234,'192.168.1.5'::inet,1, @@ -217,27 +208,25 @@ func (s PeerFlowE2ETestSuitePG) Test_Types_PG() { '{true, false}'::boolean[], '{1,2}'::smallint[]; `, srcTableName)) - e2e.EnvNoError(s.t, env, err) - - s.t.Log("Inserted 1 row into the source table") - allCols := strings.Join([]string{ - "c1", "c2", "c4", - "c40", "id", "c9", "c11", "c12", "c13", "c14", "c15", - "c21", "c29", "c33", "c34", "c35", "c36", - "c7", "c8", "c32", "c42", "c43", "c44", "c46", "c47", "c48", "c49", "c50", - }, ",") - e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize types", func() bool { - return s.comparePGTables(srcTableName, dstTableName, allCols) == nil - }) - env.CancelWorkflow() - }() + e2e.EnvNoError(s.t, env, err) + + s.t.Log("Inserted 1 row into the source table") + allCols := strings.Join([]string{ + "c1", "c2", "c4", + "c40", "id", "c9", "c11", "c12", "c13", "c14", "c15", + "c21", "c29", "c33", "c34", "c35", "c36", + "c7", "c8", "c32", "c42", "c43", "c44", "c46", "c47", "c48", "c49", "c50", + }, ",") + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize types", func() bool { + return s.comparePGTables(srcTableName, dstTableName, allCols) == nil + }) + env.Cancel() - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuitePG) Test_Enums_PG() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcTableName := s.attachSchemaSuffix("test_enum_flow") dstTableName := s.attachSchemaSuffix("test_enum_flow_dst") @@ -265,26 +254,24 @@ func (s PeerFlowE2ETestSuitePG) Test_Enums_PG() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(my_mood, my_null_mood) VALUES ('happy',null) `, srcTableName)) - e2e.EnvNoError(s.t, env, err) - s.t.Log("Inserted enums into the source table") - e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize enum", func() bool { - return s.checkEnums(srcTableName, dstTableName) == nil - }) + e2e.EnvNoError(s.t, env, err) + s.t.Log("Inserted enums into the source table") + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize enum", func() bool { + return s.checkEnums(srcTableName, dstTableName) == nil + }) - env.CancelWorkflow() - }() + env.Cancel() - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuitePG) Test_Simple_Schema_Changes_PG() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcTableName := s.attachSchemaSuffix("test_simple_schema_changes") dstTableName := s.attachSchemaSuffix("test_simple_schema_changes_dst") @@ -306,168 +293,167 @@ func (s PeerFlowE2ETestSuitePG) Test_Simple_Schema_Changes_PG() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 1 - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and then insert and mutate schema repeatedly. - go func() { - // insert first row. - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + + // insert first row. + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c1) VALUES ($1)`, srcTableName), 1) - e2e.EnvNoError(s.t, env, err) - s.t.Log("Inserted initial row in the source table") + e2e.EnvNoError(s.t, env, err) + s.t.Log("Inserted initial row in the source table") - s.WaitForSchema(env, "normalizing first row", srcTableName, dstTableName, "id,c1", &protos.TableSchema{ - TableIdentifier: dstTableName, - PrimaryKeyColumns: []string{"id"}, - Columns: []*protos.FieldDescription{ - { - Name: "id", - Type: string(qvalue.QValueKindInt64), - TypeModifier: -1, - }, - { - Name: "c1", - Type: string(qvalue.QValueKindInt64), - TypeModifier: -1, - }, - { - Name: "_PEERDB_SYNCED_AT", - Type: string(qvalue.QValueKindTimestamp), - TypeModifier: -1, - }, + s.WaitForSchema(env, "normalizing first row", srcTableName, dstTableName, "id,c1", &protos.TableSchema{ + TableIdentifier: dstTableName, + PrimaryKeyColumns: []string{"id"}, + Columns: []*protos.FieldDescription{ + { + Name: "id", + Type: string(qvalue.QValueKindInt64), + TypeModifier: -1, }, - }) + { + Name: "c1", + Type: string(qvalue.QValueKindInt64), + TypeModifier: -1, + }, + { + Name: "_PEERDB_SYNCED_AT", + Type: string(qvalue.QValueKindTimestamp), + TypeModifier: -1, + }, + }, + }) - // alter source table, add column c2 and insert another row. - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + // alter source table, add column c2 and insert another row. + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` ALTER TABLE %s ADD COLUMN c2 BIGINT`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - s.t.Log("Altered source table, added column c2") - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + s.t.Log("Altered source table, added column c2") + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c1,c2) VALUES ($1,$2)`, srcTableName), 2, 2) - e2e.EnvNoError(s.t, env, err) - s.t.Log("Inserted row with added c2 in the source table") + e2e.EnvNoError(s.t, env, err) + s.t.Log("Inserted row with added c2 in the source table") - s.WaitForSchema(env, "normalizing altered row", srcTableName, dstTableName, "id,c1,c2", &protos.TableSchema{ - TableIdentifier: dstTableName, - PrimaryKeyColumns: []string{"id"}, - Columns: []*protos.FieldDescription{ - { - Name: "id", - Type: string(qvalue.QValueKindInt64), - TypeModifier: -1, - }, - { - Name: "c1", - Type: string(qvalue.QValueKindInt64), - TypeModifier: -1, - }, - { - Name: "_PEERDB_SYNCED_AT", - Type: string(qvalue.QValueKindTimestamp), - TypeModifier: -1, - }, - { - Name: "c2", - Type: string(qvalue.QValueKindInt64), - TypeModifier: -1, - }, + s.WaitForSchema(env, "normalizing altered row", srcTableName, dstTableName, "id,c1,c2", &protos.TableSchema{ + TableIdentifier: dstTableName, + PrimaryKeyColumns: []string{"id"}, + Columns: []*protos.FieldDescription{ + { + Name: "id", + Type: string(qvalue.QValueKindInt64), + TypeModifier: -1, }, - }) + { + Name: "c1", + Type: string(qvalue.QValueKindInt64), + TypeModifier: -1, + }, + { + Name: "_PEERDB_SYNCED_AT", + Type: string(qvalue.QValueKindTimestamp), + TypeModifier: -1, + }, + { + Name: "c2", + Type: string(qvalue.QValueKindInt64), + TypeModifier: -1, + }, + }, + }) - // alter source table, add column c3, drop column c2 and insert another row. - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + // alter source table, add column c3, drop column c2 and insert another row. + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` ALTER TABLE %s DROP COLUMN c2, ADD COLUMN c3 BIGINT`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - s.t.Log("Altered source table, dropped column c2 and added column c3") - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + s.t.Log("Altered source table, dropped column c2 and added column c3") + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c1,c3) VALUES ($1,$2)`, srcTableName), 3, 3) - e2e.EnvNoError(s.t, env, err) - s.t.Log("Inserted row with added c3 in the source table") + e2e.EnvNoError(s.t, env, err) + s.t.Log("Inserted row with added c3 in the source table") - s.WaitForSchema(env, "normalizing dropped column row", srcTableName, dstTableName, "id,c1,c3", &protos.TableSchema{ - TableIdentifier: dstTableName, - PrimaryKeyColumns: []string{"id"}, - Columns: []*protos.FieldDescription{ - { - Name: "id", - Type: string(qvalue.QValueKindInt64), - TypeModifier: -1, - }, - { - Name: "c1", - Type: string(qvalue.QValueKindInt64), - TypeModifier: -1, - }, - { - Name: "c2", - Type: string(qvalue.QValueKindInt64), - TypeModifier: -1, - }, - { - Name: "_PEERDB_SYNCED_AT", - Type: string(qvalue.QValueKindTimestamp), - TypeModifier: -1, - }, - { - Name: "c3", - Type: string(qvalue.QValueKindInt64), - TypeModifier: -1, - }, + s.WaitForSchema(env, "normalizing dropped column row", srcTableName, dstTableName, "id,c1,c3", &protos.TableSchema{ + TableIdentifier: dstTableName, + PrimaryKeyColumns: []string{"id"}, + Columns: []*protos.FieldDescription{ + { + Name: "id", + Type: string(qvalue.QValueKindInt64), + TypeModifier: -1, }, - }) + { + Name: "c1", + Type: string(qvalue.QValueKindInt64), + TypeModifier: -1, + }, + { + Name: "c2", + Type: string(qvalue.QValueKindInt64), + TypeModifier: -1, + }, + { + Name: "_PEERDB_SYNCED_AT", + Type: string(qvalue.QValueKindTimestamp), + TypeModifier: -1, + }, + { + Name: "c3", + Type: string(qvalue.QValueKindInt64), + TypeModifier: -1, + }, + }, + }) - // alter source table, drop column c3 and insert another row. - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + // alter source table, drop column c3 and insert another row. + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` ALTER TABLE %s DROP COLUMN c3`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - s.t.Log("Altered source table, dropped column c3") - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + s.t.Log("Altered source table, dropped column c3") + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c1) VALUES ($1)`, srcTableName), 4) - e2e.EnvNoError(s.t, env, err) - s.t.Log("Inserted row after dropping all columns in the source table") + e2e.EnvNoError(s.t, env, err) + s.t.Log("Inserted row after dropping all columns in the source table") - s.WaitForSchema(env, "normalizing 2nd dropped column row", srcTableName, dstTableName, "id,c1", &protos.TableSchema{ - TableIdentifier: dstTableName, - PrimaryKeyColumns: []string{"id"}, - Columns: []*protos.FieldDescription{ - { - Name: "id", - Type: string(qvalue.QValueKindInt64), - TypeModifier: -1, - }, - { - Name: "c1", - Type: string(qvalue.QValueKindInt64), - TypeModifier: -1, - }, - { - Name: "_PEERDB_SYNCED_AT", - Type: string(qvalue.QValueKindTimestamp), - TypeModifier: -1, - }, - { - Name: "c2", - Type: string(qvalue.QValueKindInt64), - TypeModifier: -1, - }, - { - Name: "c3", - Type: string(qvalue.QValueKindInt64), - TypeModifier: -1, - }, + s.WaitForSchema(env, "normalizing 2nd dropped column row", srcTableName, dstTableName, "id,c1", &protos.TableSchema{ + TableIdentifier: dstTableName, + PrimaryKeyColumns: []string{"id"}, + Columns: []*protos.FieldDescription{ + { + Name: "id", + Type: string(qvalue.QValueKindInt64), + TypeModifier: -1, }, - }) + { + Name: "c1", + Type: string(qvalue.QValueKindInt64), + TypeModifier: -1, + }, + { + Name: "_PEERDB_SYNCED_AT", + Type: string(qvalue.QValueKindTimestamp), + TypeModifier: -1, + }, + { + Name: "c2", + Type: string(qvalue.QValueKindInt64), + TypeModifier: -1, + }, + { + Name: "c3", + Type: string(qvalue.QValueKindInt64), + TypeModifier: -1, + }, + }, + }) - env.CancelWorkflow() - }() + env.Cancel() - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuitePG) Test_Composite_PKey_PG() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcTableName := s.attachSchemaSuffix("test_simple_cpkey") dstTableName := s.attachSchemaSuffix("test_simple_cpkey_dst") @@ -492,41 +478,39 @@ func (s PeerFlowE2ETestSuitePG) Test_Composite_PKey_PG() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and then insert, update and delete rows in the table. - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - // insert 10 rows into the source table - for i := range 10 { - testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + // insert 10 rows into the source table + for i := range 10 { + testValue := fmt.Sprintf("test_value_%d", i) + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c2,t) VALUES ($1,$2) `, srcTableName), i, testValue) - e2e.EnvNoError(s.t, env, err) - } - s.t.Log("Inserted 10 rows into the source table") + e2e.EnvNoError(s.t, env, err) + } + s.t.Log("Inserted 10 rows into the source table") - e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize 10 rows", func() bool { - return s.comparePGTables(srcTableName, dstTableName, "id,c1,c2,t") == nil - }) + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize 10 rows", func() bool { + return s.comparePGTables(srcTableName, dstTableName, "id,c1,c2,t") == nil + }) - _, err := s.Conn().Exec(context.Background(), - fmt.Sprintf(`UPDATE %s SET c1=c1+1 WHERE MOD(c2,2)=$1`, srcTableName), 1) - e2e.EnvNoError(s.t, env, err) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=$1`, srcTableName), 0) - e2e.EnvNoError(s.t, env, err) - e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize modifications", func() bool { - return s.comparePGTables(srcTableName, dstTableName, "id,c1,c2,t") == nil - }) - env.CancelWorkflow() - }() + _, err = s.Conn().Exec(context.Background(), + fmt.Sprintf(`UPDATE %s SET c1=c1+1 WHERE MOD(c2,2)=$1`, srcTableName), 1) + e2e.EnvNoError(s.t, env, err) + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=$1`, srcTableName), 0) + e2e.EnvNoError(s.t, env, err) + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize modifications", func() bool { + return s.comparePGTables(srcTableName, dstTableName, "id,c1,c2,t") == nil + }) + env.Cancel() - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuitePG) Test_Composite_PKey_Toast_1_PG() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcTableName := s.attachSchemaSuffix("test_cpkey_toast1") randomString := s.attachSchemaSuffix("random_string") @@ -556,44 +540,42 @@ func (s PeerFlowE2ETestSuitePG) Test_Composite_PKey_Toast_1_PG() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and then insert, update and delete rows in the table. - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - rowsTx, err := s.Conn().Begin(context.Background()) - e2e.EnvNoError(s.t, env, err) - - // insert 10 rows into the source table - for i := range 10 { - testValue := fmt.Sprintf("test_value_%d", i) - _, err = rowsTx.Exec(context.Background(), fmt.Sprintf(` + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + rowsTx, err := s.Conn().Begin(context.Background()) + e2e.EnvNoError(s.t, env, err) + + // insert 10 rows into the source table + for i := range 10 { + testValue := fmt.Sprintf("test_value_%d", i) + _, err = rowsTx.Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c2,t,t2) VALUES ($1,$2,%s(9000)) `, srcTableName, randomString), i, testValue) - e2e.EnvNoError(s.t, env, err) - } - s.t.Log("Inserted 10 rows into the source table") - - _, err = rowsTx.Exec(context.Background(), - fmt.Sprintf(`UPDATE %s SET c1=c1+1 WHERE MOD(c2,2)=$1`, srcTableName), 1) - e2e.EnvNoError(s.t, env, err) - _, err = rowsTx.Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=$1`, srcTableName), 0) e2e.EnvNoError(s.t, env, err) + } + s.t.Log("Inserted 10 rows into the source table") - err = rowsTx.Commit(context.Background()) - e2e.EnvNoError(s.t, env, err) + _, err = rowsTx.Exec(context.Background(), + fmt.Sprintf(`UPDATE %s SET c1=c1+1 WHERE MOD(c2,2)=$1`, srcTableName), 1) + e2e.EnvNoError(s.t, env, err) + _, err = rowsTx.Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=$1`, srcTableName), 0) + e2e.EnvNoError(s.t, env, err) - e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize tx", func() bool { - return s.comparePGTables(srcTableName, dstTableName, "id,c1,c2,t,t2") == nil - }) - env.CancelWorkflow() - }() + err = rowsTx.Commit(context.Background()) + e2e.EnvNoError(s.t, env, err) + + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize tx", func() bool { + return s.comparePGTables(srcTableName, dstTableName, "id,c1,c2,t,t2") == nil + }) + env.Cancel() - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuitePG) Test_Composite_PKey_Toast_2_PG() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcTableName := s.attachSchemaSuffix("test_cpkey_toast2") randomString := s.attachSchemaSuffix("random_string") @@ -623,43 +605,41 @@ func (s PeerFlowE2ETestSuitePG) Test_Composite_PKey_Toast_2_PG() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and then insert, update and delete rows in the table. - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - // insert 10 rows into the source table - for i := range 10 { - testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + // insert 10 rows into the source table + for i := range 10 { + testValue := fmt.Sprintf("test_value_%d", i) + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c2,t,t2) VALUES ($1,$2,%s(9000)) `, srcTableName, randomString), i, testValue) - e2e.EnvNoError(s.t, env, err) - } - s.t.Log("Inserted 10 rows into the source table") - - e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize 10 rows", func() bool { - return s.comparePGTables(srcTableName, dstTableName, "id,c1,c2,t,t2") == nil - }) - _, err = s.Conn().Exec(context.Background(), - fmt.Sprintf(`UPDATE %s SET c1=c1+1 WHERE MOD(c2,2)=$1`, srcTableName), 1) - e2e.EnvNoError(s.t, env, err) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=$1`, srcTableName), 0) e2e.EnvNoError(s.t, env, err) + } + s.t.Log("Inserted 10 rows into the source table") - e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize update", func() bool { - return s.comparePGTables(srcTableName, dstTableName, "id,c1,c2,t,t2") == nil - }) + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize 10 rows", func() bool { + return s.comparePGTables(srcTableName, dstTableName, "id,c1,c2,t,t2") == nil + }) + _, err = s.Conn().Exec(context.Background(), + fmt.Sprintf(`UPDATE %s SET c1=c1+1 WHERE MOD(c2,2)=$1`, srcTableName), 1) + e2e.EnvNoError(s.t, env, err) + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=$1`, srcTableName), 0) + e2e.EnvNoError(s.t, env, err) + + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize update", func() bool { + return s.comparePGTables(srcTableName, dstTableName, "id,c1,c2,t,t2") == nil + }) - env.CancelWorkflow() - }() + env.Cancel() - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuitePG) Test_PeerDB_Columns() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcTableName := s.attachSchemaSuffix("test_peerdb_cols") dstTableName := s.attachSchemaSuffix("test_peerdb_cols_dst") @@ -683,35 +663,33 @@ func (s PeerFlowE2ETestSuitePG) Test_PeerDB_Columns() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - // insert 1 row into the source table - testKey := fmt.Sprintf("test_key_%d", 1) - testValue := fmt.Sprintf("test_value_%d", 1) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + // insert 1 row into the source table + testKey := fmt.Sprintf("test_key_%d", 1) + testValue := fmt.Sprintf("test_value_%d", 1) + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(key, value) VALUES ($1, $2) `, srcTableName), testKey, testValue) - e2e.EnvNoError(s.t, env, err) + e2e.EnvNoError(s.t, env, err) - // delete that row - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + // delete that row + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` DELETE FROM %s WHERE id=1 `, srcTableName)) - e2e.EnvNoError(s.t, env, err) - s.t.Log("Inserted and deleted a row for peerdb column check") + e2e.EnvNoError(s.t, env, err) + s.t.Log("Inserted and deleted a row for peerdb column check") - e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize insert/delete", func() bool { - return s.checkPeerdbColumns(dstTableName, 1) == nil - }) - env.CancelWorkflow() - }() + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize insert/delete", func() bool { + return s.checkPeerdbColumns(dstTableName, 1) == nil + }) + env.Cancel() - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuitePG) Test_Soft_Delete_Basic() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) cmpTableName := s.attachSchemaSuffix("test_softdel") srcTableName := cmpTableName + "_src" @@ -748,39 +726,36 @@ func (s PeerFlowE2ETestSuitePG) Test_Soft_Delete_Basic() { MaxBatchSize: 100, } - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and then insert, update and delete rows in the table. - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, config, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c1,c2,t) VALUES (1,2,random_string(9000))`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize row", func() bool { - return s.comparePGTables(srcTableName, dstTableName, "id,c1,c2,t") == nil - }) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize row", func() bool { + return s.comparePGTables(srcTableName, dstTableName, "id,c1,c2,t") == nil + }) + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` UPDATE %s SET c1=c1+4 WHERE id=1`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize update", func() bool { - return s.comparePGTables(srcTableName, dstTableName, "id,c1,c2,t") == nil - }) - // since we delete stuff, create another table to compare with - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize update", func() bool { + return s.comparePGTables(srcTableName, dstTableName, "id,c1,c2,t") == nil + }) + // since we delete stuff, create another table to compare with + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` CREATE TABLE %s AS SELECT * FROM %s`, cmpTableName, srcTableName)) - e2e.EnvNoError(s.t, env, err) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` DELETE FROM %s WHERE id=1`, srcTableName)) - e2e.EnvNoError(s.t, env, err) + e2e.EnvNoError(s.t, env, err) - e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize delete", func() bool { - return s.comparePGTables(srcTableName, dstTableName+` WHERE NOT "_PEERDB_IS_DELETED"`, "id,c1,c2,t") == nil - }) - - env.CancelWorkflow() - }() + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize delete", func() bool { + return s.comparePGTables(srcTableName, dstTableName+` WHERE NOT "_PEERDB_IS_DELETED"`, "id,c1,c2,t") == nil + }) - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, config, nil) + env.Cancel() e2e.RequireEnvCanceled(s.t, env) // verify our updates and delete happened @@ -797,7 +772,7 @@ func (s PeerFlowE2ETestSuitePG) Test_Soft_Delete_Basic() { } func (s PeerFlowE2ETestSuitePG) Test_Soft_Delete_IUD_Same_Batch() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) cmpTableName := s.attachSchemaSuffix("test_softdel_iud") srcTableName := cmpTableName + "_src" @@ -834,50 +809,45 @@ func (s PeerFlowE2ETestSuitePG) Test_Soft_Delete_IUD_Same_Batch() { MaxBatchSize: 100, } - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and then insert, update and delete rows in the table. - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, config, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - insertTx, err := s.Conn().Begin(context.Background()) - e2e.EnvNoError(s.t, env, err) + insertTx, err := s.Conn().Begin(context.Background()) + e2e.EnvNoError(s.t, env, err) - _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` + _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c1,c2,t) VALUES (1,2,random_string(9000))`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` UPDATE %s SET c1=c1+4 WHERE id=1`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - // since we delete stuff, create another table to compare with - _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + // since we delete stuff, create another table to compare with + _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE %s AS SELECT * FROM %s`, cmpTableName, srcTableName)) - e2e.EnvNoError(s.t, env, err) - _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` DELETE FROM %s WHERE id=1`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - - e2e.EnvNoError(s.t, env, insertTx.Commit(context.Background())) + e2e.EnvNoError(s.t, env, err) - e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize tx", func() bool { - return s.comparePGTables(cmpTableName, dstTableName, "id,c1,c2,t") == nil - }) + e2e.EnvNoError(s.t, env, insertTx.Commit(context.Background())) - softDeleteQuery := fmt.Sprintf(`SELECT COUNT(*) FROM %s WHERE "_PEERDB_IS_DELETED"`, dstTableName) - e2e.EnvWaitFor(s.t, env, time.Minute, "normalize soft delete", func() bool { - numRows, err := s.RunInt64Query(softDeleteQuery) - return err == nil && numRows == 1 - }) + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize tx", func() bool { + return s.comparePGTables(cmpTableName, dstTableName, "id,c1,c2,t") == nil + }) - env.CancelWorkflow() - }() + softDeleteQuery := fmt.Sprintf(`SELECT COUNT(*) FROM %s WHERE "_PEERDB_IS_DELETED"`, dstTableName) + e2e.EnvWaitFor(s.t, env, time.Minute, "normalize soft delete", func() bool { + numRows, err := s.RunInt64Query(softDeleteQuery) + return err == nil && numRows == 1 + }) - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, config, nil) + env.Cancel() e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuitePG) Test_Soft_Delete_UD_Same_Batch() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) - cmpTableName := s.attachSchemaSuffix("test_softdel_ud") srcTableName := cmpTableName + "_src" dstTableName := s.attachSchemaSuffix("test_softdel_ud_dst") @@ -913,40 +883,38 @@ func (s PeerFlowE2ETestSuitePG) Test_Soft_Delete_UD_Same_Batch() { MaxBatchSize: 100, } - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and then insert, update and delete rows in the table. - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + tc := e2e.NewTemporalClient(s.t) + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, config, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c1,c2,t) VALUES (1,2,random_string(9000))`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize row", func() bool { - return s.comparePGTables(srcTableName, dstTableName, "id,c1,c2,t") == nil - }) + e2e.EnvNoError(s.t, env, err) + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize row", func() bool { + return s.comparePGTables(srcTableName, dstTableName, "id,c1,c2,t") == nil + }) - insertTx, err := s.Conn().Begin(context.Background()) - e2e.EnvNoError(s.t, env, err) - _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` + insertTx, err := s.Conn().Begin(context.Background()) + e2e.EnvNoError(s.t, env, err) + _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` UPDATE %s SET t=random_string(10000) WHERE id=1`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` UPDATE %s SET c1=c1+4 WHERE id=1`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` DELETE FROM %s WHERE id=1`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - e2e.EnvNoError(s.t, env, insertTx.Commit(context.Background())) + e2e.EnvNoError(s.t, env, err) + e2e.EnvNoError(s.t, env, insertTx.Commit(context.Background())) - e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize transaction", func() bool { - return s.comparePGTables(srcTableName, - dstTableName+` WHERE NOT "_PEERDB_IS_DELETED"`, "id,c1,c2,t") == nil - }) - - env.CancelWorkflow() - }() + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize transaction", func() bool { + return s.comparePGTables(srcTableName, + dstTableName+` WHERE NOT "_PEERDB_IS_DELETED"`, "id,c1,c2,t") == nil + }) - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, config, nil) + env.Cancel() e2e.RequireEnvCanceled(s.t, env) // verify our updates and delete happened @@ -961,7 +929,7 @@ func (s PeerFlowE2ETestSuitePG) Test_Soft_Delete_UD_Same_Batch() { } func (s PeerFlowE2ETestSuitePG) Test_Soft_Delete_Insert_After_Delete() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcTableName := s.attachSchemaSuffix("test_softdel_iad") dstTableName := s.attachSchemaSuffix("test_softdel_iad_dst") @@ -997,34 +965,31 @@ func (s PeerFlowE2ETestSuitePG) Test_Soft_Delete_Insert_After_Delete() { MaxBatchSize: 100, } - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and then insert and delete rows in the table. - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, config, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c1,c2,t) VALUES (1,2,random_string(9000))`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize row", func() bool { - return s.comparePGTables(srcTableName, dstTableName, "id,c1,c2,t") == nil - }) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize row", func() bool { + return s.comparePGTables(srcTableName, dstTableName, "id,c1,c2,t") == nil + }) + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` DELETE FROM %s WHERE id=1`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize delete", func() bool { - return s.comparePGTables(srcTableName, dstTableName+` WHERE NOT "_PEERDB_IS_DELETED"`, "id,c1,c2,t") == nil - }) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize delete", func() bool { + return s.comparePGTables(srcTableName, dstTableName+` WHERE NOT "_PEERDB_IS_DELETED"`, "id,c1,c2,t") == nil + }) + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(id,c1,c2,t) VALUES (1,3,4,random_string(10000))`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize reinsert", func() bool { - return s.comparePGTables(srcTableName, dstTableName, "id,c1,c2,t") == nil - }) - - env.CancelWorkflow() - }() + e2e.EnvNoError(s.t, env, err) + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize reinsert", func() bool { + return s.comparePGTables(srcTableName, dstTableName, "id,c1,c2,t") == nil + }) - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, config, nil) + env.Cancel() e2e.RequireEnvCanceled(s.t, env) softDeleteQuery := fmt.Sprintf(` @@ -1036,7 +1001,7 @@ func (s PeerFlowE2ETestSuitePG) Test_Soft_Delete_Insert_After_Delete() { } func (s PeerFlowE2ETestSuitePG) Test_Supported_Mixed_Case_Table() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) stmtSrcTableName := fmt.Sprintf(`e2e_test_%s."%s"`, s.suffix, "testMixedCase") srcTableName := s.attachSchemaSuffix("testMixedCase") @@ -1071,48 +1036,37 @@ func (s PeerFlowE2ETestSuitePG) Test_Supported_Mixed_Case_Table() { MaxBatchSize: 100, } - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and then insert and delete rows in the table. - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - // insert 20 rows into the source table - for i := range 10 { - testKey := fmt.Sprintf("test_key_%d", i) - testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, config, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + // insert 20 rows into the source table + for i := range 10 { + testKey := fmt.Sprintf("test_key_%d", i) + testValue := fmt.Sprintf("test_value_%d", i) + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s ("highGold","eVe") VALUES ($1, $2) `, stmtSrcTableName), testKey, testValue) - e2e.EnvNoError(s.t, env, err) - } - s.t.Log("Inserted 20 rows into the source table") - - e2e.EnvWaitFor(s.t, env, 1*time.Minute, "normalize mixed case", func() bool { - return s.comparePGTables(stmtSrcTableName, stmtDstTableName, - "id,\"pulseArmor\",\"highGold\",\"eVe\"") == nil - }) + e2e.EnvNoError(s.t, env, err) + } + s.t.Log("Inserted 20 rows into the source table") - env.CancelWorkflow() - }() + e2e.EnvWaitFor(s.t, env, 1*time.Minute, "normalize mixed case", func() bool { + return s.comparePGTables(stmtSrcTableName, stmtDstTableName, + "id,\"pulseArmor\",\"highGold\",\"eVe\"") == nil + }) - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, config, nil) + env.Cancel() + e2e.RequireEnvCanceled(s.t, env) } // test don't work, make it work later func (s PeerFlowE2ETestSuitePG) Test_Dynamic_Mirror_Config_Via_Signals() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) - // needed otherwise errors out - workerOptions := worker.Options{ - EnableSessionWorker: true, - } - env.SetWorkerOptions(workerOptions) - srcTable1Name := s.attachSchemaSuffix("test_dynconfig_1") srcTable2Name := s.attachSchemaSuffix("test_dynconfig_2") dstTable1Name := s.attachSchemaSuffix("test_dynconfig_1_dst") dstTable2Name := s.attachSchemaSuffix("test_dynconfig_2_dst") - sentPause := false - sentUpdate := false _, err := s.Conn().Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( @@ -1147,6 +1101,9 @@ func (s PeerFlowE2ETestSuitePG) Test_Dynamic_Mirror_Config_Via_Signals() { SnapshotNumTablesInParallel: 1, } + tc := e2e.NewTemporalClient(s.t) + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, config, nil) + addRows := func(numRows int) { for range numRows { _, err = s.Conn().Exec(context.Background(), @@ -1161,7 +1118,7 @@ func (s PeerFlowE2ETestSuitePG) Test_Dynamic_Mirror_Config_Via_Signals() { getWorkflowState := func() peerflow.CDCFlowWorkflowState { var state peerflow.CDCFlowWorkflowState - val, err := env.QueryWorkflow(shared.CDCFlowStateQuery) + val, err := env.Query(shared.CDCFlowStateQuery) e2e.EnvNoError(s.t, env, err) err = val.Get(&state) e2e.EnvNoError(s.t, env, err) @@ -1171,7 +1128,7 @@ func (s PeerFlowE2ETestSuitePG) Test_Dynamic_Mirror_Config_Via_Signals() { getFlowStatus := func() protos.FlowStatus { var flowStatus protos.FlowStatus - val, err := env.QueryWorkflow(shared.FlowStatusQuery) + val, err := env.Query(shared.FlowStatusQuery) e2e.EnvNoError(s.t, env, err) err = val.Get(&flowStatus) e2e.EnvNoError(s.t, env, err) @@ -1179,101 +1136,60 @@ func (s PeerFlowE2ETestSuitePG) Test_Dynamic_Mirror_Config_Via_Signals() { return flowStatus } - var workflowState peerflow.CDCFlowWorkflowState - - // signals in tests are weird, you need to register them before starting the workflow - // otherwise you guessed it, errors out. really don't like this. - // too short of a gap between signals also causes issues - // might have something to do with how test workflows handle fast-forwarding time. - env.RegisterDelayedCallback(func() { - workflowState = getWorkflowState() - e2e.EnvSignalWorkflow(env, model.FlowSignal, model.PauseSignal) - s.t.Log("Sent pause signal") - sentPause = true - }, 28*time.Second) - // add before to test initial load too. addRows(18) - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - // insert 18 rows into the source tables, exactly 3 batches - addRows(18) - - e2e.EnvWaitFor(s.t, env, 1*time.Minute, "normalize 18 records - first table", func() bool { - return s.comparePGTables(srcTable1Name, dstTable1Name, "id,t") == nil - }) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + // insert 18 rows into the source tables, exactly 3 batches + addRows(18) - workflowState = getWorkflowState() - assert.EqualValues(s.t, 7, workflowState.SyncFlowOptions.IdleTimeoutSeconds) - assert.EqualValues(s.t, 6, workflowState.SyncFlowOptions.BatchSize) - assert.Len(s.t, workflowState.SyncFlowOptions.TableMappings, 1) - assert.Len(s.t, workflowState.SyncFlowOptions.SrcTableIdNameMapping, 1) - assert.Len(s.t, workflowState.SyncFlowOptions.TableNameSchemaMapping, 1) - - if !s.t.Failed() { - // wait for first RegisterDelayedCallback to hit. - e2e.EnvWaitFor(s.t, env, 1*time.Minute, "sent pause signal", func() bool { - return sentPause - }) - } else { - env.CancelWorkflow() - } - }() + e2e.EnvWaitFor(s.t, env, 1*time.Minute, "normalize 18 records - first table", func() bool { + return s.comparePGTables(srcTable1Name, dstTable1Name, "id,t") == nil + }) - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, config, nil) - err = env.GetWorkflowError() - if !workflow.IsContinueAsNewError(err) { - require.NoError(s.t, err) - require.Error(s.t, err) - } - workflowState.ActiveSignal = model.PauseSignal - env = e2e.NewTemporalTestWorkflowEnvironment(s.t) + workflowState := getWorkflowState() + assert.EqualValues(s.t, 7, workflowState.SyncFlowOptions.IdleTimeoutSeconds) + assert.EqualValues(s.t, 6, workflowState.SyncFlowOptions.BatchSize) + assert.Len(s.t, workflowState.SyncFlowOptions.TableMappings, 1) + assert.Len(s.t, workflowState.SyncFlowOptions.SrcTableIdNameMapping, 1) + assert.Len(s.t, workflowState.SyncFlowOptions.TableNameSchemaMapping, 1) + + if !s.t.Failed() { + addRows(1) + e2e.SignalWorkflow(env, model.FlowSignal, model.PauseSignal) + addRows(1) + e2e.EnvWaitFor(s.t, env, 1*time.Minute, "paused workflow", func() bool { + // keep adding 1 more row - finishing another sync + addRows(1) - // this signal being sent also unblocks another WaitFor - env.RegisterDelayedCallback(func() { - e2e.EnvWaitFor(s.t, env, 1*time.Minute, "send update signal after pause confirmed", func() bool { flowStatus := getFlowStatus() - if flowStatus != protos.FlowStatus_STATUS_PAUSED { - return false - } - e2e.EnvSignalWorkflow(env, model.CDCDynamicPropertiesSignal, &protos.CDCFlowConfigUpdate{ - IdleTimeout: 14, - BatchSize: 12, - AdditionalTables: []*protos.TableMapping{ - { - SourceTableIdentifier: srcTable2Name, - DestinationTableIdentifier: dstTable2Name, - }, - }, - }) - s.t.Log("Sent update signal") - sentUpdate = true - return true + return flowStatus == protos.FlowStatus_STATUS_PAUSED }) - }, 56*time.Second) - go func() { - // we have a paused mirror, wait for second signal to hit. - e2e.EnvWaitFor(s.t, env, 1*time.Minute, "sent updates signal", func() bool { - return sentUpdate + e2e.SignalWorkflow(env, model.CDCDynamicPropertiesSignal, &protos.CDCFlowConfigUpdate{ + IdleTimeout: 14, + BatchSize: 12, + AdditionalTables: []*protos.TableMapping{ + { + SourceTableIdentifier: srcTable2Name, + DestinationTableIdentifier: dstTable2Name, + }, + }, }) // add rows to both tables before resuming - should handle addRows(18) + e2e.SignalWorkflow(env, model.FlowSignal, model.NoopSignal) + e2e.EnvWaitFor(s.t, env, 1*time.Minute, "resumed workflow", func() bool { return getFlowStatus() == protos.FlowStatus_STATUS_RUNNING }) e2e.EnvWaitFor(s.t, env, 1*time.Minute, "normalize 18 records - first table", func() bool { return s.comparePGTables(srcTable1Name, dstTable1Name, "id,t") == nil }) - /* TODO fix in integration tests e2e.EnvWaitFor(s.t, env, 2*time.Minute, "initial load + normalize 18 records - second table", func() bool { - err := s.comparePGTables(srcTable2Name, dstTable2Name, "id,t") - s.t.Log("TEST", err) - return err == nil + return s.comparePGTables(srcTable2Name, dstTable2Name, "id,t") == nil }) - */ workflowState = getWorkflowState() assert.EqualValues(s.t, 14, workflowState.SyncFlowOptions.IdleTimeoutSeconds) @@ -1281,9 +1197,8 @@ func (s PeerFlowE2ETestSuitePG) Test_Dynamic_Mirror_Config_Via_Signals() { assert.Len(s.t, workflowState.SyncFlowOptions.TableMappings, 2) assert.Len(s.t, workflowState.SyncFlowOptions.SrcTableIdNameMapping, 2) assert.Len(s.t, workflowState.SyncFlowOptions.TableNameSchemaMapping, 2) - env.CancelWorkflow() - }() + } - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, config, &workflowState) + env.Cancel() e2e.RequireEnvCanceled(s.t, env) } diff --git a/flow/e2e/postgres/qrep_flow_pg_test.go b/flow/e2e/postgres/qrep_flow_pg_test.go index e4605ce592..a7a3672119 100644 --- a/flow/e2e/postgres/qrep_flow_pg_test.go +++ b/flow/e2e/postgres/qrep_flow_pg_test.go @@ -215,8 +215,6 @@ func (s PeerFlowE2ETestSuitePG) TestSimpleSlotCreation() { } func (s PeerFlowE2ETestSuitePG) Test_Complete_QRep_Flow_Multi_Insert_PG() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) - numRows := 10 srcTable := "test_qrep_flow_avro_pg_1" @@ -247,21 +245,16 @@ func (s PeerFlowE2ETestSuitePG) Test_Complete_QRep_Flow_Multi_Insert_PG() { ) require.NoError(s.t, err) - e2e.RunQrepFlowWorkflow(env, qrepConfig) - - // Verify workflow completes without error - require.True(s.t, env.IsWorkflowCompleted()) - - err = env.GetWorkflowError() - require.NoError(s.t, err) + tc := e2e.NewTemporalClient(s.t) + env := e2e.RunQrepFlowWorkflow(tc, qrepConfig) + e2e.EnvWaitForFinished(s.t, env, 3*time.Minute) + require.NoError(s.t, env.Error()) err = s.comparePGTables(srcSchemaQualified, dstSchemaQualified, "*") require.NoError(s.t, err) } func (s PeerFlowE2ETestSuitePG) Test_PeerDB_Columns_QRep_PG() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) - numRows := 10 srcTable := "test_qrep_columns_pg_1" @@ -289,21 +282,16 @@ func (s PeerFlowE2ETestSuitePG) Test_PeerDB_Columns_QRep_PG() { ) require.NoError(s.t, err) - e2e.RunQrepFlowWorkflow(env, qrepConfig) - - // Verify workflow completes without error - require.True(s.t, env.IsWorkflowCompleted()) - - err = env.GetWorkflowError() - require.NoError(s.t, err) + tc := e2e.NewTemporalClient(s.t) + env := e2e.RunQrepFlowWorkflow(tc, qrepConfig) + e2e.EnvWaitForFinished(s.t, env, 3*time.Minute) + require.NoError(s.t, env.Error()) err = s.checkSyncedAt(dstSchemaQualified) require.NoError(s.t, err) } func (s PeerFlowE2ETestSuitePG) Test_No_Rows_QRep_PG() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) - numRows := 0 srcTable := "test_no_rows_qrep_pg_1" @@ -331,11 +319,8 @@ func (s PeerFlowE2ETestSuitePG) Test_No_Rows_QRep_PG() { ) require.NoError(s.t, err) - e2e.RunQrepFlowWorkflow(env, qrepConfig) - - // Verify workflow completes without error - require.True(s.t, env.IsWorkflowCompleted()) - - err = env.GetWorkflowError() - require.NoError(s.t, err) + tc := e2e.NewTemporalClient(s.t) + env := e2e.RunQrepFlowWorkflow(tc, qrepConfig) + e2e.EnvWaitForFinished(s.t, env, 3*time.Minute) + require.NoError(s.t, env.Error()) } diff --git a/flow/e2e/s3/cdc_s3_test.go b/flow/e2e/s3/cdc_s3_test.go index 8a976b8a0e..9f49195b3a 100644 --- a/flow/e2e/s3/cdc_s3_test.go +++ b/flow/e2e/s3/cdc_s3_test.go @@ -20,7 +20,7 @@ func (s PeerFlowE2ETestSuiteS3) attachSuffix(input string) string { } func (s PeerFlowE2ETestSuiteS3) Test_Complete_Simple_Flow_S3() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcTableName := s.attachSchemaSuffix("test_simple_flow_s3") dstTableName := fmt.Sprintf("%s.%s", "peerdb_test_s3", "test_simple_flow_s3") @@ -42,31 +42,29 @@ func (s PeerFlowE2ETestSuiteS3) Test_Complete_Simple_Flow_S3() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 5 - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - // insert 20 rows - for i := 1; i <= 20; i++ { - testKey := fmt.Sprintf("test_key_%d", i) - testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.conn.Conn().Exec(context.Background(), fmt.Sprintf(` + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + // insert 20 rows + for i := 1; i <= 20; i++ { + testKey := fmt.Sprintf("test_key_%d", i) + testValue := fmt.Sprintf("test_value_%d", i) + _, err = s.conn.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s (key, value) VALUES ($1, $2) `, srcTableName), testKey, testValue) - e2e.EnvNoError(s.t, env, err) - } e2e.EnvNoError(s.t, env, err) + } + e2e.EnvNoError(s.t, env, err) - e2e.EnvWaitFor(s.t, env, 2*time.Minute, "waiting for blobs", func() bool { - ctx, cancel := context.WithTimeout(context.Background(), 25*time.Second) - defer cancel() - files, err := s.s3Helper.ListAllFiles(ctx, flowJobName) - s.t.Logf("Files in Test_Complete_Simple_Flow_S3 %s: %d", flowJobName, len(files)) - e2e.EnvNoError(s.t, env, err) - return len(files) == 4 - }) + e2e.EnvWaitFor(s.t, env, 2*time.Minute, "waiting for blobs", func() bool { + ctx, cancel := context.WithTimeout(context.Background(), 25*time.Second) + defer cancel() + files, err := s.s3Helper.ListAllFiles(ctx, flowJobName) + s.t.Logf("Files in Test_Complete_Simple_Flow_S3 %s: %d", flowJobName, len(files)) + e2e.EnvNoError(s.t, env, err) + return len(files) == 4 + }) - env.CancelWorkflow() - }() + env.Cancel() - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) e2e.RequireEnvCanceled(s.t, env) } diff --git a/flow/e2e/s3/qrep_flow_s3_test.go b/flow/e2e/s3/qrep_flow_s3_test.go index 54a66f0ed0..240f3a78b9 100644 --- a/flow/e2e/s3/qrep_flow_s3_test.go +++ b/flow/e2e/s3/qrep_flow_s3_test.go @@ -104,7 +104,7 @@ func (s PeerFlowE2ETestSuiteS3) Test_Complete_QRep_Flow_S3() { s.t.Skip("Skipping S3 test") } - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) jobName := "test_complete_flow_s3" schemaQualifiedName := fmt.Sprintf("e2e_test_%s.%s", s.suffix, jobName) @@ -125,13 +125,9 @@ func (s PeerFlowE2ETestSuiteS3) Test_Complete_QRep_Flow_S3() { require.NoError(s.t, err) qrepConfig.StagingPath = s.s3Helper.s3Config.Url - e2e.RunQrepFlowWorkflow(env, qrepConfig) - - // Verify workflow completes without error - require.True(s.t, env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - require.NoError(s.t, err) + env := e2e.RunQrepFlowWorkflow(tc, qrepConfig) + e2e.EnvWaitForFinished(s.t, env, 3*time.Minute) + require.NoError(s.t, env.Error()) // Verify destination has 1 file // make context with timeout @@ -150,7 +146,7 @@ func (s PeerFlowE2ETestSuiteS3) Test_Complete_QRep_Flow_S3_CTID() { s.t.Skip("Skipping S3 test") } - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) jobName := "test_complete_flow_s3_ctid" schemaQualifiedName := fmt.Sprintf("e2e_test_%s.%s", s.suffix, jobName) @@ -173,13 +169,9 @@ func (s PeerFlowE2ETestSuiteS3) Test_Complete_QRep_Flow_S3_CTID() { qrepConfig.InitialCopyOnly = true qrepConfig.WatermarkColumn = "ctid" - e2e.RunQrepFlowWorkflow(env, qrepConfig) - - // Verify workflow completes without error - require.True(s.t, env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - - require.NoError(s.t, err) + env := e2e.RunQrepFlowWorkflow(tc, qrepConfig) + e2e.EnvWaitForFinished(s.t, env, 3*time.Minute) + require.NoError(s.t, env.Error()) // Verify destination has 1 file // make context with timeout diff --git a/flow/e2e/snowflake/peer_flow_sf_test.go b/flow/e2e/snowflake/peer_flow_sf_test.go index 40dbefaa2e..9eaf491e47 100644 --- a/flow/e2e/snowflake/peer_flow_sf_test.go +++ b/flow/e2e/snowflake/peer_flow_sf_test.go @@ -127,7 +127,7 @@ func SetupSuite(t *testing.T) PeerFlowE2ETestSuiteSF { } func (s PeerFlowE2ETestSuiteSF) Test_Complete_Simple_Flow_SF() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) tableName := "test_simple_flow_sf" srcTableName := s.attachSchemaSuffix(tableName) @@ -151,26 +151,24 @@ func (s PeerFlowE2ETestSuiteSF) Test_Complete_Simple_Flow_SF() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + // wait for PeerFlowStatusQuery to finish setup // and then insert 20 rows into the source table - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - // insert 20 rows into the source table - for i := range 20 { - testKey := fmt.Sprintf("test_key_%d", i) - testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` - INSERT INTO %s (key, value) VALUES ($1, $2) - `, srcTableName), testKey, testValue) - e2e.EnvNoError(s.t, env, err) - } - s.t.Log("Inserted 20 rows into the source table") - e2e.EnvWaitForEqualTables(env, s, "normalize table", tableName, "id,key,value") + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + // insert 20 rows into the source table + for i := range 20 { + testKey := fmt.Sprintf("test_key_%d", i) + testValue := fmt.Sprintf("test_value_%d", i) + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s (key, value) VALUES ($1, $2) + `, srcTableName), testKey, testValue) + e2e.EnvNoError(s.t, env, err) + } + s.t.Log("Inserted 20 rows into the source table") + e2e.EnvWaitForEqualTables(env, s, "normalize table", tableName, "id,key,value") - env.CancelWorkflow() - }() + env.Cancel() - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) e2e.RequireEnvCanceled(s.t, env) // check the number of rows where _PEERDB_SYNCED_AT is newer than 5 mins ago @@ -184,7 +182,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Complete_Simple_Flow_SF() { } func (s PeerFlowE2ETestSuiteSF) Test_Flow_ReplicaIdentity_Index_No_Pkey() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) tableName := "test_replica_identity_no_pkey" srcTableName := s.attachSchemaSuffix(tableName) @@ -211,34 +209,32 @@ func (s PeerFlowE2ETestSuiteSF) Test_Flow_ReplicaIdentity_Index_No_Pkey() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and then insert 20 rows into the source table - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - // insert 20 rows into the source table - for i := range 20 { - testKey := fmt.Sprintf("test_key_%d", i) - testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` - INSERT INTO %s (id, key, value) VALUES ($1, $2, $3) - `, srcTableName), i, testKey, testValue) - e2e.EnvNoError(s.t, env, err) - } - s.t.Log("Inserted 20 rows into the source table") + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + // insert 20 rows into the source table + for i := range 20 { + testKey := fmt.Sprintf("test_key_%d", i) + testValue := fmt.Sprintf("test_value_%d", i) + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s (id, key, value) VALUES ($1, $2, $3) + `, srcTableName), i, testKey, testValue) + e2e.EnvNoError(s.t, env, err) + } + s.t.Log("Inserted 20 rows into the source table") - e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize insert", func() bool { - count, err := s.sfHelper.CountRows("test_replica_identity_no_pkey") - return err == nil && count == 20 - }) - env.CancelWorkflow() - }() + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize insert", func() bool { + count, err := s.sfHelper.CountRows("test_replica_identity_no_pkey") + return err == nil && count == 20 + }) + env.Cancel() - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuiteSF) Test_Invalid_Geo_SF_Avro_CDC() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) tableName := "test_invalid_geo_sf_avro_cdc" srcTableName := s.attachSchemaSuffix(tableName) @@ -262,63 +258,61 @@ func (s PeerFlowE2ETestSuiteSF) Test_Invalid_Geo_SF_Avro_CDC() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and then insert 10 rows into the source table - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - // insert 4 invalid shapes and 6 valid shapes into the source table - for range 4 { - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + // insert 4 invalid shapes and 6 valid shapes into the source table + for range 4 { + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s (line,poly) VALUES ($1,$2) `, srcTableName), "010200000001000000000000000000F03F0000000000000040", - "0103000020e6100000010000000c0000001a8361d35dc64140afdb8d2b1bc3c9bf1b8ed4685fc641405ba64c"+ - "579dc2c9bf6a6ad95a5fc64140cd82767449c2c9bf9570fbf85ec641408a07944db9c2c9bf729a18a55ec6414021b8b748c7c2c9bfba46de4c"+ - "5fc64140f2567052abc2c9bf2df9c5925fc641409394e16573c2c9bf2df9c5925fc6414049eceda9afc1c9bfdd1cc1a05fc64140fe43faedebc0"+ - "c9bf4694f6065fc64140fe43faedebc0c9bfffe7305f5ec641406693d6f2ddc0c9bf1a8361d35dc64140afdb8d2b1bc3c9bf", - ) - e2e.EnvNoError(s.t, env, err) - } - s.t.Log("Inserted 4 invalid geography rows into the source table") - for range 6 { - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + "0103000020e6100000010000000c0000001a8361d35dc64140afdb8d2b1bc3c9bf1b8ed4685fc641405ba64c"+ + "579dc2c9bf6a6ad95a5fc64140cd82767449c2c9bf9570fbf85ec641408a07944db9c2c9bf729a18a55ec6414021b8b748c7c2c9bfba46de4c"+ + "5fc64140f2567052abc2c9bf2df9c5925fc641409394e16573c2c9bf2df9c5925fc6414049eceda9afc1c9bfdd1cc1a05fc64140fe43faedebc0"+ + "c9bf4694f6065fc64140fe43faedebc0c9bfffe7305f5ec641406693d6f2ddc0c9bf1a8361d35dc64140afdb8d2b1bc3c9bf", + ) + e2e.EnvNoError(s.t, env, err) + } + s.t.Log("Inserted 4 invalid geography rows into the source table") + for range 6 { + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s (line,poly) VALUES ($1,$2) `, srcTableName), "010200000002000000000000000000F03F000000000000004000000000000008400000000000001040", - "010300000001000000050000000000000000000000000000000000000000000000"+ - "00000000000000000000f03f000000000000f03f000000000000f03f0000000000"+ - "00f03f000000000000000000000000000000000000000000000000") - e2e.EnvNoError(s.t, env, err) - } - s.t.Log("Inserted 6 valid geography rows and 10 total rows into source") + "010300000001000000050000000000000000000000000000000000000000000000"+ + "00000000000000000000f03f000000000000f03f000000000000f03f0000000000"+ + "00f03f000000000000000000000000000000000000000000000000") + e2e.EnvNoError(s.t, env, err) + } + s.t.Log("Inserted 6 valid geography rows and 10 total rows into source") - e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize shapes", func() bool { - // We inserted 4 invalid shapes in each, - // which should be filtered out as null on destination. - lineCount, err := s.sfHelper.CountNonNullRows("test_invalid_geo_sf_avro_cdc", "line") - if err != nil { - return false - } + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize shapes", func() bool { + // We inserted 4 invalid shapes in each, + // which should be filtered out as null on destination. + lineCount, err := s.sfHelper.CountNonNullRows("test_invalid_geo_sf_avro_cdc", "line") + if err != nil { + return false + } - polyCount, err := s.sfHelper.CountNonNullRows("test_invalid_geo_sf_avro_cdc", "poly") - if err != nil { - return false - } + polyCount, err := s.sfHelper.CountNonNullRows("test_invalid_geo_sf_avro_cdc", "poly") + if err != nil { + return false + } - if lineCount != 6 || polyCount != 6 { - s.t.Logf("wrong counts, expect 6 lines 6 polies, not %d lines %d polies", lineCount, polyCount) - return false - } else { - return true - } - }) - env.CancelWorkflow() - }() + if lineCount != 6 || polyCount != 6 { + s.t.Logf("wrong counts, expect 6 lines 6 polies, not %d lines %d polies", lineCount, polyCount) + return false + } else { + return true + } + }) + env.Cancel() - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuiteSF) Test_Toast_SF() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcTableName := s.attachSchemaSuffix("test_toast_sf_1") dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_1") @@ -342,36 +336,34 @@ func (s PeerFlowE2ETestSuiteSF) Test_Toast_SF() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and execute a transaction touching toast columns - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - /* - Executing a transaction which - 1. changes both toast column - 2. changes no toast column - 2. changes 1 toast column - */ - _, err = s.Conn().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)) - e2e.EnvNoError(s.t, env, err) - s.t.Log("Executed a transaction touching toast columns") - e2e.EnvWaitForEqualTables(env, s, "normalizing tx", "test_toast_sf_1", `id,t1,t2,k`) - env.CancelWorkflow() - }() + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + /* + Executing a transaction which + 1. changes both toast column + 2. changes no toast column + 2. changes 1 toast column + */ + _, err = s.Conn().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)) + e2e.EnvNoError(s.t, env, err) + s.t.Log("Executed a transaction touching toast columns") + e2e.EnvWaitForEqualTables(env, s, "normalizing tx", "test_toast_sf_1", `id,t1,t2,k`) + env.Cancel() - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuiteSF) Test_Toast_Advance_1_SF() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcTableName := s.attachSchemaSuffix("test_toast_sf_3") dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_3") @@ -395,42 +387,40 @@ func (s PeerFlowE2ETestSuiteSF) Test_Toast_Advance_1_SF() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and execute a transaction touching toast columns - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - // complex transaction with random DMLs on a table with toast columns - _, err = s.Conn().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)) - e2e.EnvNoError(s.t, env, err) - s.t.Log("Executed a transaction touching toast columns") - e2e.EnvWaitForEqualTables(env, s, "normalizing tx", "test_toast_sf_3", `id,t1,t2,k`) - env.CancelWorkflow() - }() + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + // complex transaction with random DMLs on a table with toast columns + _, err = s.Conn().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)) + e2e.EnvNoError(s.t, env, err) + s.t.Log("Executed a transaction touching toast columns") + e2e.EnvWaitForEqualTables(env, s, "normalizing tx", "test_toast_sf_3", `id,t1,t2,k`) + env.Cancel() - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuiteSF) Test_Toast_Advance_2_SF() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcTableName := s.attachSchemaSuffix("test_toast_sf_4") dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_4") @@ -453,36 +443,34 @@ func (s PeerFlowE2ETestSuiteSF) Test_Toast_Advance_2_SF() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and execute a transaction touching toast columns - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - // complex transaction with random DMLs on a table with toast columns - _, err = s.Conn().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)) - e2e.EnvNoError(s.t, env, err) - s.t.Log("Executed a transaction touching toast columns") - e2e.EnvWaitForEqualTables(env, s, "normalizing tx", "test_toast_sf_4", `id,t1,k`) - env.CancelWorkflow() - }() + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + // complex transaction with random DMLs on a table with toast columns + _, err = s.Conn().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)) + e2e.EnvNoError(s.t, env, err) + s.t.Log("Executed a transaction touching toast columns") + e2e.EnvWaitForEqualTables(env, s, "normalizing tx", "test_toast_sf_4", `id,t1,k`) + env.Cancel() - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuiteSF) Test_Toast_Advance_3_SF() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcTableName := s.attachSchemaSuffix("test_toast_sf_5") dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_5") @@ -506,36 +494,34 @@ func (s PeerFlowE2ETestSuiteSF) Test_Toast_Advance_3_SF() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and execute a transaction touching toast columns - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - /* - transaction updating a single row - multiple times with changed/unchanged toast columns - */ - _, err = s.Conn().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)) - e2e.EnvNoError(s.t, env, err) - s.t.Log("Executed a transaction touching toast columns") + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + /* + transaction updating a single row + multiple times with changed/unchanged toast columns + */ + _, err = s.Conn().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)) + e2e.EnvNoError(s.t, env, err) + s.t.Log("Executed a transaction touching toast columns") + + e2e.EnvWaitForEqualTables(env, s, "normalizing tx", "test_toast_sf_5", `id,t1,t2,k`) + env.Cancel() - e2e.EnvWaitForEqualTables(env, s, "normalizing tx", "test_toast_sf_5", `id,t1,t2,k`) - env.CancelWorkflow() - }() - - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuiteSF) Test_Types_SF() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcTableName := s.attachSchemaSuffix("test_types_sf") dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_types_sf") @@ -566,71 +552,69 @@ func (s PeerFlowE2ETestSuiteSF) Test_Types_SF() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and execute a transaction touching toast columns - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - /* test inserting various types*/ - _, err = s.Conn().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,100.24553,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'), - 'POINT(1 2)','POINT(40.7128 -74.0060)','POLYGON((0 0, 0 1, 1 1, 1 0, 0 0))', - 'LINESTRING(-74.0060 40.7128, -73.9352 40.7306, -73.9123 40.7831)','LINESTRING(0 0, 1 1, 2 2)', - 'POLYGON((-74.0060 40.7128, -73.9352 40.7306, -73.9123 40.7831, -74.0060 40.7128))', 'happy','"a"=>"a\"quote\"", "b"=>NULL', - '{2020-01-01, 2020-01-02}'::date[], - '{"2020-01-01 01:01:01+00", "2020-01-02 01:01:01+00"}'::timestamptz[], - '{"2020-01-01 01:01:01", "2020-01-02 01:01:01"}'::timestamp[], - '{true, false}'::boolean[], - '{1,2}'::smallint[]; - `, srcTableName)) - e2e.EnvNoError(s.t, env, err) - - e2e.EnvWaitFor(s.t, env, 2*time.Minute, "normalize types", func() bool { - 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", "c42", "c43", "c44", "c45", "c46", "c47", "c48", "c49", - "c50", "c51", "c52", "c53", "c54", - }) - if err != nil { - s.t.Log(err) - return false - } + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + /* test inserting various types*/ + _, err = s.Conn().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,100.24553,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'), + 'POINT(1 2)','POINT(40.7128 -74.0060)','POLYGON((0 0, 0 1, 1 1, 1 0, 0 0))', + 'LINESTRING(-74.0060 40.7128, -73.9352 40.7306, -73.9123 40.7831)','LINESTRING(0 0, 1 1, 2 2)', + 'POLYGON((-74.0060 40.7128, -73.9352 40.7306, -73.9123 40.7831, -74.0060 40.7128))', 'happy','"a"=>"a\"quote\"", "b"=>NULL', + '{2020-01-01, 2020-01-02}'::date[], + '{"2020-01-01 01:01:01+00", "2020-01-02 01:01:01+00"}'::timestamptz[], + '{"2020-01-01 01:01:01", "2020-01-02 01:01:01"}'::timestamp[], + '{true, false}'::boolean[], + '{1,2}'::smallint[]; + `, srcTableName)) + e2e.EnvNoError(s.t, env, err) + + e2e.EnvWaitFor(s.t, env, 2*time.Minute, "normalize types", func() bool { + 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", "c42", "c43", "c44", "c45", "c46", "c47", "c48", "c49", + "c50", "c51", "c52", "c53", "c54", + }) + if err != nil { + s.t.Log(err) + return false + } - // check if JSON on snowflake side is a good JSON - if err := s.checkJSONValue(dstTableName, "c17", "sai", "1"); err != nil { - return false - } + // check if JSON on snowflake side is a good JSON + if err := s.checkJSONValue(dstTableName, "c17", "sai", "1"); err != nil { + return false + } - // check if HSTORE on snowflake is a good JSON - if err := s.checkJSONValue(dstTableName, "c49", "a", `"a\"quote\""`); err != nil { - return false - } + // check if HSTORE on snowflake is a good JSON + if err := s.checkJSONValue(dstTableName, "c49", "a", `"a\"quote\""`); err != nil { + return false + } - if err := s.checkJSONValue(dstTableName, "c49", "b", "null"); err != nil { - return false - } + if err := s.checkJSONValue(dstTableName, "c49", "b", "null"); err != nil { + return false + } - return noNulls - }) + return noNulls + }) - env.CancelWorkflow() - }() + env.Cancel() - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuiteSF) Test_Multi_Table_SF() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcTable1Name := s.attachSchemaSuffix("test1_sf") srcTable2Name := s.attachSchemaSuffix("test2_sf") @@ -652,39 +636,37 @@ func (s PeerFlowE2ETestSuiteSF) Test_Multi_Table_SF() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and execute a transaction touching toast columns - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - /* inserting across multiple tables*/ - _, err = s.Conn().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)) - e2e.EnvNoError(s.t, env, err) + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + /* inserting across multiple tables*/ + _, err = s.Conn().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)) + e2e.EnvNoError(s.t, env, err) - e2e.EnvWaitFor(s.t, env, 2*time.Minute, "normalize both tables", func() bool { - count1, err := s.sfHelper.CountRows("test1_sf") - if err != nil { - return false - } - count2, err := s.sfHelper.CountRows("test2_sf") - if err != nil { - return false - } + e2e.EnvWaitFor(s.t, env, 2*time.Minute, "normalize both tables", func() bool { + count1, err := s.sfHelper.CountRows("test1_sf") + if err != nil { + return false + } + count2, err := s.sfHelper.CountRows("test2_sf") + if err != nil { + return false + } - return count1 == 1 && count2 == 1 - }) + return count1 == 1 && count2 == 1 + }) - env.CancelWorkflow() - }() + env.Cancel() - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuiteSF) Test_Simple_Schema_Changes_SF() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcTableName := s.attachSchemaSuffix("test_simple_schema_changes") dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_simple_schema_changes") @@ -706,199 +688,197 @@ func (s PeerFlowE2ETestSuiteSF) Test_Simple_Schema_Changes_SF() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and then insert and mutate schema repeatedly. - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c1) VALUES ($1)`, srcTableName), 1) - e2e.EnvNoError(s.t, env, err) - s.t.Log("Inserted initial row in the source table") - - e2e.EnvWaitForEqualTables(env, s, "normalize reinsert", "test_simple_schema_changes", "id,c1") - - expectedTableSchema := &protos.TableSchema{ - TableIdentifier: strings.ToUpper(dstTableName), - Columns: []*protos.FieldDescription{ - { - Name: "ID", - Type: string(qvalue.QValueKindNumeric), - TypeModifier: -1, - }, - { - Name: "C1", - Type: string(qvalue.QValueKindNumeric), - TypeModifier: -1, - }, - { - Name: "_PEERDB_IS_DELETED", - Type: string(qvalue.QValueKindBoolean), - TypeModifier: -1, - }, - { - Name: "_PEERDB_SYNCED_AT", - Type: string(qvalue.QValueKindTimestamp), - TypeModifier: -1, - }, + e2e.EnvNoError(s.t, env, err) + s.t.Log("Inserted initial row in the source table") + + e2e.EnvWaitForEqualTables(env, s, "normalize reinsert", "test_simple_schema_changes", "id,c1") + + expectedTableSchema := &protos.TableSchema{ + TableIdentifier: strings.ToUpper(dstTableName), + Columns: []*protos.FieldDescription{ + { + Name: "ID", + Type: string(qvalue.QValueKindNumeric), + TypeModifier: -1, }, - } - output, err := s.connector.GetTableSchema(context.Background(), &protos.GetTableSchemaBatchInput{ - TableIdentifiers: []string{dstTableName}, - }) - e2e.EnvNoError(s.t, env, err) - e2e.EnvTrue(s.t, env, e2e.CompareTableSchemas(expectedTableSchema, output.TableNameSchemaMapping[dstTableName])) + { + Name: "C1", + Type: string(qvalue.QValueKindNumeric), + TypeModifier: -1, + }, + { + Name: "_PEERDB_IS_DELETED", + Type: string(qvalue.QValueKindBoolean), + TypeModifier: -1, + }, + { + Name: "_PEERDB_SYNCED_AT", + Type: string(qvalue.QValueKindTimestamp), + TypeModifier: -1, + }, + }, + } + output, err := s.connector.GetTableSchema(context.Background(), &protos.GetTableSchemaBatchInput{ + TableIdentifiers: []string{dstTableName}, + }) + e2e.EnvNoError(s.t, env, err) + e2e.EnvTrue(s.t, env, e2e.CompareTableSchemas(expectedTableSchema, output.TableNameSchemaMapping[dstTableName])) - // alter source table, add column c2 and insert another row. - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + // alter source table, add column c2 and insert another row. + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` ALTER TABLE %s ADD COLUMN c2 BIGINT`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - s.t.Log("Altered source table, added column c2") - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + s.t.Log("Altered source table, added column c2") + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c1,c2) VALUES ($1,$2)`, srcTableName), 2, 2) - e2e.EnvNoError(s.t, env, err) - s.t.Log("Inserted row with added c2 in the source table") - - // verify we got our two rows, if schema did not match up it will error. - e2e.EnvWaitForEqualTables(env, s, "normalize altered row", "test_simple_schema_changes", "id,c1,c2") - expectedTableSchema = &protos.TableSchema{ - TableIdentifier: strings.ToUpper(dstTableName), - Columns: []*protos.FieldDescription{ - { - Name: "ID", - Type: string(qvalue.QValueKindNumeric), - TypeModifier: -1, - }, - { - Name: "C1", - Type: string(qvalue.QValueKindNumeric), - TypeModifier: -1, - }, - { - Name: "_PEERDB_SYNCED_AT", - Type: string(qvalue.QValueKindTimestamp), - TypeModifier: -1, - }, - { - Name: "C2", - Type: string(qvalue.QValueKindNumeric), - TypeModifier: -1, - }, + e2e.EnvNoError(s.t, env, err) + s.t.Log("Inserted row with added c2 in the source table") + + // verify we got our two rows, if schema did not match up it will error. + e2e.EnvWaitForEqualTables(env, s, "normalize altered row", "test_simple_schema_changes", "id,c1,c2") + expectedTableSchema = &protos.TableSchema{ + TableIdentifier: strings.ToUpper(dstTableName), + Columns: []*protos.FieldDescription{ + { + Name: "ID", + Type: string(qvalue.QValueKindNumeric), + TypeModifier: -1, }, - } - output, err = s.connector.GetTableSchema(context.Background(), &protos.GetTableSchemaBatchInput{ - TableIdentifiers: []string{dstTableName}, - }) - e2e.EnvNoError(s.t, env, err) - e2e.EnvTrue(s.t, env, e2e.CompareTableSchemas(expectedTableSchema, output.TableNameSchemaMapping[dstTableName])) - e2e.EnvEqualTables(env, s, "test_simple_schema_changes", "id,c1,c2") + { + Name: "C1", + Type: string(qvalue.QValueKindNumeric), + TypeModifier: -1, + }, + { + Name: "_PEERDB_SYNCED_AT", + Type: string(qvalue.QValueKindTimestamp), + TypeModifier: -1, + }, + { + Name: "C2", + Type: string(qvalue.QValueKindNumeric), + TypeModifier: -1, + }, + }, + } + output, err = s.connector.GetTableSchema(context.Background(), &protos.GetTableSchemaBatchInput{ + TableIdentifiers: []string{dstTableName}, + }) + e2e.EnvNoError(s.t, env, err) + e2e.EnvTrue(s.t, env, e2e.CompareTableSchemas(expectedTableSchema, output.TableNameSchemaMapping[dstTableName])) + e2e.EnvEqualTables(env, s, "test_simple_schema_changes", "id,c1,c2") - // alter source table, add column c3, drop column c2 and insert another row. - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + // alter source table, add column c3, drop column c2 and insert another row. + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` ALTER TABLE %s DROP COLUMN c2, ADD COLUMN c3 BIGINT`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - s.t.Log("Altered source table, dropped column c2 and added column c3") - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + s.t.Log("Altered source table, dropped column c2 and added column c3") + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c1,c3) VALUES ($1,$2)`, srcTableName), 3, 3) - e2e.EnvNoError(s.t, env, err) - s.t.Log("Inserted row with added c3 in the source table") - - // verify we got our two rows, if schema did not match up it will error. - e2e.EnvWaitForEqualTables(env, s, "normalize dropped c2 column", "test_simple_schema_changes", "id,c1,c3") - expectedTableSchema = &protos.TableSchema{ - TableIdentifier: strings.ToUpper(dstTableName), - Columns: []*protos.FieldDescription{ - { - Name: "ID", - Type: string(qvalue.QValueKindNumeric), - TypeModifier: -1, - }, - { - Name: "C1", - Type: string(qvalue.QValueKindNumeric), - TypeModifier: -1, - }, - { - Name: "_PEERDB_SYNCED_AT", - Type: string(qvalue.QValueKindTimestamp), - TypeModifier: -1, - }, - { - Name: "C2", - Type: string(qvalue.QValueKindNumeric), - TypeModifier: -1, - }, - { - Name: "C3", - Type: string(qvalue.QValueKindNumeric), - TypeModifier: -1, - }, + e2e.EnvNoError(s.t, env, err) + s.t.Log("Inserted row with added c3 in the source table") + + // verify we got our two rows, if schema did not match up it will error. + e2e.EnvWaitForEqualTables(env, s, "normalize dropped c2 column", "test_simple_schema_changes", "id,c1,c3") + expectedTableSchema = &protos.TableSchema{ + TableIdentifier: strings.ToUpper(dstTableName), + Columns: []*protos.FieldDescription{ + { + Name: "ID", + Type: string(qvalue.QValueKindNumeric), + TypeModifier: -1, }, - } - output, err = s.connector.GetTableSchema(context.Background(), &protos.GetTableSchemaBatchInput{ - TableIdentifiers: []string{dstTableName}, - }) - e2e.EnvNoError(s.t, env, err) - e2e.EnvTrue(s.t, env, e2e.CompareTableSchemas(expectedTableSchema, output.TableNameSchemaMapping[dstTableName])) - e2e.EnvEqualTables(env, s, "test_simple_schema_changes", "id,c1,c3") + { + Name: "C1", + Type: string(qvalue.QValueKindNumeric), + TypeModifier: -1, + }, + { + Name: "_PEERDB_SYNCED_AT", + Type: string(qvalue.QValueKindTimestamp), + TypeModifier: -1, + }, + { + Name: "C2", + Type: string(qvalue.QValueKindNumeric), + TypeModifier: -1, + }, + { + Name: "C3", + Type: string(qvalue.QValueKindNumeric), + TypeModifier: -1, + }, + }, + } + output, err = s.connector.GetTableSchema(context.Background(), &protos.GetTableSchemaBatchInput{ + TableIdentifiers: []string{dstTableName}, + }) + e2e.EnvNoError(s.t, env, err) + e2e.EnvTrue(s.t, env, e2e.CompareTableSchemas(expectedTableSchema, output.TableNameSchemaMapping[dstTableName])) + e2e.EnvEqualTables(env, s, "test_simple_schema_changes", "id,c1,c3") - // alter source table, drop column c3 and insert another row. - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + // alter source table, drop column c3 and insert another row. + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` ALTER TABLE %s DROP COLUMN c3`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - s.t.Log("Altered source table, dropped column c3") - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + s.t.Log("Altered source table, dropped column c3") + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c1) VALUES ($1)`, srcTableName), 4) - e2e.EnvNoError(s.t, env, err) - s.t.Log("Inserted row after dropping all columns in the source table") - - // verify we got our two rows, if schema did not match up it will error. - e2e.EnvWaitForEqualTables(env, s, "normalize dropped c3 column", "test_simple_schema_changes", "id,c1") - expectedTableSchema = &protos.TableSchema{ - TableIdentifier: strings.ToUpper(dstTableName), - Columns: []*protos.FieldDescription{ - { - Name: "ID", - Type: string(qvalue.QValueKindNumeric), - TypeModifier: -1, - }, - { - Name: "C1", - Type: string(qvalue.QValueKindNumeric), - TypeModifier: -1, - }, - { - Name: "_PEERDB_SYNCED_AT", - Type: string(qvalue.QValueKindTimestamp), - TypeModifier: -1, - }, - { - Name: "C2", - Type: string(qvalue.QValueKindNumeric), - TypeModifier: -1, - }, - { - Name: "C3", - Type: string(qvalue.QValueKindNumeric), - TypeModifier: -1, - }, + e2e.EnvNoError(s.t, env, err) + s.t.Log("Inserted row after dropping all columns in the source table") + + // verify we got our two rows, if schema did not match up it will error. + e2e.EnvWaitForEqualTables(env, s, "normalize dropped c3 column", "test_simple_schema_changes", "id,c1") + expectedTableSchema = &protos.TableSchema{ + TableIdentifier: strings.ToUpper(dstTableName), + Columns: []*protos.FieldDescription{ + { + Name: "ID", + Type: string(qvalue.QValueKindNumeric), + TypeModifier: -1, }, - } - output, err = s.connector.GetTableSchema(context.Background(), &protos.GetTableSchemaBatchInput{ - TableIdentifiers: []string{dstTableName}, - }) - e2e.EnvNoError(s.t, env, err) - e2e.EnvTrue(s.t, env, e2e.CompareTableSchemas(expectedTableSchema, output.TableNameSchemaMapping[dstTableName])) - e2e.EnvEqualTables(env, s, "test_simple_schema_changes", "id,c1") + { + Name: "C1", + Type: string(qvalue.QValueKindNumeric), + TypeModifier: -1, + }, + { + Name: "_PEERDB_SYNCED_AT", + Type: string(qvalue.QValueKindTimestamp), + TypeModifier: -1, + }, + { + Name: "C2", + Type: string(qvalue.QValueKindNumeric), + TypeModifier: -1, + }, + { + Name: "C3", + Type: string(qvalue.QValueKindNumeric), + TypeModifier: -1, + }, + }, + } + output, err = s.connector.GetTableSchema(context.Background(), &protos.GetTableSchemaBatchInput{ + TableIdentifiers: []string{dstTableName}, + }) + e2e.EnvNoError(s.t, env, err) + e2e.EnvTrue(s.t, env, e2e.CompareTableSchemas(expectedTableSchema, output.TableNameSchemaMapping[dstTableName])) + e2e.EnvEqualTables(env, s, "test_simple_schema_changes", "id,c1") - env.CancelWorkflow() - }() + env.Cancel() - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuiteSF) Test_Composite_PKey_SF() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcTableName := s.attachSchemaSuffix("test_simple_cpkey") dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_simple_cpkey") @@ -923,38 +903,36 @@ func (s PeerFlowE2ETestSuiteSF) Test_Composite_PKey_SF() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and then insert, update and delete rows in the table. - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - // insert 10 rows into the source table - for i := range 10 { - testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + // insert 10 rows into the source table + for i := range 10 { + testValue := fmt.Sprintf("test_value_%d", i) + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c2,t) VALUES ($1,$2) `, srcTableName), i, testValue) - e2e.EnvNoError(s.t, env, err) - } - s.t.Log("Inserted 10 rows into the source table") + e2e.EnvNoError(s.t, env, err) + } + s.t.Log("Inserted 10 rows into the source table") - e2e.EnvWaitForEqualTables(env, s, "normalize table", "test_simple_cpkey", "id,c1,c2,t") + e2e.EnvWaitForEqualTables(env, s, "normalize table", "test_simple_cpkey", "id,c1,c2,t") - _, err := s.Conn().Exec(context.Background(), - fmt.Sprintf(`UPDATE %s SET c1=c1+1 WHERE MOD(c2,2)=$1`, srcTableName), 1) - e2e.EnvNoError(s.t, env, err) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=$1`, srcTableName), 0) - e2e.EnvNoError(s.t, env, err) - e2e.EnvWaitForEqualTables(env, s, "normalize update/delete", "test_simple_cpkey", "id,c1,c2,t") + _, err = s.Conn().Exec(context.Background(), + fmt.Sprintf(`UPDATE %s SET c1=c1+1 WHERE MOD(c2,2)=$1`, srcTableName), 1) + e2e.EnvNoError(s.t, env, err) + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=$1`, srcTableName), 0) + e2e.EnvNoError(s.t, env, err) + e2e.EnvWaitForEqualTables(env, s, "normalize update/delete", "test_simple_cpkey", "id,c1,c2,t") - env.CancelWorkflow() - }() + env.Cancel() - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuiteSF) Test_Composite_PKey_Toast_1_SF() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcTableName := s.attachSchemaSuffix("test_cpkey_toast1") dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_cpkey_toast1") @@ -980,42 +958,40 @@ func (s PeerFlowE2ETestSuiteSF) Test_Composite_PKey_Toast_1_SF() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and then insert, update and delete rows in the table. - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - rowsTx, err := s.Conn().Begin(context.Background()) - e2e.EnvNoError(s.t, env, err) - - // insert 10 rows into the source table - for i := range 10 { - testValue := fmt.Sprintf("test_value_%d", i) - _, err = rowsTx.Exec(context.Background(), fmt.Sprintf(` + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + rowsTx, err := s.Conn().Begin(context.Background()) + e2e.EnvNoError(s.t, env, err) + + // insert 10 rows into the source table + for i := range 10 { + testValue := fmt.Sprintf("test_value_%d", i) + _, err = rowsTx.Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c2,t,t2) VALUES ($1,$2,random_string(9000)) `, srcTableName), i, testValue) - e2e.EnvNoError(s.t, env, err) - } - s.t.Log("Inserted 10 rows into the source table") - - _, err = rowsTx.Exec(context.Background(), - fmt.Sprintf(`UPDATE %s SET c1=c1+1 WHERE MOD(c2,2)=$1`, srcTableName), 1) - e2e.EnvNoError(s.t, env, err) - _, err = rowsTx.Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=$1`, srcTableName), 0) e2e.EnvNoError(s.t, env, err) + } + s.t.Log("Inserted 10 rows into the source table") - err = rowsTx.Commit(context.Background()) - e2e.EnvNoError(s.t, env, err) + _, err = rowsTx.Exec(context.Background(), + fmt.Sprintf(`UPDATE %s SET c1=c1+1 WHERE MOD(c2,2)=$1`, srcTableName), 1) + e2e.EnvNoError(s.t, env, err) + _, err = rowsTx.Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=$1`, srcTableName), 0) + e2e.EnvNoError(s.t, env, err) - e2e.EnvWaitForEqualTables(env, s, "normalizing tx", "test_cpkey_toast1", "id,c1,c2,t,t2") - env.CancelWorkflow() - }() + err = rowsTx.Commit(context.Background()) + e2e.EnvNoError(s.t, env, err) + + e2e.EnvWaitForEqualTables(env, s, "normalizing tx", "test_cpkey_toast1", "id,c1,c2,t,t2") + env.Cancel() - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuiteSF) Test_Composite_PKey_Toast_2_SF() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) tableName := "test_cpkey_toast2" srcTableName := s.attachSchemaSuffix(tableName) @@ -1042,38 +1018,36 @@ func (s PeerFlowE2ETestSuiteSF) Test_Composite_PKey_Toast_2_SF() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and then insert, update and delete rows in the table. - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - // insert 10 rows into the source table - for i := range 10 { - testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + // insert 10 rows into the source table + for i := range 10 { + testValue := fmt.Sprintf("test_value_%d", i) + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c2,t,t2) VALUES ($1,$2,random_string(9000)) `, srcTableName), i, testValue) - e2e.EnvNoError(s.t, env, err) - } - s.t.Log("Inserted 10 rows into the source table") - - e2e.EnvWaitForEqualTables(env, s, "normalize table", tableName, "id,c2,t,t2") - _, err = s.Conn().Exec(context.Background(), - fmt.Sprintf(`UPDATE %s SET c1=c1+1 WHERE MOD(c2,2)=$1`, srcTableName), 1) - e2e.EnvNoError(s.t, env, err) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=$1`, srcTableName), 0) e2e.EnvNoError(s.t, env, err) - e2e.EnvWaitForEqualTables(env, s, "normalize update/delete", tableName, "id,c2,t,t2") + } + s.t.Log("Inserted 10 rows into the source table") + + e2e.EnvWaitForEqualTables(env, s, "normalize table", tableName, "id,c2,t,t2") + _, err = s.Conn().Exec(context.Background(), + fmt.Sprintf(`UPDATE %s SET c1=c1+1 WHERE MOD(c2,2)=$1`, srcTableName), 1) + e2e.EnvNoError(s.t, env, err) + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=$1`, srcTableName), 0) + e2e.EnvNoError(s.t, env, err) + e2e.EnvWaitForEqualTables(env, s, "normalize update/delete", tableName, "id,c2,t,t2") - env.CancelWorkflow() - }() + env.Cancel() - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuiteSF) Test_Column_Exclusion() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) tableName := "test_exclude_sf" srcTableName := s.attachSchemaSuffix(tableName) @@ -1111,33 +1085,31 @@ func (s PeerFlowE2ETestSuiteSF) Test_Column_Exclusion() { MaxBatchSize: 100, } - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and then insert, update and delete rows in the table. - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, config, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - // insert 10 rows into the source table - for i := range 10 { - testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + // insert 10 rows into the source table + for i := range 10 { + testValue := fmt.Sprintf("test_value_%d", i) + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c2,t,t2) VALUES ($1,$2,random_string(100)) `, srcTableName), i, testValue) - e2e.EnvNoError(s.t, env, err) - } - s.t.Log("Inserted 10 rows into the source table") - - e2e.EnvWaitForEqualTables(env, s, "normalize table", tableName, "id,c1,t,t2") - _, err = s.Conn().Exec(context.Background(), - fmt.Sprintf(`UPDATE %s SET c1=c1+1 WHERE MOD(c2,2)=1`, srcTableName)) e2e.EnvNoError(s.t, env, err) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=0`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - e2e.EnvWaitForEqualTables(env, s, "normalize update/delete", tableName, "id,c1,t,t2") + } + s.t.Log("Inserted 10 rows into the source table") + + e2e.EnvWaitForEqualTables(env, s, "normalize table", tableName, "id,c1,t,t2") + _, err = s.Conn().Exec(context.Background(), + fmt.Sprintf(`UPDATE %s SET c1=c1+1 WHERE MOD(c2,2)=1`, srcTableName)) + e2e.EnvNoError(s.t, env, err) + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=0`, srcTableName)) + e2e.EnvNoError(s.t, env, err) + e2e.EnvWaitForEqualTables(env, s, "normalize update/delete", tableName, "id,c1,t,t2") - env.CancelWorkflow() - }() + env.Cancel() - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, config, nil) e2e.RequireEnvCanceled(s.t, env) sfRows, err := s.GetRows(tableName, "*") @@ -1150,7 +1122,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Column_Exclusion() { } func (s PeerFlowE2ETestSuiteSF) Test_Soft_Delete_Basic() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) tableName := "test_softdel_src" dstName := "test_softdel" @@ -1188,35 +1160,33 @@ func (s PeerFlowE2ETestSuiteSF) Test_Soft_Delete_Basic() { MaxBatchSize: 100, } - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and then insert, update and delete rows in the table. - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, config, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c1,c2,t) VALUES (1,2,random_string(9000))`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - e2e.EnvWaitForEqualTablesWithNames(env, s, "normalize row", tableName, dstName, "id,c1,c2,t") - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + e2e.EnvWaitForEqualTablesWithNames(env, s, "normalize row", tableName, dstName, "id,c1,c2,t") + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` UPDATE %s SET c1=c1+4 WHERE id=1`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - e2e.EnvWaitForEqualTablesWithNames(env, s, "normalize update", tableName, dstName, "id,c1,c2,t") - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + e2e.EnvWaitForEqualTablesWithNames(env, s, "normalize update", tableName, dstName, "id,c1,c2,t") + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` DELETE FROM %s WHERE id=1`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - e2e.EnvWaitForEqualTablesWithNames( - env, - s, - "normalize delete", - tableName, - dstName+" WHERE NOT _PEERDB_IS_DELETED", - "id,c1,c2,t", - ) + e2e.EnvNoError(s.t, env, err) + e2e.EnvWaitForEqualTablesWithNames( + env, + s, + "normalize delete", + tableName, + dstName+" WHERE NOT _PEERDB_IS_DELETED", + "id,c1,c2,t", + ) - env.CancelWorkflow() - }() + env.Cancel() - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, config, nil) e2e.RequireEnvCanceled(s.t, env) newerSyncedAtQuery := fmt.Sprintf(` @@ -1227,7 +1197,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Soft_Delete_Basic() { } func (s PeerFlowE2ETestSuiteSF) Test_Soft_Delete_IUD_Same_Batch() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) cmpTableName := s.attachSchemaSuffix("test_softdel_iud") srcTableName := cmpTableName + "_src" @@ -1264,48 +1234,46 @@ func (s PeerFlowE2ETestSuiteSF) Test_Soft_Delete_IUD_Same_Batch() { MaxBatchSize: 100, } - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and then insert, update and delete rows in the table. - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, config, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - insertTx, err := s.Conn().Begin(context.Background()) - e2e.EnvNoError(s.t, env, err) + insertTx, err := s.Conn().Begin(context.Background()) + e2e.EnvNoError(s.t, env, err) - _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` + _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c1,c2,t) VALUES (1,2,random_string(9000))`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` UPDATE %s SET c1=c1+4 WHERE id=1`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - // since we delete stuff, create another table to compare with - _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + // since we delete stuff, create another table to compare with + _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE %s AS SELECT * FROM %s`, cmpTableName, srcTableName)) - e2e.EnvNoError(s.t, env, err) - _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` DELETE FROM %s WHERE id=1`, srcTableName)) - e2e.EnvNoError(s.t, env, err) + e2e.EnvNoError(s.t, env, err) - e2e.EnvNoError(s.t, env, insertTx.Commit(context.Background())) + e2e.EnvNoError(s.t, env, insertTx.Commit(context.Background())) - e2e.EnvWaitForEqualTables(env, s, "normalizing tx", "test_softdel_iud", "id,c1,c2,t") - e2e.EnvWaitFor(s.t, env, 3*time.Minute, "checking soft delete", func() bool { - newerSyncedAtQuery := fmt.Sprintf(` + e2e.EnvWaitForEqualTables(env, s, "normalizing tx", "test_softdel_iud", "id,c1,c2,t") + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "checking soft delete", func() bool { + newerSyncedAtQuery := fmt.Sprintf(` SELECT COUNT(*) FROM %s WHERE _PEERDB_IS_DELETED`, dstTableName) - numNewRows, err := s.sfHelper.RunIntQuery(newerSyncedAtQuery) - e2e.EnvNoError(s.t, env, err) - return numNewRows == 1 - }) + numNewRows, err := s.sfHelper.RunIntQuery(newerSyncedAtQuery) + e2e.EnvNoError(s.t, env, err) + return numNewRows == 1 + }) - env.CancelWorkflow() - }() + env.Cancel() - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, config, nil) e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuiteSF) Test_Soft_Delete_UD_Same_Batch() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) tableName := "test_softdel_ud_src" dstName := "test_softdel_ud" @@ -1343,54 +1311,52 @@ func (s PeerFlowE2ETestSuiteSF) Test_Soft_Delete_UD_Same_Batch() { MaxBatchSize: 100, } - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and then insert, update and delete rows in the table. - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, config, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c1,c2,t) VALUES (1,2,random_string(9000))`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - e2e.EnvWaitForEqualTablesWithNames(env, s, "normalize insert", tableName, dstName, "id,c1,c2,t") + e2e.EnvNoError(s.t, env, err) + e2e.EnvWaitForEqualTablesWithNames(env, s, "normalize insert", tableName, dstName, "id,c1,c2,t") - insertTx, err := s.Conn().Begin(context.Background()) - e2e.EnvNoError(s.t, env, err) - _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` + insertTx, err := s.Conn().Begin(context.Background()) + e2e.EnvNoError(s.t, env, err) + _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` UPDATE %s SET t=random_string(10000) WHERE id=1`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` UPDATE %s SET c1=c1+4 WHERE id=1`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` DELETE FROM %s WHERE id=1`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - - e2e.EnvNoError(s.t, env, insertTx.Commit(context.Background())) - e2e.EnvWaitForEqualTablesWithNames( - env, - s, - "normalize transaction", - tableName, - dstName+" WHERE NOT _PEERDB_IS_DELETED", - "id,c1,c2,t", - ) - e2e.EnvWaitFor(s.t, env, 3*time.Minute, "checking soft delete", func() bool { - newerSyncedAtQuery := fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + + e2e.EnvNoError(s.t, env, insertTx.Commit(context.Background())) + e2e.EnvWaitForEqualTablesWithNames( + env, + s, + "normalize transaction", + tableName, + dstName+" WHERE NOT _PEERDB_IS_DELETED", + "id,c1,c2,t", + ) + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "checking soft delete", func() bool { + newerSyncedAtQuery := fmt.Sprintf(` SELECT COUNT(*) FROM %s WHERE _PEERDB_IS_DELETED`, dstTableName) - numNewRows, err := s.sfHelper.RunIntQuery(newerSyncedAtQuery) - e2e.EnvNoError(s.t, env, err) - return numNewRows == 1 - }) + numNewRows, err := s.sfHelper.RunIntQuery(newerSyncedAtQuery) + e2e.EnvNoError(s.t, env, err) + return numNewRows == 1 + }) - env.CancelWorkflow() - }() + env.Cancel() - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, config, nil) e2e.RequireEnvCanceled(s.t, env) } func (s PeerFlowE2ETestSuiteSF) Test_Soft_Delete_Insert_After_Delete() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) tableName := "test_softdel_iad" srcTableName := s.attachSchemaSuffix(tableName) @@ -1427,36 +1393,34 @@ func (s PeerFlowE2ETestSuiteSF) Test_Soft_Delete_Insert_After_Delete() { MaxBatchSize: 100, } - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and then insert and delete rows in the table. - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, config, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(c1,c2,t) VALUES (1,2,random_string(9000))`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - e2e.EnvWaitForEqualTables(env, s, "normalize row", tableName, "id,c1,c2,t") - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + e2e.EnvNoError(s.t, env, err) + e2e.EnvWaitForEqualTables(env, s, "normalize row", tableName, "id,c1,c2,t") + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` DELETE FROM %s WHERE id=1`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - e2e.EnvWaitForEqualTablesWithNames( - env, - s, - "normalize delete", - tableName, - tableName+" WHERE NOT _PEERDB_IS_DELETED", - "id,c1,c2,t", - ) + e2e.EnvNoError(s.t, env, err) + e2e.EnvWaitForEqualTablesWithNames( + env, + s, + "normalize delete", + tableName, + tableName+" WHERE NOT _PEERDB_IS_DELETED", + "id,c1,c2,t", + ) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s(id,c1,c2,t) VALUES (1,3,4,random_string(10000))`, srcTableName)) - e2e.EnvNoError(s.t, env, err) - e2e.EnvWaitForEqualTables(env, s, "normalize reinsert", tableName, "id,c1,c2,t") + e2e.EnvNoError(s.t, env, err) + e2e.EnvWaitForEqualTables(env, s, "normalize reinsert", tableName, "id,c1,c2,t") - env.CancelWorkflow() - }() + env.Cancel() - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, config, nil) e2e.RequireEnvCanceled(s.t, env) newerSyncedAtQuery := fmt.Sprintf(` @@ -1467,7 +1431,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Soft_Delete_Insert_After_Delete() { } func (s PeerFlowE2ETestSuiteSF) Test_Supported_Mixed_Case_Table_SF() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) srcTableName := s.attachSchemaSuffix("testMixedCase") dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "testMixedCase") @@ -1491,32 +1455,30 @@ func (s PeerFlowE2ETestSuiteSF) Test_Supported_Mixed_Case_Table_SF() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() flowConnConfig.MaxBatchSize = 100 - // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // wait for PeerFlowStatusQuery to finish setup // and then insert 20 rows into the source table - go func() { - e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - // insert 20 rows into the source table - for i := range 20 { - testKey := fmt.Sprintf("test_key_%d", i) - testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + // insert 20 rows into the source table + for i := range 20 { + testKey := fmt.Sprintf("test_key_%d", i) + testValue := fmt.Sprintf("test_value_%d", i) + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` INSERT INTO e2e_test_%s."%s"("highGold","eVe") VALUES ($1, $2) `, s.pgSuffix, "testMixedCase"), testKey, testValue) - e2e.EnvNoError(s.t, env, err) - } - s.t.Log("Inserted 20 rows into the source table") - e2e.EnvWaitForEqualTablesWithNames( - env, - s, - "normalize mixed case", - "testMixedCase", - "\"testMixedCase\"", - "id,\"pulseArmor\",\"highGold\",\"eVe\"", - ) + e2e.EnvNoError(s.t, env, err) + } + s.t.Log("Inserted 20 rows into the source table") + e2e.EnvWaitForEqualTablesWithNames( + env, + s, + "normalize mixed case", + "testMixedCase", + "\"testMixedCase\"", + "id,\"pulseArmor\",\"highGold\",\"eVe\"", + ) - env.CancelWorkflow() - }() + env.Cancel() - env.ExecuteWorkflow(peerflow.CDCFlowWorkflow, flowConnConfig, nil) e2e.RequireEnvCanceled(s.t, env) } diff --git a/flow/e2e/snowflake/qrep_flow_sf_test.go b/flow/e2e/snowflake/qrep_flow_sf_test.go index a662513e6a..7a417238a6 100644 --- a/flow/e2e/snowflake/qrep_flow_sf_test.go +++ b/flow/e2e/snowflake/qrep_flow_sf_test.go @@ -2,6 +2,7 @@ package e2e_snowflake import ( "fmt" + "time" "github.com/google/uuid" "github.com/stretchr/testify/require" @@ -46,7 +47,7 @@ func (s PeerFlowE2ETestSuiteSF) compareTableContentsWithDiffSelectorsSF(tableNam } func (s PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) numRows := 10 @@ -71,13 +72,9 @@ func (s PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF() { qrepConfig.SetupWatermarkTableOnDestination = true require.NoError(s.t, err) - e2e.RunQrepFlowWorkflow(env, qrepConfig) - - // Verify workflow completes without error - require.True(s.t, env.IsWorkflowCompleted()) - - err = env.GetWorkflowError() - require.NoError(s.t, err) + env := e2e.RunQrepFlowWorkflow(tc, qrepConfig) + e2e.EnvWaitForFinished(s.t, env, 3*time.Minute) + require.NoError(s.t, env.Error()) sel := e2e.GetOwnersSelectorStringsSF() s.compareTableContentsWithDiffSelectorsSF(tblName, sel[0], sel[1]) @@ -87,7 +84,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF() { } func (s PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF_Upsert_Simple() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) numRows := 10 @@ -116,20 +113,16 @@ func (s PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF_Upsert_Simple() qrepConfig.SetupWatermarkTableOnDestination = true require.NoError(s.t, err) - e2e.RunQrepFlowWorkflow(env, qrepConfig) - - // Verify workflow completes without error - require.True(s.t, env.IsWorkflowCompleted()) - - err = env.GetWorkflowError() - require.NoError(s.t, err) + env := e2e.RunQrepFlowWorkflow(tc, qrepConfig) + e2e.EnvWaitForFinished(s.t, env, 3*time.Minute) + require.NoError(s.t, env.Error()) sel := e2e.GetOwnersSelectorStringsSF() s.compareTableContentsWithDiffSelectorsSF(tblName, sel[0], sel[1]) } func (s PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF_S3() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) numRows := 10 @@ -155,20 +148,16 @@ func (s PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF_S3() { qrepConfig.StagingPath = fmt.Sprintf("s3://peerdb-test-bucket/avro/%s", uuid.New()) qrepConfig.SetupWatermarkTableOnDestination = true - e2e.RunQrepFlowWorkflow(env, qrepConfig) - - // Verify workflow completes without error - require.True(s.t, env.IsWorkflowCompleted()) - - err = env.GetWorkflowError() - require.NoError(s.t, err) + env := e2e.RunQrepFlowWorkflow(tc, qrepConfig) + e2e.EnvWaitForFinished(s.t, env, 3*time.Minute) + require.NoError(s.t, env.Error()) sel := e2e.GetOwnersSelectorStringsSF() s.compareTableContentsWithDiffSelectorsSF(tblName, sel[0], sel[1]) } func (s PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF_Upsert_XMIN() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) numRows := 10 @@ -198,20 +187,16 @@ func (s PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF_Upsert_XMIN() { qrepConfig.SetupWatermarkTableOnDestination = true require.NoError(s.t, err) - e2e.RunXminFlowWorkflow(env, qrepConfig) - - // Verify workflow completes without error - require.True(s.t, env.IsWorkflowCompleted()) - - err = env.GetWorkflowError() - require.NoError(s.t, err) + env := e2e.RunXminFlowWorkflow(tc, qrepConfig) + e2e.EnvWaitForFinished(s.t, env, 3*time.Minute) + require.NoError(s.t, env.Error()) sel := e2e.GetOwnersSelectorStringsSF() s.compareTableContentsWithDiffSelectorsSF(tblName, sel[0], sel[1]) } func (s PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF_S3_Integration() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) numRows := 10 @@ -240,20 +225,16 @@ func (s PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF_S3_Integration() qrepConfig.StagingPath = fmt.Sprintf("s3://peerdb-test-bucket/avro/%s", uuid.New()) qrepConfig.SetupWatermarkTableOnDestination = true - e2e.RunQrepFlowWorkflow(env, qrepConfig) - - // Verify workflow completes without error - require.True(s.t, env.IsWorkflowCompleted()) - - err = env.GetWorkflowError() - require.NoError(s.t, err) + env := e2e.RunQrepFlowWorkflow(tc, qrepConfig) + e2e.EnvWaitForFinished(s.t, env, 3*time.Minute) + require.NoError(s.t, env.Error()) sel := e2e.GetOwnersSelectorStringsSF() s.compareTableContentsWithDiffSelectorsSF(tblName, sel[0], sel[1]) } func (s PeerFlowE2ETestSuiteSF) Test_PeerDB_Columns_QRep_SF() { - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) numRows := 10 @@ -282,13 +263,9 @@ func (s PeerFlowE2ETestSuiteSF) Test_PeerDB_Columns_QRep_SF() { qrepConfig.SetupWatermarkTableOnDestination = true require.NoError(s.t, err) - e2e.RunQrepFlowWorkflow(env, qrepConfig) - - // Verify workflow completes without error - require.True(s.t, env.IsWorkflowCompleted()) - - err = env.GetWorkflowError() - require.NoError(s.t, err) + env := e2e.RunQrepFlowWorkflow(tc, qrepConfig) + e2e.EnvWaitForFinished(s.t, env, 3*time.Minute) + require.NoError(s.t, env.Error()) err = s.sfHelper.checkSyncedAt(fmt.Sprintf(`SELECT "_PEERDB_SYNCED_AT" FROM %s.%s`, s.sfHelper.testSchemaName, tblName)) diff --git a/flow/e2e/sqlserver/qrep_flow_sqlserver_test.go b/flow/e2e/sqlserver/qrep_flow_sqlserver_test.go index 2e7572cdb0..448d0f32b7 100644 --- a/flow/e2e/sqlserver/qrep_flow_sqlserver_test.go +++ b/flow/e2e/sqlserver/qrep_flow_sqlserver_test.go @@ -147,7 +147,7 @@ func (s PeerFlowE2ETestSuiteSQLServer) Test_Complete_QRep_Flow_SqlServer_Append( s.t.Skip("Skipping SQL Server test") } - env := e2e.NewTemporalTestWorkflowEnvironment(s.t) + tc := e2e.NewTemporalClient(s.t) numRows := 10 tblName := "test_qrep_flow_avro_ss_append" @@ -178,18 +178,14 @@ func (s PeerFlowE2ETestSuiteSQLServer) Test_Complete_QRep_Flow_SqlServer_Append( WaitBetweenBatchesSeconds: 5, } - e2e.RunQrepFlowWorkflow(env, qrepConfig) - - // Verify workflow completes without error - require.True(s.t, env.IsWorkflowCompleted()) - - err := env.GetWorkflowError() - require.NoError(s.t, err) + env := e2e.RunQrepFlowWorkflow(tc, qrepConfig) + e2e.EnvWaitForFinished(s.t, env, 3*time.Minute) + require.NoError(s.t, env.Error()) // Verify that the destination table has the same number of rows as the source table var numRowsInDest pgtype.Int8 countQuery := "SELECT COUNT(*) FROM " + dstTableName - err = s.Conn().QueryRow(context.Background(), countQuery).Scan(&numRowsInDest) + err := s.Conn().QueryRow(context.Background(), countQuery).Scan(&numRowsInDest) require.NoError(s.t, err) require.Equal(s.t, numRows, int(numRowsInDest.Int64)) diff --git a/flow/e2e/test_utils.go b/flow/e2e/test_utils.go index 6239f7d148..92d04d1f3d 100644 --- a/flow/e2e/test_utils.go +++ b/flow/e2e/test_utils.go @@ -6,7 +6,6 @@ import ( "errors" "fmt" "log/slog" - "runtime" "slices" "strings" "testing" @@ -16,19 +15,15 @@ import ( "github.com/jackc/pgerrcode" "github.com/jackc/pgx/v5" "github.com/jackc/pgx/v5/pgconn" - "github.com/jackc/pgx/v5/pgxpool" "github.com/stretchr/testify/require" + "go.temporal.io/api/enums/v1" + "go.temporal.io/sdk/client" + "go.temporal.io/sdk/converter" "go.temporal.io/sdk/temporal" - "go.temporal.io/sdk/testsuite" - "go.temporal.io/sdk/worker" - "github.com/PeerDB-io/peer-flow/activities" - "github.com/PeerDB-io/peer-flow/alerting" - "github.com/PeerDB-io/peer-flow/connectors" connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" connsnowflake "github.com/PeerDB-io/peer-flow/connectors/snowflake" "github.com/PeerDB-io/peer-flow/connectors/utils" - catalog "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" "github.com/PeerDB-io/peer-flow/e2eshared" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/logger" @@ -49,61 +44,25 @@ type RowSource interface { GetRows(table, cols string) (*model.QRecordBatch, error) } -func RegisterWorkflowsAndActivities(t *testing.T, env *testsuite.TestWorkflowEnvironment) { - t.Helper() - - conn, err := pgxpool.New(context.Background(), catalog.GetCatalogConnectionStringFromEnv()) - if err != nil { - t.Fatalf("unable to create catalog connection pool: %v", err) - } - - // set a 5 minute timeout for the workflow to execute a few runs. - env.SetTestTimeout(5 * time.Minute) - - peerflow.RegisterFlowWorkerWorkflows(env) - env.RegisterWorkflow(peerflow.SnapshotFlowWorkflow) - - alerter, err := alerting.NewAlerter(conn) - if err != nil { - t.Fatalf("unable to create alerter: %v", err) - } - - env.RegisterActivity(&activities.FlowableActivity{ - CatalogPool: conn, - Alerter: alerter, - CdcCache: make(map[string]connectors.CDCPullConnector), - }) - env.RegisterActivity(&activities.SnapshotActivity{ - SnapshotConnections: make(map[string]activities.SlotSnapshotSignal), - Alerter: alerter, - }) -} - -func EnvSignalWorkflow[T any](env *testsuite.TestWorkflowEnvironment, signal model.TypedSignal[T], value T) { - env.SignalWorkflow(signal.Name, value) -} - // Helper function to assert errors in go routines running concurrent to workflows // This achieves two goals: // 1. cancel workflow to avoid waiting on goroutine which has failed // 2. get around t.FailNow being incorrect when called from non initial goroutine -func EnvNoError(t *testing.T, env *testsuite.TestWorkflowEnvironment, err error) { +func EnvNoError(t *testing.T, env WorkflowRun, err error) { t.Helper() if err != nil { - t.Error("UNEXPECTED ERROR", err.Error()) - env.CancelWorkflow() - runtime.Goexit() + env.Cancel() + t.Fatal("UNEXPECTED ERROR", err.Error()) } } -func EnvTrue(t *testing.T, env *testsuite.TestWorkflowEnvironment, val bool) { +func EnvTrue(t *testing.T, env WorkflowRun, val bool) { t.Helper() if !val { - t.Error("UNEXPECTED FALSE") - env.CancelWorkflow() - runtime.Goexit() + env.Cancel() + t.Fatal("UNEXPECTED FALSE") } } @@ -130,7 +89,7 @@ func RequireEqualTables(suite RowSource, table string, cols string) { require.True(t, e2eshared.CheckEqualRecordBatches(t, pgRows, rows)) } -func EnvEqualTables(env *testsuite.TestWorkflowEnvironment, suite RowSource, table string, cols string) { +func EnvEqualTables(env WorkflowRun, suite RowSource, table string, cols string) { t := suite.T() t.Helper() @@ -144,7 +103,7 @@ func EnvEqualTables(env *testsuite.TestWorkflowEnvironment, suite RowSource, tab } func EnvWaitForEqualTables( - env *testsuite.TestWorkflowEnvironment, + env WorkflowRun, suite RowSource, reason string, table string, @@ -155,7 +114,7 @@ func EnvWaitForEqualTables( } func EnvWaitForEqualTablesWithNames( - env *testsuite.TestWorkflowEnvironment, + env WorkflowRun, suite RowSource, reason string, srcTable string, @@ -182,9 +141,10 @@ func EnvWaitForEqualTablesWithNames( }) } -func RequireEnvCanceled(t *testing.T, env *testsuite.TestWorkflowEnvironment) { +func RequireEnvCanceled(t *testing.T, env WorkflowRun) { t.Helper() - err := env.GetWorkflowError() + EnvWaitForFinished(t, env, time.Minute) + err := env.Error() var panicErr *temporal.PanicError var canceledErr *temporal.CanceledError if err == nil { @@ -196,31 +156,25 @@ func RequireEnvCanceled(t *testing.T, env *testsuite.TestWorkflowEnvironment) { } } -func SetupCDCFlowStatusQuery(t *testing.T, env *testsuite.TestWorkflowEnvironment, - connectionGen FlowConnectionGenerationConfig, -) { +func SetupCDCFlowStatusQuery(t *testing.T, env WorkflowRun, connectionGen FlowConnectionGenerationConfig) { t.Helper() // errors expected while PeerFlowStatusQuery is setup counter := 0 for { time.Sleep(time.Second) counter++ - response, err := env.QueryWorkflow( - shared.CDCFlowStateQuery, - connectionGen.FlowJobName, - ) + response, err := env.Query(shared.CDCFlowStateQuery, connectionGen.FlowJobName) if err == nil { var state peerflow.CDCFlowWorkflowState err = response.Get(&state) if err != nil { - t.Log(err.Error()) + t.Fatal(err) } else if state.CurrentFlowStatus == protos.FlowStatus_STATUS_RUNNING { return } } else if counter > 15 { - t.Error("UNEXPECTED SETUP CDC TIMEOUT", err.Error()) - env.CancelWorkflow() - runtime.Goexit() + env.Cancel() + t.Fatal("UNEXPECTED SETUP CDC TIMEOUT", err.Error()) } else if counter > 5 { // log the error for informational purposes t.Log(err.Error()) @@ -453,15 +407,15 @@ func CreateQRepWorkflowConfig( return qrepConfig, nil } -func RunQrepFlowWorkflow(env *testsuite.TestWorkflowEnvironment, config *protos.QRepConfig) { +func RunQrepFlowWorkflow(tc client.Client, config *protos.QRepConfig) WorkflowRun { state := peerflow.NewQRepFlowState() - env.ExecuteWorkflow(peerflow.QRepFlowWorkflow, config, state) + return ExecutePeerflow(tc, peerflow.QRepFlowWorkflow, config, state) } -func RunXminFlowWorkflow(env *testsuite.TestWorkflowEnvironment, config *protos.QRepConfig) { +func RunXminFlowWorkflow(tc client.Client, config *protos.QRepConfig) WorkflowRun { state := peerflow.NewQRepFlowState() state.LastPartition.PartitionId = uuid.New().String() - env.ExecuteWorkflow(peerflow.XminFlowWorkflow, config, state) + return ExecutePeerflow(tc, peerflow.XminFlowWorkflow, config, state) } func GetOwnersSchema() *model.QRecordSchema { @@ -550,9 +504,8 @@ func (tw *testWriter) Write(p []byte) (int, error) { return len(p), nil } -func NewTemporalTestWorkflowEnvironment(t *testing.T) *testsuite.TestWorkflowEnvironment { +func NewTemporalClient(t *testing.T) client.Client { t.Helper() - testSuite := &testsuite.WorkflowTestSuite{} logger := slog.New(logger.NewHandler( slog.NewJSONHandler( @@ -560,36 +513,76 @@ func NewTemporalTestWorkflowEnvironment(t *testing.T) *testsuite.TestWorkflowEnv &slog.HandlerOptions{Level: slog.LevelWarn}, ), )) - testSuite.SetLogger(&TStructuredLogger{logger: logger}) - env := testSuite.NewTestWorkflowEnvironment() - env.SetWorkerOptions(worker.Options{EnableSessionWorker: true}) - RegisterWorkflowsAndActivities(t, env) - return env + tc, err := client.Dial(client.Options{ + HostPort: "localhost:7233", + Logger: logger, + }) + if err != nil { + t.Fatalf("Failed to connect temporal client: %v", err) + } + return tc } -type TStructuredLogger struct { - logger *slog.Logger +type WorkflowRun struct { + client.WorkflowRun + c client.Client } -func (l *TStructuredLogger) keyvalsToFields(keyvals []interface{}) slog.Attr { - return slog.Group("test-log", keyvals...) +func ExecutePeerflow(tc client.Client, wf interface{}, args ...interface{}) WorkflowRun { + return ExecuteWorkflow(tc, shared.PeerFlowTaskQueue, wf, args...) } -func (l *TStructuredLogger) Debug(msg string, keyvals ...interface{}) { - l.logger.With(l.keyvalsToFields(keyvals)).Debug(msg) +func ExecuteWorkflow(tc client.Client, taskQueueID shared.TaskQueueID, wf interface{}, args ...interface{}) WorkflowRun { + taskQueue := shared.GetPeerFlowTaskQueueName(taskQueueID) + + wr, err := tc.ExecuteWorkflow( + context.Background(), + client.StartWorkflowOptions{ + TaskQueue: taskQueue, + WorkflowExecutionTimeout: 5 * time.Minute, + }, + wf, + args..., + ) + if err != nil { + panic(err) + } + return WorkflowRun{ + WorkflowRun: wr, + c: tc, + } +} + +func (env WorkflowRun) Finished() bool { + desc, err := env.c.DescribeWorkflowExecution(context.Background(), env.GetID(), "") + if err != nil { + return false + } + return desc.GetWorkflowExecutionInfo().GetStatus() != enums.WORKFLOW_EXECUTION_STATUS_RUNNING +} + +func (env WorkflowRun) Error() error { + if env.Finished() { + return env.Get(context.Background(), nil) + } else { + return nil + } } -func (l *TStructuredLogger) Info(msg string, keyvals ...interface{}) { - l.logger.With(l.keyvalsToFields(keyvals)).Info(msg) +func (env WorkflowRun) Cancel() { + _ = env.c.CancelWorkflow(context.Background(), env.GetID(), "") } -func (l *TStructuredLogger) Warn(msg string, keyvals ...interface{}) { - l.logger.With(l.keyvalsToFields(keyvals)).Warn(msg) +func (env WorkflowRun) Query(queryType string, args ...interface{}) (converter.EncodedValue, error) { + return env.c.QueryWorkflow(context.Background(), env.GetID(), "", queryType, args...) } -func (l *TStructuredLogger) Error(msg string, keyvals ...interface{}) { - l.logger.With(l.keyvalsToFields(keyvals)).Error(msg) +func SignalWorkflow[T any](env WorkflowRun, signal model.TypedSignal[T], value T) { + err := env.c.SignalWorkflow(context.Background(), env.GetID(), "", signal.Name, value) + if err != nil { + panic(err) + } } func CompareTableSchemas(x *protos.TableSchema, y *protos.TableSchema) bool { @@ -624,22 +617,43 @@ func RequireEqualRecordBatches(t *testing.T, q *model.QRecordBatch, other *model require.True(t, e2eshared.CheckEqualRecordBatches(t, q, other)) } -func EnvEqualRecordBatches(t *testing.T, env *testsuite.TestWorkflowEnvironment, q *model.QRecordBatch, other *model.QRecordBatch) { +func EnvEqualRecordBatches(t *testing.T, env WorkflowRun, q *model.QRecordBatch, other *model.QRecordBatch) { t.Helper() EnvTrue(t, env, e2eshared.CheckEqualRecordBatches(t, q, other)) } -func EnvWaitFor(t *testing.T, env *testsuite.TestWorkflowEnvironment, timeout time.Duration, reason string, f func() bool) { +func EnvWaitFor(t *testing.T, env WorkflowRun, timeout time.Duration, reason string, f func() bool) { t.Helper() t.Log("WaitFor", reason, time.Now()) deadline := time.Now().Add(timeout) for !f() { if time.Now().After(deadline) { - t.Error("UNEXPECTED TIMEOUT", reason, time.Now()) - env.CancelWorkflow() - runtime.Goexit() + env.Cancel() + t.Fatal("UNEXPECTED TIMEOUT", reason, time.Now()) } time.Sleep(time.Second) } } + +func EnvWaitForFinished(t *testing.T, env WorkflowRun, timeout time.Duration) { + t.Helper() + + EnvWaitFor(t, env, timeout, "finish", func() bool { + t.Helper() + + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + defer cancel() + desc, err := env.c.DescribeWorkflowExecution(ctx, env.GetID(), "") + if err != nil { + t.Log("Not finished", err) + return false + } + status := desc.GetWorkflowExecutionInfo().GetStatus() + if status != enums.WORKFLOW_EXECUTION_STATUS_RUNNING { + t.Log("Finished Status", status) + return true + } + return false + }) +} diff --git a/flow/cmd/main.go b/flow/main.go similarity index 58% rename from flow/cmd/main.go rename to flow/main.go index f7fa4734fb..34c66f57a0 100644 --- a/flow/cmd/main.go +++ b/flow/main.go @@ -6,11 +6,14 @@ import ( "log/slog" "os" "os/signal" + "runtime" "syscall" "github.com/urfave/cli/v3" + "go.temporal.io/sdk/worker" _ "go.uber.org/automaxprocs" + "github.com/PeerDB-io/peer-flow/cmd" "github.com/PeerDB-io/peer-flow/logger" ) @@ -64,16 +67,21 @@ func main() { Commands: []*cli.Command{ { Name: "worker", - Action: func(ctx context.Context, cmd *cli.Command) error { - temporalHostPort := cmd.String("temporal-host-port") - return WorkerMain(&WorkerOptions{ + Action: func(ctx context.Context, clicmd *cli.Command) error { + temporalHostPort := clicmd.String("temporal-host-port") + c, w, err := cmd.WorkerMain(&cmd.WorkerOptions{ TemporalHostPort: temporalHostPort, - EnableProfiling: cmd.Bool("enable-profiling"), - PyroscopeServer: cmd.String("pyroscope-server-address"), - TemporalNamespace: cmd.String("temporal-namespace"), - TemporalCert: cmd.String("temporal-cert"), - TemporalKey: cmd.String("temporal-key"), + EnableProfiling: clicmd.Bool("enable-profiling"), + PyroscopeServer: clicmd.String("pyroscope-server-address"), + TemporalNamespace: clicmd.String("temporal-namespace"), + TemporalCert: clicmd.String("temporal-cert"), + TemporalKey: clicmd.String("temporal-key"), }) + if err != nil { + return err + } + defer c.Close() + return w.Run(worker.InterruptCh()) }, Flags: []cli.Flag{ temporalHostPortFlag, @@ -86,14 +94,19 @@ func main() { }, { Name: "snapshot-worker", - Action: func(ctx context.Context, cmd *cli.Command) error { - temporalHostPort := cmd.String("temporal-host-port") - return SnapshotWorkerMain(&SnapshotWorkerOptions{ + Action: func(ctx context.Context, clicmd *cli.Command) error { + temporalHostPort := clicmd.String("temporal-host-port") + c, w, err := cmd.SnapshotWorkerMain(&cmd.SnapshotWorkerOptions{ TemporalHostPort: temporalHostPort, - TemporalNamespace: cmd.String("temporal-namespace"), - TemporalCert: cmd.String("temporal-cert"), - TemporalKey: cmd.String("temporal-key"), + TemporalNamespace: clicmd.String("temporal-namespace"), + TemporalCert: clicmd.String("temporal-cert"), + TemporalKey: clicmd.String("temporal-key"), }) + if err != nil { + return err + } + defer c.Close() + return w.Run(worker.InterruptCh()) }, Flags: []cli.Flag{ temporalHostPortFlag, @@ -120,22 +133,33 @@ func main() { &temporalCertFlag, &temporalKeyFlag, }, - Action: func(ctx context.Context, cmd *cli.Command) error { - temporalHostPort := cmd.String("temporal-host-port") + Action: func(ctx context.Context, clicmd *cli.Command) error { + temporalHostPort := clicmd.String("temporal-host-port") - return APIMain(ctx, &APIServerParams{ - Port: uint16(cmd.Uint("port")), + return cmd.APIMain(ctx, &cmd.APIServerParams{ + Port: uint16(clicmd.Uint("port")), TemporalHostPort: temporalHostPort, - GatewayPort: uint16(cmd.Uint("gateway-port")), - TemporalNamespace: cmd.String("temporal-namespace"), - TemporalCert: cmd.String("temporal-cert"), - TemporalKey: cmd.String("temporal-key"), + GatewayPort: uint16(clicmd.Uint("gateway-port")), + TemporalNamespace: clicmd.String("temporal-namespace"), + TemporalCert: clicmd.String("temporal-cert"), + TemporalKey: clicmd.String("temporal-key"), }) }, }, }, } + go func() { + sigs := make(chan os.Signal, 1) + signal.Notify(sigs, syscall.SIGQUIT) + buf := make([]byte, 1<<20) + for { + <-sigs + stacklen := runtime.Stack(buf, true) + log.Printf("=== received SIGQUIT ===\n*** goroutine dump...\n%s\n*** end\n", buf[:stacklen]) + } + }() + if err := app.Run(appCtx, os.Args); err != nil { log.Printf("error running app: %+v", err) } diff --git a/flow/workflows/sync_flow.go b/flow/workflows/sync_flow.go index 0e4c786e2c..0dd5a1e4d9 100644 --- a/flow/workflows/sync_flow.go +++ b/flow/workflows/sync_flow.go @@ -204,7 +204,7 @@ func SyncFlowWorkflow( } } if err := ctx.Err(); err != nil { - logger.Info("sync canceled: %v", err) + logger.Info("sync canceled", slog.Any("error", err)) return err } else if stop { return nil diff --git a/stacks/flow.Dockerfile b/stacks/flow.Dockerfile index 68e7e7d5aa..f991421c40 100644 --- a/stacks/flow.Dockerfile +++ b/stacks/flow.Dockerfile @@ -13,10 +13,10 @@ RUN go mod download # Copy all the code COPY flow . -# build the binary from cmd folder -WORKDIR /root/flow/cmd +# build the binary from flow folder +WORKDIR /root/flow ENV CGO_ENABLED=1 -RUN go build -ldflags="-s -w" -o /root/peer-flow . +RUN go build -ldflags="-s -w" -o /root/peer-flow FROM debian:bookworm-slim AS flow-base RUN apt-get update && apt-get install -y ca-certificates libgeos-c1v5