From 3c27655bc438ff2d0fd0da80451b8c3fa3d66d8b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Mon, 29 Jan 2024 18:18:04 +0000 Subject: [PATCH] postgres connector: replace pgxpool.Pool with pgx.Conn (#1163) Pools make sense when shared across application (eg, CatalogPool) It doesn't make sense creating a pool per connector, & it turns out with a bit of modification we can get by with connections This comes with some restrictions: Connections require exclusion So when using a goroutine, a 2nd connection may be necessary (this was transparently handled by pool before) Similarly, rows must be closed before executing another query --- flow/activities/slot.go | 60 +------- flow/connectors/core.go | 8 +- flow/connectors/postgres/cdc.go | 1 - flow/connectors/postgres/client.go | 81 +++++----- flow/connectors/postgres/postgres.go | 140 ++++++++++++------ .../postgres/postgres_schema_delta_test.go | 12 +- flow/connectors/postgres/qrep.go | 26 ++-- flow/connectors/postgres/qrep_bench_test.go | 12 +- .../postgres/qrep_partition_test.go | 22 +-- .../postgres/qrep_query_executor.go | 28 ++-- .../postgres/qrep_query_executor_test.go | 65 +++----- flow/connectors/postgres/qrep_sql_sync.go | 18 ++- flow/connectors/utils/postgres.go | 5 +- flow/e2e/bigquery/peer_flow_bq_test.go | 140 +++++++++--------- flow/e2e/bigquery/qrep_flow_bq_test.go | 4 +- flow/e2e/congen.go | 85 +++++------ flow/e2e/postgres/peer_flow_pg_test.go | 90 +++++------ flow/e2e/postgres/qrep_flow_pg_test.go | 30 ++-- flow/e2e/s3/cdc_s3_test.go | 4 +- flow/e2e/s3/qrep_flow_s3_test.go | 18 +-- flow/e2e/snowflake/peer_flow_sf_test.go | 132 ++++++++--------- flow/e2e/snowflake/qrep_flow_sf_test.go | 6 +- .../e2e/sqlserver/qrep_flow_sqlserver_test.go | 18 +-- flow/e2e/test_utils.go | 35 ++--- flow/e2eshared/e2eshared.go | 4 +- 25 files changed, 520 insertions(+), 524 deletions(-) diff --git a/flow/activities/slot.go b/flow/activities/slot.go index 90c20deeb4..8f6f3325b4 100644 --- a/flow/activities/slot.go +++ b/flow/activities/slot.go @@ -2,67 +2,11 @@ package activities import ( "context" - "fmt" - "log/slog" "time" "github.com/PeerDB-io/peer-flow/connectors" - "github.com/PeerDB-io/peer-flow/connectors/utils/monitoring" - "github.com/PeerDB-io/peer-flow/dynamicconf" - "github.com/PeerDB-io/peer-flow/peerdbenv" ) -func (a *FlowableActivity) handleSlotInfo( - ctx context.Context, - srcConn connectors.CDCPullConnector, - slotName string, - peerName string, -) error { - slotInfo, err := srcConn.GetSlotInfo(slotName) - if err != nil { - slog.WarnContext(ctx, "warning: failed to get slot info", slog.Any("error", err)) - return err - } - - if len(slotInfo) == 0 { - slog.WarnContext(ctx, "warning: unable to get slot info", slog.Any("slotName", slotName)) - return nil - } - - deploymentUIDPrefix := "" - if peerdbenv.PeerDBDeploymentUID() != "" { - deploymentUIDPrefix = fmt.Sprintf("[%s] ", peerdbenv.PeerDBDeploymentUID()) - } - - slotLagInMBThreshold := dynamicconf.PeerDBSlotLagMBAlertThreshold(ctx) - if (slotLagInMBThreshold > 0) && (slotInfo[0].LagInMb >= float32(slotLagInMBThreshold)) { - a.Alerter.AlertIf(ctx, fmt.Sprintf("%s-slot-lag-threshold-exceeded", peerName), - fmt.Sprintf(`%sSlot `+"`%s`"+` on peer `+"`%s`"+` has exceeded threshold size of %dMB, currently at %.2fMB! -cc: `, - deploymentUIDPrefix, slotName, peerName, slotLagInMBThreshold, slotInfo[0].LagInMb)) - } - - // Also handles alerts for PeerDB user connections exceeding a given limit here - maxOpenConnectionsThreshold := dynamicconf.PeerDBOpenConnectionsAlertThreshold(ctx) - res, err := srcConn.GetOpenConnectionsForUser() - if err != nil { - slog.WarnContext(ctx, "warning: failed to get current open connections", slog.Any("error", err)) - return err - } - if (maxOpenConnectionsThreshold > 0) && (res.CurrentOpenConnections >= int64(maxOpenConnectionsThreshold)) { - a.Alerter.AlertIf(ctx, fmt.Sprintf("%s-max-open-connections-threshold-exceeded", peerName), - fmt.Sprintf(`%sOpen connections from PeerDB user `+"`%s`"+` on peer `+"`%s`"+ - ` has exceeded threshold size of %d connections, currently at %d connections! -cc: `, - deploymentUIDPrefix, res.UserName, peerName, maxOpenConnectionsThreshold, res.CurrentOpenConnections)) - } - - if len(slotInfo) != 0 { - return monitoring.AppendSlotSizeInfo(ctx, a.CatalogPool, peerName, slotInfo[0]) - } - return nil -} - func (a *FlowableActivity) recordSlotSizePeriodically( ctx context.Context, srcConn connectors.CDCPullConnector, @@ -70,7 +14,7 @@ func (a *FlowableActivity) recordSlotSizePeriodically( peerName string, ) { // ensures slot info is logged at least once per SyncFlow - err := a.handleSlotInfo(ctx, srcConn, slotName, peerName) + err := srcConn.HandleSlotInfo(ctx, a.Alerter, a.CatalogPool, slotName, peerName) if err != nil { return } @@ -81,7 +25,7 @@ func (a *FlowableActivity) recordSlotSizePeriodically( for { select { case <-ticker.C: - err := a.handleSlotInfo(ctx, srcConn, slotName, peerName) + err := srcConn.HandleSlotInfo(ctx, a.Alerter, a.CatalogPool, slotName, peerName) if err != nil { return } diff --git a/flow/connectors/core.go b/flow/connectors/core.go index 1e28822181..5659fdf7f7 100644 --- a/flow/connectors/core.go +++ b/flow/connectors/core.go @@ -17,6 +17,7 @@ import ( connsqlserver "github.com/PeerDB-io/peer-flow/connectors/sqlserver" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" + "github.com/PeerDB-io/peer-flow/shared/alerting" ) var ErrUnsupportedFunctionality = errors.New("requested connector does not support functionality") @@ -45,12 +46,13 @@ type CDCPullConnector interface { // PullFlowCleanup drops both the Postgres publication and replication slot, as a part of DROP MIRROR PullFlowCleanup(jobName string) error + // HandleSlotInfo update monitoring info on slot size etc + // threadsafe + HandleSlotInfo(ctx context.Context, alerter *alerting.Alerter, catalogPool *pgxpool.Pool, slotName string, peerName string) error + // GetSlotInfo returns the WAL (or equivalent) info of a slot for the connector. GetSlotInfo(slotName string) ([]*protos.SlotInfo, error) - // GetOpenConnectionsForUser returns the number of open connections for the user configured in the peer. - GetOpenConnectionsForUser() (*protos.GetOpenConnectionsForUserResult, error) - // AddTablesToPublication adds additional tables added to a mirror to the publication also AddTablesToPublication(req *protos.AddTablesToPublicationInput) error } diff --git a/flow/connectors/postgres/cdc.go b/flow/connectors/postgres/cdc.go index 61ac9ca8a4..18cada42c7 100644 --- a/flow/connectors/postgres/cdc.go +++ b/flow/connectors/postgres/cdc.go @@ -139,7 +139,6 @@ func (p *PostgresCDCSource) PullRecords(req *model.PullRecordsRequest) error { } pgConn := p.replConn.PgConn() - p.logger.Info("created replication connection") // start replication var clientXLogPos, startLSN pglogrepl.LSN diff --git a/flow/connectors/postgres/client.go b/flow/connectors/postgres/client.go index f5c08bb8a7..8c582b7b05 100644 --- a/flow/connectors/postgres/client.go +++ b/flow/connectors/postgres/client.go @@ -1,6 +1,7 @@ package connpostgres import ( + "context" "errors" "fmt" "regexp" @@ -96,7 +97,7 @@ const ( // getRelIDForTable returns the relation ID for a table. func (c *PostgresConnector) getRelIDForTable(schemaTable *utils.SchemaTable) (uint32, error) { var relID pgtype.Uint32 - err := c.pool.QueryRow(c.ctx, + err := c.conn.QueryRow(c.ctx, `SELECT c.oid FROM pg_class c JOIN pg_namespace n ON n.oid = c.relnamespace WHERE n.nspname=$1 AND c.relname=$2`, schemaTable.Schema, schemaTable.Table).Scan(&relID) @@ -115,7 +116,7 @@ func (c *PostgresConnector) getReplicaIdentityType(schemaTable *utils.SchemaTabl } var replicaIdentity rune - err := c.pool.QueryRow(c.ctx, + err := c.conn.QueryRow(c.ctx, `SELECT relreplident FROM pg_class WHERE oid = $1;`, relID).Scan(&replicaIdentity) if err != nil { @@ -147,7 +148,7 @@ func (c *PostgresConnector) getUniqueColumns( // Find the primary key index OID, for replica identity 'd'/default or 'f'/full var pkIndexOID oid.Oid - err = c.pool.QueryRow(c.ctx, + err = c.conn.QueryRow(c.ctx, `SELECT indexrelid FROM pg_index WHERE indrelid = $1 AND indisprimary`, relID).Scan(&pkIndexOID) if err != nil { @@ -165,7 +166,7 @@ func (c *PostgresConnector) getUniqueColumns( func (c *PostgresConnector) getReplicaIdentityIndexColumns(relID uint32, schemaTable *utils.SchemaTable) ([]string, error) { var indexRelID oid.Oid // Fetch the OID of the index used as the replica identity - err := c.pool.QueryRow(c.ctx, + err := c.conn.QueryRow(c.ctx, `SELECT indexrelid FROM pg_index WHERE indrelid=$1 AND indisreplident=true`, relID).Scan(&indexRelID) @@ -178,7 +179,7 @@ func (c *PostgresConnector) getReplicaIdentityIndexColumns(relID uint32, schemaT // getColumnNamesForIndex returns the column names for a given index. func (c *PostgresConnector) getColumnNamesForIndex(indexOID oid.Oid) ([]string, error) { - rows, err := c.pool.Query(c.ctx, + rows, err := c.conn.Query(c.ctx, `SELECT a.attname FROM pg_index i JOIN pg_attribute a ON a.attrelid = i.indrelid AND a.attnum = ANY(i.indkey) WHERE i.indexrelid = $1 ORDER BY a.attname ASC`, @@ -196,7 +197,7 @@ func (c *PostgresConnector) getColumnNamesForIndex(indexOID oid.Oid) ([]string, func (c *PostgresConnector) tableExists(schemaTable *utils.SchemaTable) (bool, error) { var exists pgtype.Bool - err := c.pool.QueryRow(c.ctx, + err := c.conn.QueryRow(c.ctx, `SELECT EXISTS ( SELECT FROM pg_tables WHERE schemaname = $1 @@ -219,7 +220,7 @@ func (c *PostgresConnector) checkSlotAndPublication(slot string, publication str // Check if the replication slot exists var slotName pgtype.Text - err := c.pool.QueryRow(c.ctx, + err := c.conn.QueryRow(c.ctx, "SELECT slot_name FROM pg_replication_slots WHERE slot_name = $1", slot).Scan(&slotName) if err != nil { @@ -233,7 +234,7 @@ func (c *PostgresConnector) checkSlotAndPublication(slot string, publication str // Check if the publication exists var pubName pgtype.Text - err = c.pool.QueryRow(c.ctx, + err = c.conn.QueryRow(c.ctx, "SELECT pubname FROM pg_publication WHERE pubname = $1", publication).Scan(&pubName) if err != nil { @@ -251,18 +252,15 @@ func (c *PostgresConnector) checkSlotAndPublication(slot string, publication str }, nil } -// GetSlotInfo gets the information about the replication slot size and LSNs -// If slotName input is empty, all slot info rows are returned - this is for UI. -// Else, only the row pertaining to that slotName will be returned. -func (c *PostgresConnector) GetSlotInfo(slotName string) ([]*protos.SlotInfo, error) { +func getSlotInfo(ctx context.Context, conn *pgx.Conn, slotName string, database string) ([]*protos.SlotInfo, error) { var whereClause string if slotName != "" { whereClause = fmt.Sprintf("WHERE slot_name=%s", QuoteLiteral(slotName)) } else { - whereClause = fmt.Sprintf("WHERE database=%s", QuoteLiteral(c.config.Database)) + whereClause = fmt.Sprintf("WHERE database=%s", QuoteLiteral(database)) } - hasWALStatus, _, err := c.MajorVersionCheck(POSTGRES_13) + hasWALStatus, _, err := majorVersionCheck(ctx, conn, POSTGRES_13) if err != nil { return nil, err } @@ -270,7 +268,7 @@ func (c *PostgresConnector) GetSlotInfo(slotName string) ([]*protos.SlotInfo, er if !hasWALStatus { walStatusSelector = "'unknown'" } - rows, err := c.pool.Query(c.ctx, fmt.Sprintf(`SELECT slot_name, redo_lsn::Text,restart_lsn::text,%s, + rows, err := conn.Query(ctx, fmt.Sprintf(`SELECT slot_name, redo_lsn::Text,restart_lsn::text,%s, confirmed_flush_lsn::text,active, round((CASE WHEN pg_is_in_recovery() THEN pg_last_wal_receive_lsn() ELSE pg_current_wal_lsn() END - confirmed_flush_lsn) / 1024 / 1024) AS MB_Behind @@ -306,6 +304,13 @@ func (c *PostgresConnector) GetSlotInfo(slotName string) ([]*protos.SlotInfo, er return slotInfoRows, nil } +// GetSlotInfo gets the information about the replication slot size and LSNs +// If slotName input is empty, all slot info rows are returned - this is for UI. +// Else, only the row pertaining to that slotName will be returned. +func (c *PostgresConnector) GetSlotInfo(slotName string) ([]*protos.SlotInfo, error) { + return getSlotInfo(c.ctx, c.conn, slotName, c.config.Database) +} + // createSlotAndPublication creates the replication slot and publication. func (c *PostgresConnector) createSlotAndPublication( signal SlotSignal, @@ -341,7 +346,7 @@ func (c *PostgresConnector) createSlotAndPublication( } // Create the publication to help filter changes only for the given tables stmt := fmt.Sprintf("CREATE PUBLICATION %s FOR TABLE %s %s", publication, tableNameString, pubViaRootString) - _, err = c.pool.Exec(c.ctx, stmt) + _, err = c.conn.Exec(c.ctx, stmt) if err != nil { c.logger.Warn(fmt.Sprintf("Error creating publication '%s': %v", publication, err)) return fmt.Errorf("error creating publication '%s' : %w", publication, err) @@ -352,7 +357,7 @@ func (c *PostgresConnector) createSlotAndPublication( if !s.SlotExists { conn, err := c.CreateReplConn(c.ctx) if err != nil { - return fmt.Errorf("[slot] error acquiring pool: %w", err) + return fmt.Errorf("[slot] error acquiring connection: %w", err) } defer conn.Close(c.ctx) @@ -400,7 +405,7 @@ func (c *PostgresConnector) createSlotAndPublication( } func (c *PostgresConnector) createMetadataSchema() error { - _, err := c.pool.Exec(c.ctx, fmt.Sprintf(createSchemaSQL, c.metadataSchema)) + _, err := c.conn.Exec(c.ctx, fmt.Sprintf(createSchemaSQL, c.metadataSchema)) if err != nil && !utils.IsUniqueError(err) { return fmt.Errorf("error while creating internal schema: %w", err) } @@ -450,7 +455,7 @@ func generateCreateTableSQLForNormalizedTable( func (c *PostgresConnector) GetLastSyncBatchID(jobName string) (int64, error) { var result pgtype.Int8 - err := c.pool.QueryRow(c.ctx, fmt.Sprintf( + err := c.conn.QueryRow(c.ctx, fmt.Sprintf( getLastSyncBatchID_SQL, c.metadataSchema, mirrorJobsTableIdentifier, @@ -467,7 +472,7 @@ func (c *PostgresConnector) GetLastSyncBatchID(jobName string) (int64, error) { func (c *PostgresConnector) GetLastNormalizeBatchID(jobName string) (int64, error) { var result pgtype.Int8 - err := c.pool.QueryRow(c.ctx, fmt.Sprintf( + err := c.conn.QueryRow(c.ctx, fmt.Sprintf( getLastNormalizeBatchID_SQL, c.metadataSchema, mirrorJobsTableIdentifier, @@ -484,7 +489,7 @@ func (c *PostgresConnector) GetLastNormalizeBatchID(jobName string) (int64, erro func (c *PostgresConnector) jobMetadataExists(jobName string) (bool, error) { var result pgtype.Bool - err := c.pool.QueryRow(c.ctx, + err := c.conn.QueryRow(c.ctx, fmt.Sprintf(checkIfJobMetadataExistsSQL, c.metadataSchema, mirrorJobsTableIdentifier), jobName).Scan(&result) if err != nil { return false, fmt.Errorf("error reading result row: %w", err) @@ -492,9 +497,9 @@ func (c *PostgresConnector) jobMetadataExists(jobName string) (bool, error) { return result.Bool, nil } -func (c *PostgresConnector) MajorVersionCheck(majorVersion PGVersion) (bool, int64, error) { +func majorVersionCheck(ctx context.Context, conn *pgx.Conn, majorVersion PGVersion) (bool, int64, error) { var version pgtype.Int8 - err := c.pool.QueryRow(c.ctx, "SELECT current_setting('server_version_num')::INTEGER").Scan(&version) + err := conn.QueryRow(ctx, "SELECT current_setting('server_version_num')::INTEGER").Scan(&version) if err != nil { return false, 0, fmt.Errorf("failed to get server version: %w", err) } @@ -502,6 +507,10 @@ func (c *PostgresConnector) MajorVersionCheck(majorVersion PGVersion) (bool, int return version.Int64 >= int64(majorVersion), version.Int64, nil } +func (c *PostgresConnector) MajorVersionCheck(majorVersion PGVersion) (bool, int64, error) { + return majorVersionCheck(c.ctx, c.conn, majorVersion) +} + func (c *PostgresConnector) updateSyncMetadata(flowJobName string, lastCP int64, syncBatchID int64, syncRecordsTx pgx.Tx, ) error { @@ -533,7 +542,7 @@ func (c *PostgresConnector) getDistinctTableNamesInBatch(flowJobName string, syn ) ([]string, error) { rawTableIdentifier := getRawTableIdentifier(flowJobName) - rows, err := c.pool.Query(c.ctx, fmt.Sprintf(getDistinctDestinationTableNamesSQL, c.metadataSchema, + rows, err := c.conn.Query(c.ctx, fmt.Sprintf(getDistinctDestinationTableNamesSQL, c.metadataSchema, rawTableIdentifier), normalizeBatchID, syncBatchID) if err != nil { return nil, fmt.Errorf("error while retrieving table names for normalization: %w", err) @@ -551,7 +560,7 @@ func (c *PostgresConnector) getTableNametoUnchangedCols(flowJobName string, sync ) (map[string][]string, error) { rawTableIdentifier := getRawTableIdentifier(flowJobName) - rows, err := c.pool.Query(c.ctx, fmt.Sprintf(getTableNameToUnchangedToastColsSQL, c.metadataSchema, + rows, err := c.conn.Query(c.ctx, fmt.Sprintf(getTableNameToUnchangedToastColsSQL, c.metadataSchema, rawTableIdentifier), normalizeBatchID, syncBatchID) if err != nil { return nil, fmt.Errorf("error while retrieving table names for normalization: %w", err) @@ -577,7 +586,7 @@ func (c *PostgresConnector) getTableNametoUnchangedCols(flowJobName string, sync } func (c *PostgresConnector) getCurrentLSN() (pglogrepl.LSN, error) { - row := c.pool.QueryRow(c.ctx, + row := c.conn.QueryRow(c.ctx, "SELECT CASE WHEN pg_is_in_recovery() THEN pg_last_wal_receive_lsn() ELSE pg_current_wal_lsn() END") var result pgtype.Text err := row.Scan(&result) @@ -592,8 +601,8 @@ func (c *PostgresConnector) getDefaultPublicationName(jobName string) string { } func (c *PostgresConnector) CheckSourceTables(tableNames []string, pubName string) error { - if c.pool == nil { - return fmt.Errorf("check tables: pool is nil") + if c.conn == nil { + return fmt.Errorf("check tables: conn is nil") } // Check that we can select from all tables @@ -606,7 +615,7 @@ func (c *PostgresConnector) CheckSourceTables(tableNames []string, pubName strin } tableArr = append(tableArr, fmt.Sprintf(`(%s::text, %s::text)`, QuoteLiteral(schemaName), QuoteLiteral(tableName))) - err := c.pool.QueryRow(c.ctx, + err := c.conn.QueryRow(c.ctx, fmt.Sprintf("SELECT * FROM %s.%s LIMIT 0;", QuoteIdentifier(schemaName), QuoteIdentifier(tableName))).Scan(&row) if err != nil && err != pgx.ErrNoRows { return err @@ -617,7 +626,7 @@ func (c *PostgresConnector) CheckSourceTables(tableNames []string, pubName strin tableStr := strings.Join(tableArr, ",") if pubName != "" { var pubTableCount int - err := c.pool.QueryRow(c.ctx, fmt.Sprintf(` + err := c.conn.QueryRow(c.ctx, fmt.Sprintf(` with source_table_components (sname, tname) as (values %s) select COUNT(DISTINCT(schemaname,tablename)) from pg_publication_tables INNER JOIN source_table_components stc @@ -635,12 +644,12 @@ func (c *PostgresConnector) CheckSourceTables(tableNames []string, pubName strin } func (c *PostgresConnector) CheckReplicationPermissions(username string) error { - if c.pool == nil { - return fmt.Errorf("check replication permissions: pool is nil") + if c.conn == nil { + return fmt.Errorf("check replication permissions: conn is nil") } var replicationRes bool - err := c.pool.QueryRow(c.ctx, "SELECT rolreplication FROM pg_roles WHERE rolname = $1;", username).Scan(&replicationRes) + err := c.conn.QueryRow(c.ctx, "SELECT rolreplication FROM pg_roles WHERE rolname = $1;", username).Scan(&replicationRes) if err != nil { return err } @@ -648,7 +657,7 @@ func (c *PostgresConnector) CheckReplicationPermissions(username string) error { if !replicationRes { // RDS case: check pg_settings for rds.logical_replication var setting string - err := c.pool.QueryRow(c.ctx, "SELECT setting FROM pg_settings WHERE name = 'rds.logical_replication';").Scan(&setting) + err := c.conn.QueryRow(c.ctx, "SELECT setting FROM pg_settings WHERE name = 'rds.logical_replication';").Scan(&setting) if err != nil || setting != "on" { return fmt.Errorf("postgres user does not have replication role") } @@ -656,7 +665,7 @@ func (c *PostgresConnector) CheckReplicationPermissions(username string) error { // check wal_level var walLevel string - err = c.pool.QueryRow(c.ctx, "SHOW wal_level;").Scan(&walLevel) + err = c.conn.QueryRow(c.ctx, "SHOW wal_level;").Scan(&walLevel) if err != nil { return err } @@ -667,7 +676,7 @@ func (c *PostgresConnector) CheckReplicationPermissions(username string) error { // max_wal_senders must be at least 2 var maxWalSendersRes string - err = c.pool.QueryRow(c.ctx, "SHOW max_wal_senders;").Scan(&maxWalSendersRes) + err = c.conn.QueryRow(c.ctx, "SHOW max_wal_senders;").Scan(&maxWalSendersRes) if err != nil { return err } diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index af7d4c2e67..7305a66170 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -16,10 +16,13 @@ import ( "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/connectors/utils/monitoring" + "github.com/PeerDB-io/peer-flow/dynamicconf" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" + "github.com/PeerDB-io/peer-flow/peerdbenv" "github.com/PeerDB-io/peer-flow/shared" + "github.com/PeerDB-io/peer-flow/shared/alerting" ) // PostgresConnector is a Connector implementation for Postgres. @@ -28,7 +31,7 @@ type PostgresConnector struct { ctx context.Context config *protos.PostgresConfig ssh *SSHTunnel - pool *pgxpool.Pool + conn *pgx.Conn replConfig *pgx.ConnConfig customTypesMapping map[uint32]string metadataSchema string @@ -41,35 +44,32 @@ func NewPostgresConnector(ctx context.Context, pgConfig *protos.PostgresConfig) // create a separate connection pool for non-replication queries as replication connections cannot // be used for extended query protocol, i.e. prepared statements - connConfig, err := pgxpool.ParseConfig(connectionString) - replConfig := connConfig.ConnConfig.Copy() + connConfig, err := pgx.ParseConfig(connectionString) + replConfig := connConfig.Copy() if err != nil { return nil, fmt.Errorf("failed to parse connection string: %w", err) } - runtimeParams := connConfig.ConnConfig.RuntimeParams + runtimeParams := connConfig.Config.RuntimeParams runtimeParams["application_name"] = "peerdb_query_executor" runtimeParams["idle_in_transaction_session_timeout"] = "0" runtimeParams["statement_timeout"] = "0" - // set pool size to 3 to avoid connection pool exhaustion - connConfig.MaxConns = 3 - tunnel, err := NewSSHTunnel(ctx, pgConfig.SshConfig) if err != nil { return nil, fmt.Errorf("failed to create ssh tunnel: %w", err) } - pool, err := tunnel.NewPostgresPoolFromConfig(ctx, connConfig) + conn, err := tunnel.NewPostgresConnFromConfig(ctx, connConfig) if err != nil { - return nil, fmt.Errorf("failed to create connection pool: %w", err) + return nil, fmt.Errorf("failed to create connection: %w", err) } // ensure that replication is set to database - replConfig.RuntimeParams["replication"] = "database" - replConfig.RuntimeParams["bytea_output"] = "hex" + replConfig.Config.RuntimeParams["replication"] = "database" + replConfig.Config.RuntimeParams["bytea_output"] = "hex" - customTypeMap, err := utils.GetCustomDataTypes(ctx, pool) + customTypeMap, err := utils.GetCustomDataTypes(ctx, conn) if err != nil { return nil, fmt.Errorf("failed to get custom type map: %w", err) } @@ -86,7 +86,7 @@ func NewPostgresConnector(ctx context.Context, pgConfig *protos.PostgresConfig) ctx: ctx, config: pgConfig, ssh: tunnel, - pool: pool, + conn: conn, replConfig: replConfig, customTypesMapping: customTypeMap, metadataSchema: metadataSchema, @@ -94,16 +94,11 @@ func NewPostgresConnector(ctx context.Context, pgConfig *protos.PostgresConfig) }, nil } -// GetPool returns the connection pool. -func (c *PostgresConnector) GetPool() *pgxpool.Pool { - return c.pool -} - func (c *PostgresConnector) CreateReplConn(ctx context.Context) (*pgx.Conn, error) { conn, err := c.ssh.NewPostgresConnFromConfig(ctx, c.replConfig) if err != nil { - slog.Error("failed to create replication connection pool", slog.Any("error", err)) - return nil, fmt.Errorf("failed to create replication connection pool: %w", err) + slog.Error("failed to create replication connection", slog.Any("error", err)) + return nil, fmt.Errorf("failed to create replication connection: %w", err) } return conn, nil @@ -112,7 +107,7 @@ func (c *PostgresConnector) CreateReplConn(ctx context.Context) (*pgx.Conn, erro // Close closes all connections. func (c *PostgresConnector) Close() error { if c != nil { - c.pool.Close() + c.conn.Close(c.ctx) c.ssh.Close() } return nil @@ -120,10 +115,10 @@ func (c *PostgresConnector) Close() error { // ConnectionActive returns true if the connection is active. func (c *PostgresConnector) ConnectionActive() error { - if c.pool == nil { - return fmt.Errorf("connection pool is nil") + if c.conn == nil { + return fmt.Errorf("connection is nil") } - pingErr := c.pool.Ping(c.ctx) + pingErr := c.conn.Ping(c.ctx) return pingErr } @@ -146,7 +141,7 @@ func (c *PostgresConnector) SetupMetadataTables() error { return err } - _, err = c.pool.Exec(c.ctx, fmt.Sprintf(createMirrorJobsTableSQL, + _, err = c.conn.Exec(c.ctx, fmt.Sprintf(createMirrorJobsTableSQL, c.metadataSchema, mirrorJobsTableIdentifier)) if err != nil && !utils.IsUniqueError(err) { return fmt.Errorf("error creating table %s: %w", mirrorJobsTableIdentifier, err) @@ -158,7 +153,7 @@ func (c *PostgresConnector) SetupMetadataTables() error { // GetLastOffset returns the last synced offset for a job. func (c *PostgresConnector) GetLastOffset(jobName string) (int64, error) { var result pgtype.Int8 - err := c.pool.QueryRow(c.ctx, fmt.Sprintf(getLastOffsetSQL, c.metadataSchema, mirrorJobsTableIdentifier), jobName).Scan(&result) + err := c.conn.QueryRow(c.ctx, fmt.Sprintf(getLastOffsetSQL, c.metadataSchema, mirrorJobsTableIdentifier), jobName).Scan(&result) if err != nil { if err == pgx.ErrNoRows { c.logger.Info("No row found, returning nil") @@ -175,7 +170,7 @@ func (c *PostgresConnector) GetLastOffset(jobName string) (int64, error) { // SetLastOffset updates the last synced offset for a job. func (c *PostgresConnector) SetLastOffset(jobName string, lastOffset int64) error { - _, err := c.pool. + _, err := c.conn. Exec(c.ctx, fmt.Sprintf(setLastOffsetSQL, c.metadataSchema, mirrorJobsTableIdentifier), lastOffset, jobName) if err != nil { return fmt.Errorf("error setting last offset for job %s: %w", jobName, err) @@ -204,7 +199,7 @@ func (c *PostgresConnector) PullRecords(catalogPool *pgxpool.Pool, req *model.Pu // Check if the replication slot and publication exist exists, err := c.checkSlotAndPublication(slotName, publicationName) if err != nil { - return fmt.Errorf("error checking for replication slot and publication: %w", err) + return err } if !exists.PublicationExists { @@ -351,7 +346,7 @@ func (c *PostgresConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S }, nil } - syncRecordsTx, err := c.pool.Begin(c.ctx) + syncRecordsTx, err := c.conn.Begin(c.ctx) if err != nil { return nil, fmt.Errorf("error starting transaction for syncing records: %w", err) } @@ -446,7 +441,7 @@ func (c *PostgresConnector) NormalizeRecords(req *model.NormalizeRecordsRequest) return nil, err } - normalizeRecordsTx, err := c.pool.Begin(c.ctx) + normalizeRecordsTx, err := c.conn.Begin(c.ctx) if err != nil { return nil, fmt.Errorf("error starting transaction for normalizing records: %w", err) } @@ -528,7 +523,7 @@ func (c *PostgresConnector) CreateRawTable(req *protos.CreateRawTableInput) (*pr return nil, fmt.Errorf("error creating internal schema: %w", err) } - createRawTableTx, err := c.pool.Begin(c.ctx) + createRawTableTx, err := c.conn.Begin(c.ctx) if err != nil { return nil, fmt.Errorf("error starting transaction for creating raw table: %w", err) } @@ -600,7 +595,7 @@ func (c *PostgresConnector) getTableSchemaForTable( } // Get the column names and types - rows, err := c.pool.Query(c.ctx, + rows, err := c.conn.Query(c.ctx, fmt.Sprintf(`SELECT * FROM %s LIMIT 0`, schemaTable.String()), pgx.QueryExecModeSimpleProtocol) if err != nil { @@ -649,7 +644,7 @@ func (c *PostgresConnector) SetupNormalizedTables(req *protos.SetupNormalizedTab ) { tableExistsMapping := make(map[string]bool) // Postgres is cool and supports transactional DDL. So we use a transaction. - createNormalizedTablesTx, err := c.pool.Begin(c.ctx) + createNormalizedTablesTx, err := c.conn.Begin(c.ctx) if err != nil { return nil, fmt.Errorf("error starting transaction for creating raw table: %w", err) } @@ -709,7 +704,7 @@ func (c *PostgresConnector) ReplayTableSchemaDeltas( } // Postgres is cool and supports transactional DDL. So we use a transaction. - tableSchemaModifyTx, err := c.pool.Begin(c.ctx) + tableSchemaModifyTx, err := c.conn.Begin(c.ctx) if err != nil { return fmt.Errorf("error starting transaction for schema modification: %w", err) @@ -823,7 +818,7 @@ func (c *PostgresConnector) SetupReplication(signal SlotSignal, req *protos.Setu // Check if the replication slot and publication exist exists, err := c.checkSlotAndPublication(slotName, publicationName) if err != nil { - return fmt.Errorf("error checking for replication slot and publication: %w", err) + return err } tableNameMapping := make(map[string]model.NameAndExclude) @@ -849,7 +844,7 @@ func (c *PostgresConnector) PullFlowCleanup(jobName string) error { publicationName := c.getDefaultPublicationName(jobName) - pullFlowCleanupTx, err := c.pool.Begin(c.ctx) + pullFlowCleanupTx, err := c.conn.Begin(c.ctx) if err != nil { return fmt.Errorf("error starting transaction for flow cleanup: %w", err) } @@ -880,7 +875,7 @@ func (c *PostgresConnector) PullFlowCleanup(jobName string) error { } func (c *PostgresConnector) SyncFlowCleanup(jobName string) error { - syncFlowCleanupTx, err := c.pool.Begin(c.ctx) + syncFlowCleanupTx, err := c.conn.Begin(c.ctx) if err != nil { return fmt.Errorf("unable to begin transaction for sync flow cleanup: %w", err) } @@ -908,10 +903,69 @@ func (c *PostgresConnector) SyncFlowCleanup(jobName string) error { return nil } +func (c *PostgresConnector) HandleSlotInfo( + ctx context.Context, + alerter *alerting.Alerter, + catalogPool *pgxpool.Pool, + slotName string, + peerName string, +) error { + // must create new connection because HandleSlotInfo is threadsafe + conn, err := c.ssh.NewPostgresConnFromPostgresConfig(ctx, c.config) + if err != nil { + slog.WarnContext(ctx, "warning: failed to connect to get slot info", slog.Any("error", err)) + return err + } + defer conn.Close(ctx) + + slotInfo, err := c.GetSlotInfo(slotName) + if err != nil { + slog.WarnContext(ctx, "warning: failed to get slot info", slog.Any("error", err)) + return err + } + + if len(slotInfo) == 0 { + slog.WarnContext(ctx, "warning: unable to get slot info", slog.Any("slotName", slotName)) + return nil + } + + deploymentUIDPrefix := "" + if peerdbenv.PeerDBDeploymentUID() != "" { + deploymentUIDPrefix = fmt.Sprintf("[%s] ", peerdbenv.PeerDBDeploymentUID()) + } + + slotLagInMBThreshold := dynamicconf.PeerDBSlotLagMBAlertThreshold(ctx) + if (slotLagInMBThreshold > 0) && (slotInfo[0].LagInMb >= float32(slotLagInMBThreshold)) { + alerter.AlertIf(ctx, fmt.Sprintf("%s-slot-lag-threshold-exceeded", peerName), + fmt.Sprintf(`%sSlot `+"`%s`"+` on peer `+"`%s`"+` has exceeded threshold size of %dMB, currently at %.2fMB! +cc: `, + deploymentUIDPrefix, slotName, peerName, slotLagInMBThreshold, slotInfo[0].LagInMb)) + } + + // Also handles alerts for PeerDB user connections exceeding a given limit here + maxOpenConnectionsThreshold := dynamicconf.PeerDBOpenConnectionsAlertThreshold(ctx) + res, err := getOpenConnectionsForUser(ctx, conn, c.config.User) + if err != nil { + slog.WarnContext(ctx, "warning: failed to get current open connections", slog.Any("error", err)) + return err + } + if (maxOpenConnectionsThreshold > 0) && (res.CurrentOpenConnections >= int64(maxOpenConnectionsThreshold)) { + alerter.AlertIf(ctx, fmt.Sprintf("%s-max-open-connections-threshold-exceeded", peerName), + fmt.Sprintf(`%sOpen connections from PeerDB user `+"`%s`"+` on peer `+"`%s`"+ + ` has exceeded threshold size of %d connections, currently at %d connections! +cc: `, + deploymentUIDPrefix, res.UserName, peerName, maxOpenConnectionsThreshold, res.CurrentOpenConnections)) + } + + if len(slotInfo) != 0 { + return monitoring.AppendSlotSizeInfo(ctx, catalogPool, peerName, slotInfo[0]) + } + return nil +} + // GetLastOffset returns the last synced offset for a job. -func (c *PostgresConnector) GetOpenConnectionsForUser() (*protos.GetOpenConnectionsForUserResult, error) { - row := c.pool. - QueryRow(c.ctx, getNumConnectionsForUser, c.config.User) +func getOpenConnectionsForUser(ctx context.Context, conn *pgx.Conn, user string) (*protos.GetOpenConnectionsForUserResult, error) { + row := conn.QueryRow(ctx, getNumConnectionsForUser, user) // COUNT() returns BIGINT var result pgtype.Int8 @@ -921,7 +975,7 @@ func (c *PostgresConnector) GetOpenConnectionsForUser() (*protos.GetOpenConnecti } return &protos.GetOpenConnectionsForUserResult{ - UserName: c.config.User, + UserName: user, CurrentOpenConnections: result.Int64, }, nil } @@ -939,7 +993,7 @@ func (c *PostgresConnector) AddTablesToPublication(req *protos.AddTablesToPublic // just check if we have all the tables already in the publication if req.PublicationName != "" { - rows, err := c.pool.Query(c.ctx, + rows, err := c.conn.Query(c.ctx, "SELECT tablename FROM pg_publication_tables WHERE pubname=$1", req.PublicationName) if err != nil { return fmt.Errorf("failed to check tables in publication: %w", err) @@ -957,7 +1011,7 @@ func (c *PostgresConnector) AddTablesToPublication(req *protos.AddTablesToPublic } additionalSrcTablesString := strings.Join(additionalSrcTables, ",") - _, err := c.pool.Exec(c.ctx, fmt.Sprintf("ALTER PUBLICATION %s ADD TABLE %s", + _, err := c.conn.Exec(c.ctx, fmt.Sprintf("ALTER PUBLICATION %s ADD TABLE %s", c.getDefaultPublicationName(req.FlowJobName), additionalSrcTablesString)) if err != nil { return fmt.Errorf("failed to alter publication: %w", err) diff --git a/flow/connectors/postgres/postgres_schema_delta_test.go b/flow/connectors/postgres/postgres_schema_delta_test.go index 450303da42..79e34b4aa4 100644 --- a/flow/connectors/postgres/postgres_schema_delta_test.go +++ b/flow/connectors/postgres/postgres_schema_delta_test.go @@ -33,7 +33,7 @@ func SetupSuite(t *testing.T) PostgresSchemaDeltaTestSuite { }) require.NoError(t, err) - setupTx, err := connector.pool.Begin(context.Background()) + setupTx, err := connector.conn.Begin(context.Background()) require.NoError(t, err) defer func() { err := setupTx.Rollback(context.Background()) @@ -59,7 +59,7 @@ func SetupSuite(t *testing.T) PostgresSchemaDeltaTestSuite { func (s PostgresSchemaDeltaTestSuite) TestSimpleAddColumn() { tableName := fmt.Sprintf("%s.simple_add_column", s.schema) - _, err := s.connector.pool.Exec(context.Background(), + _, err := s.connector.conn.Exec(context.Background(), fmt.Sprintf("CREATE TABLE %s(id INT PRIMARY KEY)", tableName)) require.NoError(s.t, err) @@ -87,7 +87,7 @@ func (s PostgresSchemaDeltaTestSuite) TestSimpleAddColumn() { func (s PostgresSchemaDeltaTestSuite) TestAddAllColumnTypes() { tableName := fmt.Sprintf("%s.add_drop_all_column_types", s.schema) - _, err := s.connector.pool.Exec(context.Background(), + _, err := s.connector.conn.Exec(context.Background(), fmt.Sprintf("CREATE TABLE %s(id INT PRIMARY KEY)", tableName)) require.NoError(s.t, err) @@ -145,7 +145,7 @@ func (s PostgresSchemaDeltaTestSuite) TestAddAllColumnTypes() { func (s PostgresSchemaDeltaTestSuite) TestAddTrickyColumnNames() { tableName := fmt.Sprintf("%s.add_drop_tricky_column_names", s.schema) - _, err := s.connector.pool.Exec(context.Background(), + _, err := s.connector.conn.Exec(context.Background(), fmt.Sprintf("CREATE TABLE %s(id INT PRIMARY KEY)", tableName)) require.NoError(s.t, err) @@ -195,7 +195,7 @@ func (s PostgresSchemaDeltaTestSuite) TestAddTrickyColumnNames() { func (s PostgresSchemaDeltaTestSuite) TestAddDropWhitespaceColumnNames() { tableName := fmt.Sprintf("%s.add_drop_whitespace_column_names", s.schema) - _, err := s.connector.pool.Exec(context.Background(), + _, err := s.connector.conn.Exec(context.Background(), fmt.Sprintf("CREATE TABLE %s(\" \" INT PRIMARY KEY)", tableName)) require.NoError(s.t, err) @@ -236,7 +236,7 @@ func (s PostgresSchemaDeltaTestSuite) TestAddDropWhitespaceColumnNames() { func TestPostgresSchemaDeltaTestSuite(t *testing.T) { e2eshared.RunSuite(t, SetupSuite, func(s PostgresSchemaDeltaTestSuite) { - teardownTx, err := s.connector.pool.Begin(context.Background()) + teardownTx, err := s.connector.conn.Begin(context.Background()) require.NoError(s.t, err) defer func() { err := teardownTx.Rollback(context.Background()) diff --git a/flow/connectors/postgres/qrep.go b/flow/connectors/postgres/qrep.go index 7efb544650..a73c568cdd 100644 --- a/flow/connectors/postgres/qrep.go +++ b/flow/connectors/postgres/qrep.go @@ -36,7 +36,7 @@ func (c *PostgresConnector) GetQRepPartitions( } // begin a transaction - tx, err := c.pool.BeginTx(c.ctx, pgx.TxOptions{ + tx, err := c.conn.BeginTx(c.ctx, pgx.TxOptions{ AccessMode: pgx.ReadOnly, IsoLevel: pgx.RepeatableRead, }) @@ -169,6 +169,7 @@ func (c *PostgresConnector) getNumRowsPartitions( c.logger.Error(fmt.Sprintf("failed to query for partitions: %v", err)) return nil, fmt.Errorf("failed to query for partitions: %w", err) } + defer rows.Close() partitionHelper := partition_utils.NewPartitionHelper() for rows.Next() { @@ -184,6 +185,11 @@ func (c *PostgresConnector) getNumRowsPartitions( } } + err = rows.Err() + if err != nil { + return nil, fmt.Errorf("failed to read rows: %w", err) + } + err = tx.Commit(c.ctx) if err != nil { return nil, fmt.Errorf("failed to commit transaction: %w", err) @@ -271,7 +277,7 @@ func (c *PostgresConnector) getMinMaxValues( func (c *PostgresConnector) CheckForUpdatedMaxValue(config *protos.QRepConfig, last *protos.QRepPartition, ) (bool, error) { - tx, err := c.pool.Begin(c.ctx) + tx, err := c.conn.Begin(c.ctx) if err != nil { return false, fmt.Errorf("unable to begin transaction for getting max value: %w", err) } @@ -311,7 +317,7 @@ func (c *PostgresConnector) PullQRepRecords( if partition.FullTablePartition { c.logger.Info("pulling full table partition", partitionIdLog) executor, err := NewQRepQueryExecutorSnapshot( - c.pool, c.ctx, c.config.TransactionSnapshot, + c.conn, c.ctx, c.config.TransactionSnapshot, config.FlowJobName, partition.PartitionId) if err != nil { return nil, err @@ -355,7 +361,7 @@ func (c *PostgresConnector) PullQRepRecords( } executor, err := NewQRepQueryExecutorSnapshot( - c.pool, c.ctx, c.config.TransactionSnapshot, + c.conn, c.ctx, c.config.TransactionSnapshot, config.FlowJobName, partition.PartitionId) if err != nil { return nil, err @@ -379,7 +385,7 @@ func (c *PostgresConnector) PullQRepRecordStream( if partition.FullTablePartition { c.logger.Info("pulling full table partition", partitionIdLog) executor, err := NewQRepQueryExecutorSnapshot( - c.pool, c.ctx, c.config.TransactionSnapshot, + c.conn, c.ctx, c.config.TransactionSnapshot, config.FlowJobName, partition.PartitionId) if err != nil { return 0, err @@ -425,7 +431,7 @@ func (c *PostgresConnector) PullQRepRecordStream( } executor, err := NewQRepQueryExecutorSnapshot( - c.pool, c.ctx, c.config.TransactionSnapshot, + c.conn, c.ctx, c.config.TransactionSnapshot, config.FlowJobName, partition.PartitionId) if err != nil { return 0, err @@ -492,7 +498,7 @@ func (c *PostgresConnector) SetupQRepMetadataTables(config *protos.QRepConfig) e syncFinishTime TIMESTAMP DEFAULT NOW() )`, metadataTableIdentifier.Sanitize()) // execute create table query - _, err = c.pool.Exec(c.ctx, createQRepMetadataTableSQL) + _, err = c.conn.Exec(c.ctx, createQRepMetadataTableSQL) if err != nil && !utils.IsUniqueError(err) { return fmt.Errorf("failed to create table %s: %w", qRepMetadataTableName, err) } @@ -500,7 +506,7 @@ func (c *PostgresConnector) SetupQRepMetadataTables(config *protos.QRepConfig) e if config.WriteMode != nil && config.WriteMode.WriteType == protos.QRepWriteType_QREP_WRITE_MODE_OVERWRITE { - _, err = c.pool.Exec(c.ctx, + _, err = c.conn.Exec(c.ctx, fmt.Sprintf("TRUNCATE TABLE %s", config.DestinationTableIdentifier)) if err != nil { return fmt.Errorf("failed to TRUNCATE table before query replication: %w", err) @@ -524,7 +530,7 @@ func (c *PostgresConnector) PullXminRecordStream( } executor, err := NewQRepQueryExecutorSnapshot( - c.pool, c.ctx, c.config.TransactionSnapshot, + c.conn, c.ctx, c.config.TransactionSnapshot, config.FlowJobName, partition.PartitionId) if err != nil { return 0, currentSnapshotXmin, err @@ -578,7 +584,7 @@ func (c *PostgresConnector) isPartitionSynced(partitionID string) (bool, error) // prepare and execute the query var result bool - err := c.pool.QueryRow(c.ctx, queryString, partitionID).Scan(&result) + err := c.conn.QueryRow(c.ctx, queryString, partitionID).Scan(&result) if err != nil { return false, fmt.Errorf("failed to execute query: %w", err) } diff --git a/flow/connectors/postgres/qrep_bench_test.go b/flow/connectors/postgres/qrep_bench_test.go index a8ee28ef15..3848171e83 100644 --- a/flow/connectors/postgres/qrep_bench_test.go +++ b/flow/connectors/postgres/qrep_bench_test.go @@ -4,7 +4,7 @@ import ( "context" "testing" - "github.com/jackc/pgx/v5/pgxpool" + "github.com/jackc/pgx/v5" ) func BenchmarkQRepQueryExecutor(b *testing.B) { @@ -13,15 +13,15 @@ func BenchmarkQRepQueryExecutor(b *testing.B) { ctx := context.Background() - // Create a separate connection pool for non-replication queries - pool, err := pgxpool.New(ctx, connectionString) + // Create a separate connection for non-replication queries + conn, err := pgx.Connect(ctx, connectionString) if err != nil { - b.Fatalf("failed to create connection pool: %v", err) + b.Fatalf("failed to create connection: %v", err) } - defer pool.Close() + defer conn.Close(context.Background()) // Create a new QRepQueryExecutor instance - qe := NewQRepQueryExecutor(pool, context.Background(), "test flow", "test part") + qe := NewQRepQueryExecutor(conn, context.Background(), "test flow", "test part") // Run the benchmark b.ResetTimer() diff --git a/flow/connectors/postgres/qrep_partition_test.go b/flow/connectors/postgres/qrep_partition_test.go index 0d0e0039ba..8ac3a207c1 100644 --- a/flow/connectors/postgres/qrep_partition_test.go +++ b/flow/connectors/postgres/qrep_partition_test.go @@ -7,7 +7,7 @@ import ( "testing" "time" - "github.com/jackc/pgx/v5/pgxpool" + "github.com/jackc/pgx/v5" "github.com/stretchr/testify/assert" "github.com/PeerDB-io/peer-flow/generated/protos" @@ -67,7 +67,7 @@ func TestGetQRepPartitions(t *testing.T) { const connStr = "postgres://postgres:postgres@localhost:7132/postgres" // Setup the DB - config, err := pgxpool.ParseConfig(connStr) + config, err := pgx.ParseConfig(connStr) if err != nil { t.Fatalf("Failed to parse config: %v", err) } @@ -78,11 +78,11 @@ func TestGetQRepPartitions(t *testing.T) { } defer tunnel.Close() - pool, err := tunnel.NewPostgresPoolFromConfig(context.Background(), config) + conn, err := tunnel.NewPostgresConnFromConfig(context.Background(), config) if err != nil { - t.Fatalf("Failed to create pool: %v", err) + t.Fatalf("Failed to create connection: %v", err) } - defer pool.Close() + defer conn.Close(context.Background()) // Generate a random schema name rndUint, err := shared.RandomUInt64() @@ -92,13 +92,13 @@ func TestGetQRepPartitions(t *testing.T) { schemaName := fmt.Sprintf("test_%d", rndUint) // Create the schema - _, err = pool.Exec(context.Background(), fmt.Sprintf(`CREATE SCHEMA %s;`, schemaName)) + _, err = conn.Exec(context.Background(), fmt.Sprintf(`CREATE SCHEMA %s;`, schemaName)) if err != nil { t.Fatalf("Failed to create schema: %v", err) } // Create the table in the new schema - _, err = pool.Exec(context.Background(), fmt.Sprintf(` + _, err = conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s.test ( id SERIAL PRIMARY KEY, value INT NOT NULL, @@ -110,7 +110,7 @@ func TestGetQRepPartitions(t *testing.T) { } // from 2010 Jan 1 10:00 AM UTC to 2010 Jan 30 10:00 AM UTC - numRows := prepareTestData(t, pool, schemaName) + numRows := prepareTestData(t, conn, schemaName) // Define the test cases testCases := []*testCase{ @@ -173,7 +173,7 @@ func TestGetQRepPartitions(t *testing.T) { connStr: connStr, ctx: context.Background(), config: &protos.PostgresConfig{}, - pool: pool, + conn: conn, logger: *slog.With(slog.String(string(shared.FlowNameKey), "testGetQRepPartitions")), } @@ -209,14 +209,14 @@ func TestGetQRepPartitions(t *testing.T) { } // Drop the schema at the end - _, err = pool.Exec(context.Background(), fmt.Sprintf(`DROP SCHEMA %s CASCADE;`, schemaName)) + _, err = conn.Exec(context.Background(), fmt.Sprintf(`DROP SCHEMA %s CASCADE;`, schemaName)) if err != nil { t.Fatalf("Failed to drop schema: %v", err) } } // returns the number of rows inserted -func prepareTestData(t *testing.T, pool *pgxpool.Pool, schema string) int { +func prepareTestData(t *testing.T, pool *pgx.Conn, schema string) int { t.Helper() // Define the start and end times diff --git a/flow/connectors/postgres/qrep_query_executor.go b/flow/connectors/postgres/qrep_query_executor.go index b45d291957..ce4cb0d253 100644 --- a/flow/connectors/postgres/qrep_query_executor.go +++ b/flow/connectors/postgres/qrep_query_executor.go @@ -8,7 +8,6 @@ import ( "github.com/jackc/pgx/v5" "github.com/jackc/pgx/v5/pgconn" "github.com/jackc/pgx/v5/pgtype" - "github.com/jackc/pgx/v5/pgxpool" "go.temporal.io/sdk/activity" "github.com/PeerDB-io/peer-flow/connectors/utils" @@ -19,7 +18,7 @@ import ( ) type QRepQueryExecutor struct { - pool *pgxpool.Pool + conn *pgx.Conn ctx context.Context snapshot string testEnv bool @@ -29,11 +28,11 @@ type QRepQueryExecutor struct { logger slog.Logger } -func NewQRepQueryExecutor(pool *pgxpool.Pool, ctx context.Context, +func NewQRepQueryExecutor(conn *pgx.Conn, ctx context.Context, flowJobName string, partitionID string, ) *QRepQueryExecutor { return &QRepQueryExecutor{ - pool: pool, + conn: conn, ctx: ctx, snapshot: "", flowJobName: flowJobName, @@ -44,18 +43,18 @@ func NewQRepQueryExecutor(pool *pgxpool.Pool, ctx context.Context, } } -func NewQRepQueryExecutorSnapshot(pool *pgxpool.Pool, ctx context.Context, snapshot string, +func NewQRepQueryExecutorSnapshot(conn *pgx.Conn, ctx context.Context, snapshot string, flowJobName string, partitionID string, ) (*QRepQueryExecutor, error) { qrepLog := slog.Group("qrep-metadata", slog.String(string(shared.FlowNameKey), flowJobName), slog.String(string(shared.PartitionIDKey), partitionID)) slog.Info("Declared new qrep executor for snapshot", qrepLog) - CustomTypeMap, err := utils.GetCustomDataTypes(ctx, pool) + CustomTypeMap, err := utils.GetCustomDataTypes(ctx, conn) if err != nil { return nil, fmt.Errorf("failed to get custom data types: %w", err) } return &QRepQueryExecutor{ - pool: pool, + conn: conn, ctx: ctx, snapshot: snapshot, flowJobName: flowJobName, @@ -70,7 +69,7 @@ func (qe *QRepQueryExecutor) SetTestEnv(testEnv bool) { } func (qe *QRepQueryExecutor) ExecuteQuery(query string, args ...interface{}) (pgx.Rows, error) { - rows, err := qe.pool.Query(qe.ctx, query, args...) + rows, err := qe.conn.Query(qe.ctx, query, args...) if err != nil { qe.logger.Error("[pg_query_executor] failed to execute query", slog.Any("error", err)) return nil, err @@ -243,8 +242,6 @@ func (qe *QRepQueryExecutor) processFetchedRows( return 0, fmt.Errorf("failed to process rows: %w", err) } - rows.Close() - if rows.Err() != nil { stream.Records <- model.QRecordOrError{ Err: rows.Err(), @@ -263,9 +260,11 @@ func (qe *QRepQueryExecutor) ExecuteAndProcessQuery( ) (*model.QRecordBatch, error) { stream := model.NewQRecordStream(1024) errors := make(chan error, 1) - defer close(errors) qe.logger.Info("Executing and processing query", slog.String("query", query)) + + // must wait on errors to close before returning to maintain qe.conn exclusion go func() { + defer close(errors) _, err := qe.ExecuteAndProcessQueryStream(stream, query, args...) if err != nil { qe.logger.Error("[pg_query_executor] failed to execute and process query stream", slog.Any("error", err)) @@ -279,6 +278,7 @@ func (qe *QRepQueryExecutor) ExecuteAndProcessQuery( case schema := <-stream.SchemaChan(): if schema.Err != nil { qe.logger.Error("[pg_query_executor] failed to get schema from stream", slog.Any("error", schema.Err)) + <-errors return nil, fmt.Errorf("failed to get schema from stream: %w", schema.Err) } batch := &model.QRecordBatch{ @@ -289,9 +289,11 @@ func (qe *QRepQueryExecutor) ExecuteAndProcessQuery( if record.Err == nil { batch.Records = append(batch.Records, record.Record) } else { + <-errors return nil, fmt.Errorf("[pg] failed to get record from stream: %w", record.Err) } } + <-errors return batch, nil } } @@ -304,7 +306,7 @@ func (qe *QRepQueryExecutor) ExecuteAndProcessQueryStream( qe.logger.Info("Executing and processing query stream", slog.String("query", query)) defer close(stream.Records) - tx, err := qe.pool.BeginTx(qe.ctx, pgx.TxOptions{ + tx, err := qe.conn.BeginTx(qe.ctx, pgx.TxOptions{ AccessMode: pgx.ReadOnly, IsoLevel: pgx.RepeatableRead, }) @@ -326,7 +328,7 @@ func (qe *QRepQueryExecutor) ExecuteAndProcessQueryStreamGettingCurrentSnapshotX qe.logger.Info("Executing and processing query stream", slog.String("query", query)) defer close(stream.Records) - tx, err := qe.pool.BeginTx(qe.ctx, pgx.TxOptions{ + tx, err := qe.conn.BeginTx(qe.ctx, pgx.TxOptions{ AccessMode: pgx.ReadOnly, IsoLevel: pgx.RepeatableRead, }) diff --git a/flow/connectors/postgres/qrep_query_executor_test.go b/flow/connectors/postgres/qrep_query_executor_test.go index fc336e2070..f012dec05a 100644 --- a/flow/connectors/postgres/qrep_query_executor_test.go +++ b/flow/connectors/postgres/qrep_query_executor_test.go @@ -9,18 +9,18 @@ import ( "time" "github.com/google/uuid" - "github.com/jackc/pgx/v5/pgxpool" + "github.com/jackc/pgx/v5" ) -func setupDB(t *testing.T) (*pgxpool.Pool, string) { +func setupDB(t *testing.T) (*pgx.Conn, string) { t.Helper() - config, err := pgxpool.ParseConfig("postgres://postgres:postgres@localhost:7132/postgres") + config, err := pgx.ParseConfig("postgres://postgres:postgres@localhost:7132/postgres") if err != nil { t.Fatalf("unable to parse config: %v", err) } - pool, err := pgxpool.NewWithConfig(context.Background(), config) + conn, err := pgx.ConnectConfig(context.Background(), config) if err != nil { t.Fatalf("unable to connect to database: %v", err) } @@ -29,61 +29,45 @@ func setupDB(t *testing.T) (*pgxpool.Pool, string) { schemaName := fmt.Sprintf("schema_%d", time.Now().Unix()) // Create the schema - _, err = pool.Exec(context.Background(), fmt.Sprintf("CREATE SCHEMA %s;", schemaName)) + _, err = conn.Exec(context.Background(), fmt.Sprintf("CREATE SCHEMA %s;", schemaName)) if err != nil { t.Fatalf("unable to create schema: %v", err) } - return pool, schemaName + return conn, schemaName } -func teardownDB(t *testing.T, pool *pgxpool.Pool, schemaName string) { +func teardownDB(t *testing.T, conn *pgx.Conn, schemaName string) { t.Helper() - _, err := pool.Exec(context.Background(), fmt.Sprintf("DROP SCHEMA %s CASCADE;", schemaName)) + _, err := conn.Exec(context.Background(), fmt.Sprintf("DROP SCHEMA %s CASCADE;", schemaName)) if err != nil { t.Fatalf("error while dropping schema: %v", err) } } -func TestNewQRepQueryExecutor(t *testing.T) { - pool, schema := setupDB(t) - defer pool.Close() - - defer teardownDB(t, pool, schema) - - ctx := context.Background() - qe := NewQRepQueryExecutor(pool, ctx, "test flow", "test part") - - if qe == nil { - t.Fatalf("expected QRepQueryExecutor, got nil") - } -} - func TestExecuteAndProcessQuery(t *testing.T) { - pool, schemaName := setupDB(t) - defer pool.Close() + conn, schemaName := setupDB(t) + defer conn.Close(context.Background()) - defer teardownDB(t, pool, schemaName) + defer teardownDB(t, conn, schemaName) ctx := context.Background() - qe := NewQRepQueryExecutor(pool, ctx, "test flow", "test part") - qe.SetTestEnv(true) - query := fmt.Sprintf("CREATE TABLE IF NOT EXISTS %s.test(id SERIAL PRIMARY KEY, data TEXT);", schemaName) - rows, err := qe.ExecuteQuery(query) + _, err := conn.Exec(ctx, query) if err != nil { t.Fatalf("error while creating test table: %v", err) } - rows.Close() query = fmt.Sprintf("INSERT INTO %s.test(data) VALUES('testdata');", schemaName) - rows, err = qe.ExecuteQuery(query) + _, err = conn.Exec(ctx, query) if err != nil { t.Fatalf("error while inserting into test table: %v", err) } - rows.Close() + + qe := NewQRepQueryExecutor(conn, ctx, "test flow", "test part") + qe.SetTestEnv(true) query = fmt.Sprintf("SELECT * FROM %s.test;", schemaName) batch, err := qe.ExecuteAndProcessQuery(query) @@ -101,14 +85,13 @@ func TestExecuteAndProcessQuery(t *testing.T) { } func TestAllDataTypes(t *testing.T) { - pool, schemaName := setupDB(t) - defer pool.Close() + conn, schemaName := setupDB(t) + defer conn.Close(context.Background()) // Call teardownDB function after test - defer teardownDB(t, pool, schemaName) + defer teardownDB(t, conn, schemaName) ctx := context.Background() - qe := NewQRepQueryExecutor(pool, ctx, "test flow", "test part") // Create a table that contains every data type we want to test query := fmt.Sprintf(` @@ -131,11 +114,10 @@ func TestAllDataTypes(t *testing.T) { col_date DATE );`, schemaName) - rows, err := qe.ExecuteQuery(query) + _, err := conn.Exec(ctx, query) if err != nil { t.Fatalf("error while creating test table: %v", err) } - rows.Close() // Insert a row into the table query = fmt.Sprintf(` @@ -164,7 +146,7 @@ func TestAllDataTypes(t *testing.T) { savedTime := time.Now() savedUUID := uuid.New() - _, err = pool.Exec( + _, err = conn.Exec( context.Background(), query, true, // col_bool @@ -188,11 +170,12 @@ func TestAllDataTypes(t *testing.T) { t.Fatalf("error while inserting into test table: %v", err) } + qe := NewQRepQueryExecutor(conn, ctx, "test flow", "test part") // Select the row back out of the table query = fmt.Sprintf("SELECT * FROM %s.test;", schemaName) - rows, err = qe.ExecuteQuery(query) + rows, err := qe.ExecuteQuery(query) if err != nil { - t.Fatalf("error while executing and processing query: %v", err) + t.Fatalf("error while executing query: %v", err) } defer rows.Close() diff --git a/flow/connectors/postgres/qrep_sql_sync.go b/flow/connectors/postgres/qrep_sql_sync.go index 84593d7668..1ca15ef43f 100644 --- a/flow/connectors/postgres/qrep_sql_sync.go +++ b/flow/connectors/postgres/qrep_sql_sync.go @@ -8,7 +8,6 @@ import ( "time" "github.com/jackc/pgx/v5" - "github.com/jackc/pgx/v5/pgxpool" "google.golang.org/protobuf/encoding/protojson" "github.com/PeerDB-io/peer-flow/connectors/utils" @@ -51,17 +50,22 @@ func (s *QRepStagingTableSync) SyncQRepRecords( return 0, fmt.Errorf("failed to get schema from stream: %w", err) } - txConfig := s.connector.pool.Config() - txConfig.AfterConnect = utils.RegisterHStore - txPool, err := pgxpool.NewWithConfig(s.connector.ctx, txConfig) + txConfig := s.connector.conn.Config() + txConn, err := pgx.ConnectConfig(s.connector.ctx, txConfig) if err != nil { - return 0, fmt.Errorf("failed to create tx pool: %v", err) + return 0, fmt.Errorf("failed to create tx pool: %w", err) + } + defer txConn.Close(s.connector.ctx) + + err = utils.RegisterHStore(s.connector.ctx, txConn) + if err != nil { + return 0, fmt.Errorf("failed to register hstore: %w", err) } // Second transaction - to handle rest of the processing - tx, err := txPool.Begin(context.Background()) + tx, err := txConn.Begin(s.connector.ctx) if err != nil { - return 0, fmt.Errorf("failed to begin transaction: %v", err) + return 0, fmt.Errorf("failed to begin transaction: %w", err) } defer func() { if err := tx.Rollback(context.Background()); err != nil { diff --git a/flow/connectors/utils/postgres.go b/flow/connectors/utils/postgres.go index ed97364417..e39185d802 100644 --- a/flow/connectors/utils/postgres.go +++ b/flow/connectors/utils/postgres.go @@ -10,7 +10,6 @@ import ( "github.com/jackc/pgx/v5" "github.com/jackc/pgx/v5/pgconn" "github.com/jackc/pgx/v5/pgtype" - "github.com/jackc/pgx/v5/pgxpool" "github.com/PeerDB-io/peer-flow/generated/protos" ) @@ -34,8 +33,8 @@ func GetPGConnectionString(pgConfig *protos.PostgresConfig) string { return connString } -func GetCustomDataTypes(ctx context.Context, pool *pgxpool.Pool) (map[uint32]string, error) { - rows, err := pool.Query(ctx, ` +func GetCustomDataTypes(ctx context.Context, conn *pgx.Conn) (map[uint32]string, error) { + rows, err := conn.Query(ctx, ` SELECT t.oid, t.typname as type FROM pg_type t LEFT JOIN pg_catalog.pg_namespace n ON n.oid = t.typnamespace diff --git a/flow/e2e/bigquery/peer_flow_bq_test.go b/flow/e2e/bigquery/peer_flow_bq_test.go index 651f5551a4..4ac89c3b6d 100644 --- a/flow/e2e/bigquery/peer_flow_bq_test.go +++ b/flow/e2e/bigquery/peer_flow_bq_test.go @@ -11,8 +11,8 @@ import ( "time" "github.com/jackc/pgerrcode" + "github.com/jackc/pgx/v5" "github.com/jackc/pgx/v5/pgconn" - "github.com/jackc/pgx/v5/pgxpool" "github.com/joho/godotenv" "github.com/stretchr/testify/require" @@ -30,7 +30,7 @@ type PeerFlowE2ETestSuiteBQ struct { t *testing.T bqSuffix string - pool *pgxpool.Pool + conn *pgx.Conn bqHelper *BigQueryTestHelper } @@ -38,8 +38,8 @@ func (s PeerFlowE2ETestSuiteBQ) T() *testing.T { return s.t } -func (s PeerFlowE2ETestSuiteBQ) Pool() *pgxpool.Pool { - return s.pool +func (s PeerFlowE2ETestSuiteBQ) Conn() *pgx.Conn { + return s.conn } func (s PeerFlowE2ETestSuiteBQ) Suffix() string { @@ -174,8 +174,8 @@ func setupSuite(t *testing.T) PeerFlowE2ETestSuiteBQ { suffix := shared.RandomString(8) tsSuffix := time.Now().Format("20060102150405") bqSuffix := fmt.Sprintf("bq_%s_%s", strings.ToLower(suffix), tsSuffix) - pool, err := e2e.SetupPostgres(bqSuffix) - if err != nil || pool == nil { + conn, err := e2e.SetupPostgres(bqSuffix) + if err != nil || conn == nil { slog.Error("failed to setup postgres", slog.Any("error", err)) t.FailNow() } @@ -185,7 +185,7 @@ func setupSuite(t *testing.T) PeerFlowE2ETestSuiteBQ { return PeerFlowE2ETestSuiteBQ{ t: t, bqSuffix: bqSuffix, - pool: pool, + conn: conn, bqHelper: bq, } } @@ -217,7 +217,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Complete_Flow_No_Data() { srcTableName := s.attachSchemaSuffix("test_no_data") dstTableName := "test_no_data" - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id SERIAL PRIMARY KEY, key TEXT NOT NULL, @@ -258,7 +258,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Char_ColType_Error() { srcTableName := s.attachSchemaSuffix("test_char_coltype") dstTableName := "test_char_coltype" - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id SERIAL PRIMARY KEY, key TEXT NOT NULL, @@ -302,7 +302,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Complete_Simple_Flow_BQ() { srcTableName := s.attachSchemaSuffix("test_simple_flow_bq") dstTableName := "test_simple_flow_bq" - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id SERIAL PRIMARY KEY, key TEXT NOT NULL, @@ -334,7 +334,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Complete_Simple_Flow_BQ() { for i := 0; i < 10; i++ { testKey := fmt.Sprintf("test_key_%d", i) testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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) @@ -366,7 +366,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Toast_BQ() { srcTableName := s.attachSchemaSuffix("test_toast_bq_1") dstTableName := "test_toast_bq_1" - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id SERIAL PRIMARY KEY, t1 text, @@ -401,7 +401,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Toast_BQ() { 2. changes no toast column 2. changes 1 toast column */ - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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); @@ -432,7 +432,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Toast_Nochanges_BQ() { srcTableName := s.attachSchemaSuffix("test_toast_bq_2") dstTableName := "test_toast_bq_2" - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id SERIAL PRIMARY KEY, t1 text, @@ -464,7 +464,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Toast_Nochanges_BQ() { go func() { e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) /* transaction updating no rows */ - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err = s.conn.Exec(context.Background(), fmt.Sprintf(` BEGIN; UPDATE %s SET k=102 WHERE id=1; UPDATE %s SET t1='dummy' WHERE id=2; @@ -495,7 +495,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_1_BQ() { srcTableName := s.attachSchemaSuffix("test_toast_bq_3") dstTableName := "test_toast_bq_3" - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id SERIAL PRIMARY KEY, t1 text, @@ -525,7 +525,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_1_BQ() { go func() { e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) // complex transaction with random DMLs on a table with toast columns - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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); @@ -567,7 +567,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_2_BQ() { srcTableName := s.attachSchemaSuffix("test_toast_bq_4") dstTableName := "test_toast_bq_4" - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE %s ( id SERIAL PRIMARY KEY, t1 text, @@ -596,7 +596,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_2_BQ() { go func() { e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) // complex transaction with random DMLs on a table with toast columns - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err = s.conn.Exec(context.Background(), fmt.Sprintf(` BEGIN; INSERT INTO %s(t1,k) SELECT random_string(9000), 1 FROM generate_series(1,1); @@ -632,7 +632,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_3_BQ() { srcTableName := s.attachSchemaSuffix("test_toast_bq_5") dstTableName := "test_toast_bq_5" - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id SERIAL PRIMARY KEY, t1 text, @@ -665,7 +665,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_3_BQ() { transaction updating a single row multiple times with changed/unchanged toast columns */ - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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); @@ -698,12 +698,12 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Types_BQ() { dstTableName := "test_types_bq" createMoodEnum := "CREATE TYPE mood AS ENUM ('happy', 'sad', 'angry');" var pgErr *pgconn.PgError - _, enumErr := s.pool.Exec(context.Background(), createMoodEnum) + _, enumErr := s.conn.Exec(context.Background(), createMoodEnum) if errors.As(enumErr, &pgErr) && pgErr.Code != pgerrcode.DuplicateObject && !utils.IsUniqueError(pgErr) { require.NoError(s.t, enumErr) } - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s (id serial PRIMARY KEY,c1 BIGINT,c2 BIT,c3 VARBIT,c4 BOOLEAN, c6 BYTEA,c7 CHARACTER,c8 varchar,c9 CIDR,c11 DATE,c12 FLOAT,c13 DOUBLE PRECISION, c14 INET,c15 INTEGER,c16 INTERVAL,c17 JSON,c18 JSONB,c21 MACADDR,c22 MONEY, @@ -734,7 +734,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Types_BQ() { go func() { e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) /* test inserting various types*/ - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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, @@ -798,7 +798,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_NaN_Doubles_BQ() { srcTableName := s.attachSchemaSuffix("test_nans_bq") dstTableName := "test_nans_bq" - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s (id serial PRIMARY KEY,c1 double precision,c2 double precision[]); `, srcTableName)) require.NoError(s.t, err) @@ -823,7 +823,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_NaN_Doubles_BQ() { go func() { e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) /* test inserting various types*/ - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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) @@ -851,7 +851,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Invalid_Geo_BQ_Avro_CDC() { srcTableName := s.attachSchemaSuffix("test_invalid_geo_bq_avro_cdc") dstTableName := "test_invalid_geo_bq_avro_cdc" - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id SERIAL PRIMARY KEY, line GEOMETRY(LINESTRING) NOT NULL, @@ -881,7 +881,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Invalid_Geo_BQ_Avro_CDC() { e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) // insert 4 invalid shapes and 6 valid shapes into the source table for i := 0; i < 4; i++ { - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err = s.conn.Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s (line,"polyPoly") VALUES ($1,$2) `, srcTableName), "010200000001000000000000000000F03F0000000000000040", "0103000020e6100000010000000c0000001a8361d35dc64140afdb8d2b1bc3c9bf1b8ed4685fc641405ba64c"+ @@ -893,7 +893,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Invalid_Geo_BQ_Avro_CDC() { } s.t.Log("Inserted 4 invalid geography rows into the source table") for i := 4; i < 10; i++ { - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err = s.conn.Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s (line,"polyPoly") VALUES ($1,$2) `, srcTableName), "010200000002000000000000000000F03F000000000000004000000000000008400000000000001040", "010300000001000000050000000000000000000000000000000000000000000000"+ @@ -934,7 +934,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Multi_Table_BQ() { srcTable2Name := s.attachSchemaSuffix("test2_bq") dstTable2Name := "test2_bq" - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE %s (id serial primary key, c1 int, c2 text); CREATE TABLE %s(id serial primary key, c1 int, c2 text); `, srcTable1Name, srcTable2Name)) @@ -960,7 +960,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Multi_Table_BQ() { go func() { e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) /* inserting across multiple tables*/ - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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)) @@ -991,7 +991,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Simple_Schema_Changes_BQ() { tableName := "test_simple_schema_changes" srcTableName := s.attachSchemaSuffix(tableName) - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id BIGINT PRIMARY KEY GENERATED ALWAYS AS IDENTITY, c1 BIGINT @@ -1019,7 +1019,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Simple_Schema_Changes_BQ() { go func() { // insert first row. e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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") @@ -1027,11 +1027,11 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Simple_Schema_Changes_BQ() { e2e.EnvWaitForEqualTables(env, s, "normalize insert", tableName, "id,c1") // alter source table, add column c2 and insert another row. - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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") @@ -1040,11 +1040,11 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Simple_Schema_Changes_BQ() { 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.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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") @@ -1053,11 +1053,11 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Simple_Schema_Changes_BQ() { e2e.EnvWaitForEqualTables(env, s, "normalize altered row", tableName, "id,c1,c3") // alter source table, drop column c3 and insert another row. - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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") @@ -1078,7 +1078,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Composite_PKey_BQ() { tableName := "test_simple_cpkey" srcTableName := s.attachSchemaSuffix("test_simple_cpkey") - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id INT GENERATED ALWAYS AS IDENTITY, c1 INT GENERATED BY DEFAULT AS IDENTITY, @@ -1111,7 +1111,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Composite_PKey_BQ() { // insert 10 rows into the source table for i := 0; i < 10; i++ { testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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) @@ -1121,10 +1121,10 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Composite_PKey_BQ() { // verify we got our 10 rows e2e.EnvWaitForEqualTables(env, s, "normalize table", tableName, "id,c1,c2,t") - _, err := s.pool.Exec(context.Background(), + _, 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.pool.Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=$1`, srcTableName), 0) + _, 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,c1,c2,t") @@ -1142,7 +1142,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Composite_PKey_Toast_1_BQ() { srcTableName := s.attachSchemaSuffix("test_cpkey_toast1") dstTableName := "test_cpkey_toast1" - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id INT GENERATED ALWAYS AS IDENTITY, c1 INT GENERATED BY DEFAULT AS IDENTITY, @@ -1173,7 +1173,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Composite_PKey_Toast_1_BQ() { // and then insert, update and delete rows in the table. go func() { e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - rowsTx, err := s.pool.Begin(context.Background()) + rowsTx, err := s.conn.Begin(context.Background()) e2e.EnvNoError(s.t, env, err) // insert 10 rows into the source table @@ -1216,7 +1216,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Composite_PKey_Toast_2_BQ() { tableName := "test_cpkey_toast2" srcTableName := s.attachSchemaSuffix("test_cpkey_toast2") - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id INT GENERATED ALWAYS AS IDENTITY, c1 INT GENERATED BY DEFAULT AS IDENTITY, @@ -1251,7 +1251,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Composite_PKey_Toast_2_BQ() { // insert 10 rows into the source table for i := 0; i < 10; i++ { testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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) @@ -1259,10 +1259,10 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Composite_PKey_Toast_2_BQ() { s.t.Log("Inserted 10 rows into the source table") e2e.EnvWaitForEqualTables(env, s, "normalize table", tableName, "id,c2,t,t2") - _, err = s.pool.Exec(context.Background(), + _, 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.pool.Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=$1`, srcTableName), 0) + _, 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") @@ -1278,7 +1278,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Columns_BQ() { srcTableName := s.attachSchemaSuffix("test_peerdb_cols") dstTableName := "test_peerdb_cols_dst" - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id SERIAL PRIMARY KEY, key TEXT NOT NULL, @@ -1307,13 +1307,13 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Columns_BQ() { // insert 1 row into the source table testKey := fmt.Sprintf("test_key_%d", 1) testValue := fmt.Sprintf("test_value_%d", 1) - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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) // delete that row - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err = s.conn.Exec(context.Background(), fmt.Sprintf(` DELETE FROM %s WHERE id=1 `, srcTableName)) e2e.EnvNoError(s.t, env, err) @@ -1342,7 +1342,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Multi_Table_Multi_Dataset_BQ() { srcTable2Name := s.attachSchemaSuffix("test2_bq") dstTable2Name := "test2_bq" - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE %s(id serial primary key, c1 int, c2 text); CREATE TABLE %s(id serial primary key, c1 int, c2 text); `, srcTable1Name, srcTable2Name)) @@ -1371,7 +1371,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Multi_Table_Multi_Dataset_BQ() { go func() { e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) /* inserting across multiple tables*/ - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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)) @@ -1405,7 +1405,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Soft_Delete_Basic() { srcName := "test_softdel_src" srcTableName := s.attachSchemaSuffix(srcName) - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id INT PRIMARY KEY GENERATED ALWAYS AS IDENTITY, c1 INT, @@ -1445,19 +1445,19 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Soft_Delete_Basic() { go func() { e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - _, err = s.pool.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.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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.pool, s.bqSuffix, srcName, "id,c1,c2,t") + pgRows, err := e2e.GetPgRows(s.conn, s.bqSuffix, srcName, "id,c1,c2,t") if err != nil { return false } @@ -1489,7 +1489,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Soft_Delete_IUD_Same_Batch() { srcTableName := fmt.Sprintf("%s_src", cmpTableName) dstTableName := "test_softdel_iud" - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id INT PRIMARY KEY GENERATED ALWAYS AS IDENTITY, c1 INT, @@ -1529,7 +1529,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Soft_Delete_IUD_Same_Batch() { go func() { e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - insertTx, err := s.pool.Begin(context.Background()) + insertTx, err := s.conn.Begin(context.Background()) e2e.EnvNoError(s.t, env, err) _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` @@ -1573,7 +1573,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Soft_Delete_UD_Same_Batch() { srcTableName := s.attachSchemaSuffix(srcName) dstName := "test_softdel_ud" - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id INT PRIMARY KEY GENERATED ALWAYS AS IDENTITY, c1 INT, @@ -1613,12 +1613,12 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Soft_Delete_UD_Same_Batch() { go func() { e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - _, err = s.pool.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") - insertTx, err := s.pool.Begin(context.Background()) + 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)) @@ -1632,7 +1632,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Soft_Delete_UD_Same_Batch() { 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.pool, s.bqSuffix, srcName, "id,c1,c2,t") + 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 { @@ -1661,7 +1661,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Soft_Delete_Insert_After_Delete() { tableName := "test_softdel_iad" srcTableName := s.attachSchemaSuffix(tableName) - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id INT PRIMARY KEY GENERATED BY DEFAULT AS IDENTITY, c1 INT, @@ -1701,15 +1701,15 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Soft_Delete_Insert_After_Delete() { go func() { e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - _, err = s.pool.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.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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.pool, s.bqSuffix, tableName, "id,c1,c2,t") + pgRows, err := e2e.GetPgRows(s.conn, s.bqSuffix, tableName, "id,c1,c2,t") if err != nil { return false } @@ -1719,7 +1719,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Soft_Delete_Insert_After_Delete() { } return e2eshared.CheckEqualRecordBatches(s.t, pgRows, rows) }) - _, err = s.pool.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") diff --git a/flow/e2e/bigquery/qrep_flow_bq_test.go b/flow/e2e/bigquery/qrep_flow_bq_test.go index 4302b222e5..d8c3385510 100644 --- a/flow/e2e/bigquery/qrep_flow_bq_test.go +++ b/flow/e2e/bigquery/qrep_flow_bq_test.go @@ -9,9 +9,9 @@ import ( ) func (s PeerFlowE2ETestSuiteBQ) setupSourceTable(tableName string, rowCount int) { - err := e2e.CreateTableForQRep(s.pool, s.bqSuffix, tableName) + err := e2e.CreateTableForQRep(s.conn, s.bqSuffix, tableName) require.NoError(s.t, err) - err = e2e.PopulateSourceTable(s.pool, s.bqSuffix, tableName, rowCount) + err = e2e.PopulateSourceTable(s.conn, s.bqSuffix, tableName, rowCount) require.NoError(s.t, err) } diff --git a/flow/e2e/congen.go b/flow/e2e/congen.go index cb3c35bc29..c6b02138d1 100644 --- a/flow/e2e/congen.go +++ b/flow/e2e/congen.go @@ -7,8 +7,6 @@ import ( "time" "github.com/jackc/pgx/v5" - "github.com/jackc/pgx/v5/pgtype" - "github.com/jackc/pgx/v5/pgxpool" "github.com/stretchr/testify/require" "github.com/PeerDB-io/peer-flow/connectors/utils" @@ -34,15 +32,15 @@ func GetTestPostgresConf() *protos.PostgresConfig { } } -func cleanPostgres(pool *pgxpool.Pool, suffix string) error { +func cleanPostgres(conn *pgx.Conn, suffix string) error { // drop the e2e_test schema with the given suffix if it exists - _, err := pool.Exec(context.Background(), fmt.Sprintf("DROP SCHEMA IF EXISTS e2e_test_%s CASCADE", suffix)) + _, err := conn.Exec(context.Background(), fmt.Sprintf("DROP SCHEMA IF EXISTS e2e_test_%s CASCADE", suffix)) if err != nil { return fmt.Errorf("failed to drop e2e_test schema: %w", err) } // drop all open slots with the given suffix - _, err = pool.Exec( + _, err = conn.Exec( context.Background(), "SELECT pg_drop_replication_slot(slot_name) FROM pg_replication_slots WHERE slot_name LIKE $1", fmt.Sprintf("%%_%s", suffix), @@ -52,52 +50,38 @@ func cleanPostgres(pool *pgxpool.Pool, suffix string) error { } // list all publications from pg_publication table - rows, err := pool.Query(context.Background(), + rows, err := conn.Query(context.Background(), "SELECT pubname FROM pg_publication WHERE pubname LIKE $1", fmt.Sprintf("%%_%s", suffix), ) if err != nil { return fmt.Errorf("failed to list publications: %w", err) } + publications, err := pgx.CollectRows[string](rows, pgx.RowTo) + if err != nil { + return fmt.Errorf("failed to read publications: %w", err) + } - // drop all publications with the given suffix - for rows.Next() { - var pubName pgtype.Text - err = rows.Scan(&pubName) - if err != nil { - return fmt.Errorf("failed to scan publication name: %w", err) - } - - _, err = pool.Exec(context.Background(), fmt.Sprintf("DROP PUBLICATION %s", pubName.String)) + for _, pubName := range publications { + _, err = conn.Exec(context.Background(), fmt.Sprintf("DROP PUBLICATION %s", pubName)) if err != nil { - return fmt.Errorf("failed to drop publication %s: %w", pubName.String, err) + return fmt.Errorf("failed to drop publication %s: %w", pubName, err) } } return nil } -// setupPostgres sets up the postgres connection pool. -func SetupPostgres(suffix string) (*pgxpool.Pool, error) { - pool, err := pgxpool.New(context.Background(), utils.GetPGConnectionString(GetTestPostgresConf())) - if err != nil { - return nil, fmt.Errorf("failed to create postgres connection pool: %w", err) - } - - err = cleanPostgres(pool, suffix) - if err != nil { - return nil, err - } - - setupTx, err := pool.Begin(context.Background()) +func setupPostgresSchema(conn *pgx.Conn, suffix string) error { + setupTx, err := conn.Begin(context.Background()) if err != nil { - return nil, fmt.Errorf("failed to start setup transaction") + return fmt.Errorf("failed to start setup transaction") } // create an e2e_test schema _, err = setupTx.Exec(context.Background(), "SELECT pg_advisory_xact_lock(hashtext('Megaton Mile'))") if err != nil { - return nil, fmt.Errorf("failed to get lock: %w", err) + return fmt.Errorf("failed to get lock: %w", err) } defer func() { deferErr := setupTx.Rollback(context.Background()) @@ -109,7 +93,7 @@ func SetupPostgres(suffix string) (*pgxpool.Pool, error) { // create an e2e_test schema _, err = setupTx.Exec(context.Background(), fmt.Sprintf("CREATE SCHEMA e2e_test_%s", suffix)) if err != nil { - return nil, fmt.Errorf("failed to create e2e_test schema: %w", err) + return fmt.Errorf("failed to create e2e_test schema: %w", err) } _, err = setupTx.Exec(context.Background(), ` @@ -127,33 +111,50 @@ func SetupPostgres(suffix string) (*pgxpool.Pool, error) { SET search_path = 'pg_catalog'; `) if err != nil { - return nil, fmt.Errorf("failed to create utility functions: %w", err) + return fmt.Errorf("failed to create utility functions: %w", err) + } + + return setupTx.Commit(context.Background()) +} + +// setupPostgres sets up the postgres connection. +func SetupPostgres(suffix string) (*pgx.Conn, error) { + conn, err := pgx.Connect(context.Background(), utils.GetPGConnectionString(GetTestPostgresConf())) + if err != nil { + return nil, fmt.Errorf("failed to create postgres connection: %w", err) } - err = setupTx.Commit(context.Background()) + err = cleanPostgres(conn, suffix) if err != nil { - return nil, fmt.Errorf("error committing setup transaction: %w", err) + conn.Close(context.Background()) + return nil, err + } + + err = setupPostgresSchema(conn, suffix) + if err != nil { + conn.Close(context.Background()) + return nil, err } - return pool, nil + return conn, nil } func TearDownPostgres[T e2eshared.Suite](s T) { t := s.T() t.Helper() - pool := s.Pool() - suffix := s.Suffix() - if pool != nil { + conn := s.Conn() + if conn != nil { + suffix := s.Suffix() t.Log("begin tearing down postgres schema", suffix) deadline := time.Now().Add(2 * time.Minute) for { - err := cleanPostgres(pool, suffix) + err := cleanPostgres(conn, suffix) if err == nil { - pool.Close() + conn.Close(context.Background()) return } else if time.Now().After(deadline) { - require.Fail(t, "failed to teardown postgres schema", suffix) + require.Fail(t, "failed to teardown postgres schema", "%s: %v", suffix, err) } time.Sleep(time.Second) } diff --git a/flow/e2e/postgres/peer_flow_pg_test.go b/flow/e2e/postgres/peer_flow_pg_test.go index 27be975acb..67161a5f29 100644 --- a/flow/e2e/postgres/peer_flow_pg_test.go +++ b/flow/e2e/postgres/peer_flow_pg_test.go @@ -33,7 +33,7 @@ func (s PeerFlowE2ETestSuitePG) checkPeerdbColumns(dstSchemaQualified string, ro dstSchemaQualified, rowID) var isDeleted pgtype.Bool var syncedAt pgtype.Timestamp - err := s.pool.QueryRow(context.Background(), query).Scan(&isDeleted, &syncedAt) + err := s.conn.QueryRow(context.Background(), query).Scan(&isDeleted, &syncedAt) if err != nil { return fmt.Errorf("failed to query row: %w", err) } @@ -82,7 +82,7 @@ func (s PeerFlowE2ETestSuitePG) Test_Simple_Flow_PG() { srcTableName := s.attachSchemaSuffix("test_simple_flow") dstTableName := s.attachSchemaSuffix("test_simple_flow_dst") - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id SERIAL PRIMARY KEY, key TEXT NOT NULL, @@ -114,7 +114,7 @@ func (s PeerFlowE2ETestSuitePG) Test_Simple_Flow_PG() { for i := 0; i < 10; i++ { testKey := fmt.Sprintf("test_key_%d", i) testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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) @@ -142,7 +142,7 @@ func (s PeerFlowE2ETestSuitePG) Test_Geospatial_PG() { srcTableName := s.attachSchemaSuffix("test_geospatial_pg") dstTableName := s.attachSchemaSuffix("test_geospatial_pg_dst") - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id SERIAL PRIMARY KEY, gg geography NOT NULL, @@ -168,7 +168,7 @@ func (s PeerFlowE2ETestSuitePG) Test_Geospatial_PG() { go func() { e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) // insert 1 row into the source table - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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) @@ -196,7 +196,7 @@ func (s PeerFlowE2ETestSuitePG) Test_Types_PG() { srcTableName := s.attachSchemaSuffix("test_types_pg") dstTableName := s.attachSchemaSuffix("test_types_pg_dst") - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s (id serial PRIMARY KEY,c1 BIGINT,c2 BIT,c4 BOOLEAN, c7 CHARACTER,c8 varchar,c9 CIDR,c11 DATE,c12 FLOAT,c13 DOUBLE PRECISION, c14 INET,c15 INTEGER,c21 MACADDR, @@ -223,7 +223,7 @@ func (s PeerFlowE2ETestSuitePG) Test_Types_PG() { go func() { e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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, @@ -271,11 +271,11 @@ func (s PeerFlowE2ETestSuitePG) Test_Enums_PG() { dstTableName := s.attachSchemaSuffix("test_enum_flow_dst") createMoodEnum := "CREATE TYPE mood AS ENUM ('happy', 'sad', 'angry');" var pgErr *pgconn.PgError - _, enumErr := s.pool.Exec(context.Background(), createMoodEnum) + _, enumErr := s.conn.Exec(context.Background(), createMoodEnum) if errors.As(enumErr, &pgErr) && pgErr.Code != pgerrcode.DuplicateObject && !utils.IsUniqueError(enumErr) { require.NoError(s.t, enumErr) } - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id SERIAL PRIMARY KEY, my_mood mood, @@ -300,7 +300,7 @@ func (s PeerFlowE2ETestSuitePG) Test_Enums_PG() { go func() { e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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) @@ -327,7 +327,7 @@ func (s PeerFlowE2ETestSuitePG) Test_Simple_Schema_Changes_PG() { srcTableName := s.attachSchemaSuffix("test_simple_schema_changes") dstTableName := s.attachSchemaSuffix("test_simple_schema_changes_dst") - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id BIGINT PRIMARY KEY GENERATED ALWAYS AS IDENTITY, c1 BIGINT @@ -354,7 +354,7 @@ func (s PeerFlowE2ETestSuitePG) Test_Simple_Schema_Changes_PG() { go func() { // insert first row. e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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") @@ -371,11 +371,11 @@ func (s PeerFlowE2ETestSuitePG) Test_Simple_Schema_Changes_PG() { }) // alter source table, add column c2 and insert another row. - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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") @@ -393,11 +393,11 @@ func (s PeerFlowE2ETestSuitePG) Test_Simple_Schema_Changes_PG() { }) // alter source table, add column c3, drop column c2 and insert another row. - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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") @@ -416,11 +416,11 @@ func (s PeerFlowE2ETestSuitePG) Test_Simple_Schema_Changes_PG() { }) // alter source table, drop column c3 and insert another row. - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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") @@ -451,7 +451,7 @@ func (s PeerFlowE2ETestSuitePG) Test_Composite_PKey_PG() { srcTableName := s.attachSchemaSuffix("test_simple_cpkey") dstTableName := s.attachSchemaSuffix("test_simple_cpkey_dst") - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id INT GENERATED ALWAYS AS IDENTITY, c1 INT GENERATED BY DEFAULT AS IDENTITY, @@ -483,7 +483,7 @@ func (s PeerFlowE2ETestSuitePG) Test_Composite_PKey_PG() { // insert 10 rows into the source table for i := 0; i < 10; i++ { testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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) @@ -494,10 +494,10 @@ func (s PeerFlowE2ETestSuitePG) Test_Composite_PKey_PG() { return s.comparePGTables(srcTableName, dstTableName, "id,c1,c2,t") == nil }) - _, err := s.pool.Exec(context.Background(), + _, 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.pool.Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=$1`, srcTableName), 0) + _, 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 @@ -516,7 +516,7 @@ func (s PeerFlowE2ETestSuitePG) Test_Composite_PKey_Toast_1_PG() { randomString := s.attachSchemaSuffix("random_string") dstTableName := s.attachSchemaSuffix("test_cpkey_toast1_dst") - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id INT GENERATED ALWAYS AS IDENTITY, c1 INT GENERATED BY DEFAULT AS IDENTITY, @@ -549,7 +549,7 @@ func (s PeerFlowE2ETestSuitePG) Test_Composite_PKey_Toast_1_PG() { // and then insert, update and delete rows in the table. go func() { e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - rowsTx, err := s.pool.Begin(context.Background()) + rowsTx, err := s.conn.Begin(context.Background()) e2e.EnvNoError(s.t, env, err) // insert 10 rows into the source table @@ -594,7 +594,7 @@ func (s PeerFlowE2ETestSuitePG) Test_Composite_PKey_Toast_2_PG() { randomString := s.attachSchemaSuffix("random_string") dstTableName := s.attachSchemaSuffix("test_cpkey_toast2_dst") - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id INT GENERATED ALWAYS AS IDENTITY, c1 INT GENERATED BY DEFAULT AS IDENTITY, @@ -631,7 +631,7 @@ func (s PeerFlowE2ETestSuitePG) Test_Composite_PKey_Toast_2_PG() { // insert 10 rows into the source table for i := 0; i < 10; i++ { testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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) @@ -641,10 +641,10 @@ func (s PeerFlowE2ETestSuitePG) Test_Composite_PKey_Toast_2_PG() { 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.pool.Exec(context.Background(), + _, 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.pool.Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=$1`, srcTableName), 0) + _, 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 { @@ -664,7 +664,7 @@ func (s PeerFlowE2ETestSuitePG) Test_PeerDB_Columns() { srcTableName := s.attachSchemaSuffix("test_peerdb_cols") dstTableName := s.attachSchemaSuffix("test_peerdb_cols_dst") - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id SERIAL PRIMARY KEY, key TEXT NOT NULL, @@ -693,13 +693,13 @@ func (s PeerFlowE2ETestSuitePG) Test_PeerDB_Columns() { // insert 1 row into the source table testKey := fmt.Sprintf("test_key_%d", 1) testValue := fmt.Sprintf("test_value_%d", 1) - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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) // delete that row - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err = s.conn.Exec(context.Background(), fmt.Sprintf(` DELETE FROM %s WHERE id=1 `, srcTableName)) e2e.EnvNoError(s.t, env, err) @@ -726,7 +726,7 @@ func (s PeerFlowE2ETestSuitePG) Test_Soft_Delete_Basic() { srcTableName := fmt.Sprintf("%s_src", cmpTableName) dstTableName := s.attachSchemaSuffix("test_softdel_dst") - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id INT PRIMARY KEY GENERATED ALWAYS AS IDENTITY, c1 INT, @@ -766,23 +766,23 @@ func (s PeerFlowE2ETestSuitePG) Test_Soft_Delete_Basic() { go func() { e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - _, err = s.pool.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.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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.pool.Exec(context.Background(), fmt.Sprintf(` + _, err = s.conn.Exec(context.Background(), fmt.Sprintf(` DELETE FROM %s WHERE id=1`, srcTableName)) e2e.EnvNoError(s.t, env, err) @@ -816,7 +816,7 @@ func (s PeerFlowE2ETestSuitePG) Test_Soft_Delete_IUD_Same_Batch() { srcTableName := fmt.Sprintf("%s_src", cmpTableName) dstTableName := s.attachSchemaSuffix("test_softdel_iud_dst") - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id INT PRIMARY KEY GENERATED ALWAYS AS IDENTITY, c1 INT, @@ -856,7 +856,7 @@ func (s PeerFlowE2ETestSuitePG) Test_Soft_Delete_IUD_Same_Batch() { go func() { e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - insertTx, err := s.pool.Begin(context.Background()) + insertTx, err := s.conn.Begin(context.Background()) e2e.EnvNoError(s.t, env, err) _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` @@ -901,7 +901,7 @@ func (s PeerFlowE2ETestSuitePG) Test_Soft_Delete_UD_Same_Batch() { srcTableName := fmt.Sprintf("%s_src", cmpTableName) dstTableName := s.attachSchemaSuffix("test_softdel_ud_dst") - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id INT PRIMARY KEY GENERATED ALWAYS AS IDENTITY, c1 INT, @@ -941,14 +941,14 @@ func (s PeerFlowE2ETestSuitePG) Test_Soft_Delete_UD_Same_Batch() { go func() { e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - _, err = s.pool.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 }) - insertTx, err := s.pool.Begin(context.Background()) + 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)) @@ -988,7 +988,7 @@ func (s PeerFlowE2ETestSuitePG) Test_Soft_Delete_Insert_After_Delete() { srcTableName := s.attachSchemaSuffix("test_softdel_iad") dstTableName := s.attachSchemaSuffix("test_softdel_iad_dst") - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id INT PRIMARY KEY GENERATED BY DEFAULT AS IDENTITY, c1 INT, @@ -1028,19 +1028,19 @@ func (s PeerFlowE2ETestSuitePG) Test_Soft_Delete_Insert_After_Delete() { go func() { e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - _, err = s.pool.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.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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.pool.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.EnvWaitFor(s.t, env, 3*time.Minute, "normalize reinsert", func() bool { diff --git a/flow/e2e/postgres/qrep_flow_pg_test.go b/flow/e2e/postgres/qrep_flow_pg_test.go index dcb32c7fd9..9bcca843ad 100644 --- a/flow/e2e/postgres/qrep_flow_pg_test.go +++ b/flow/e2e/postgres/qrep_flow_pg_test.go @@ -10,7 +10,6 @@ import ( "github.com/jackc/pgx/v5" "github.com/jackc/pgx/v5/pgtype" - "github.com/jackc/pgx/v5/pgxpool" "github.com/joho/godotenv" "github.com/stretchr/testify/require" @@ -24,7 +23,7 @@ import ( type PeerFlowE2ETestSuitePG struct { t *testing.T - pool *pgxpool.Pool + conn *pgx.Conn peer *protos.Peer connector *connpostgres.PostgresConnector suffix string @@ -34,8 +33,8 @@ func (s PeerFlowE2ETestSuitePG) T() *testing.T { return s.t } -func (s PeerFlowE2ETestSuitePG) Pool() *pgxpool.Pool { - return s.pool +func (s PeerFlowE2ETestSuitePG) Conn() *pgx.Conn { + return s.conn } func (s PeerFlowE2ETestSuitePG) Suffix() string { @@ -59,13 +58,12 @@ func SetupSuite(t *testing.T) PeerFlowE2ETestSuitePG { } suffix := "pg_" + strings.ToLower(shared.RandomString(8)) - pool, err := e2e.SetupPostgres(suffix) + conn, err := e2e.SetupPostgres(suffix) if err != nil { require.Fail(t, "failed to setup postgres", err) } - var connector *connpostgres.PostgresConnector - connector, err = connpostgres.NewPostgresConnector(context.Background(), + connector, err := connpostgres.NewPostgresConnector(context.Background(), &protos.PostgresConfig{ Host: "localhost", Port: 7132, @@ -77,7 +75,7 @@ func SetupSuite(t *testing.T) PeerFlowE2ETestSuitePG { return PeerFlowE2ETestSuitePG{ t: t, - pool: pool, + conn: conn, peer: generatePGPeer(e2e.GetTestPostgresConf()), connector: connector, suffix: suffix, @@ -85,9 +83,9 @@ func SetupSuite(t *testing.T) PeerFlowE2ETestSuitePG { } func (s PeerFlowE2ETestSuitePG) setupSourceTable(tableName string, rowCount int) { - err := e2e.CreateTableForQRep(s.pool, s.suffix, tableName) + err := e2e.CreateTableForQRep(s.conn, s.suffix, tableName) require.NoError(s.t, err) - err = e2e.PopulateSourceTable(s.pool, s.suffix, tableName, rowCount) + err = e2e.PopulateSourceTable(s.conn, s.suffix, tableName, rowCount) require.NoError(s.t, err) } @@ -111,7 +109,7 @@ func (s PeerFlowE2ETestSuitePG) checkEnums(srcSchemaQualified, dstSchemaQualifie "SELECT 1 FROM %s dst "+ "WHERE src.my_mood::text = dst.my_mood::text)) LIMIT 1;", srcSchemaQualified, dstSchemaQualified) - err := s.pool.QueryRow(context.Background(), query).Scan(&exists) + err := s.conn.QueryRow(context.Background(), query).Scan(&exists) if err != nil { return err } @@ -125,7 +123,7 @@ func (s PeerFlowE2ETestSuitePG) checkEnums(srcSchemaQualified, dstSchemaQualifie func (s PeerFlowE2ETestSuitePG) compareQuery(srcSchemaQualified, dstSchemaQualified, selector string) error { query := fmt.Sprintf("SELECT %s FROM %s EXCEPT SELECT %s FROM %s", selector, srcSchemaQualified, selector, dstSchemaQualified) - rows, err := s.pool.Query(context.Background(), query, pgx.QueryExecModeExec) + rows, err := s.conn.Query(context.Background(), query, pgx.QueryExecModeExec) if err != nil { return err } @@ -159,7 +157,7 @@ func (s PeerFlowE2ETestSuitePG) compareQuery(srcSchemaQualified, dstSchemaQualif func (s PeerFlowE2ETestSuitePG) checkSyncedAt(dstSchemaQualified string) error { query := fmt.Sprintf(`SELECT "_PEERDB_SYNCED_AT" FROM %s`, dstSchemaQualified) - rows, _ := s.pool.Query(context.Background(), query) + rows, _ := s.conn.Query(context.Background(), query) defer rows.Close() for rows.Next() { @@ -179,12 +177,12 @@ func (s PeerFlowE2ETestSuitePG) checkSyncedAt(dstSchemaQualified string) error { func (s PeerFlowE2ETestSuitePG) RunInt64Query(query string) (int64, error) { var count pgtype.Int8 - err := s.pool.QueryRow(context.Background(), query).Scan(&count) + err := s.conn.QueryRow(context.Background(), query).Scan(&count) return count.Int64, err } func (s PeerFlowE2ETestSuitePG) TestSimpleSlotCreation() { - setupTx, err := s.pool.Begin(context.Background()) + setupTx, err := s.conn.Begin(context.Background()) require.NoError(s.t, err) // setup 3 tables in pgpeer_repl_test schema // test_1, test_2, test_3, all have 5 columns all text, c1, c2, c3, c4, c5 @@ -237,7 +235,7 @@ func (s PeerFlowE2ETestSuitePG) Test_Complete_QRep_Flow_Multi_Insert_PG() { dstTable := "test_qrep_flow_avro_pg_2" - err := e2e.CreateTableForQRep(s.pool, s.suffix, dstTable) + err := e2e.CreateTableForQRep(s.conn, s.suffix, dstTable) require.NoError(s.t, err) srcSchemaQualified := fmt.Sprintf("%s_%s.%s", "e2e_test", s.suffix, srcTable) diff --git a/flow/e2e/s3/cdc_s3_test.go b/flow/e2e/s3/cdc_s3_test.go index 69c26b6809..98f6c6b208 100644 --- a/flow/e2e/s3/cdc_s3_test.go +++ b/flow/e2e/s3/cdc_s3_test.go @@ -26,7 +26,7 @@ func (s PeerFlowE2ETestSuiteS3) Test_Complete_Simple_Flow_S3() { srcTableName := s.attachSchemaSuffix("test_simple_flow_s3") dstTableName := fmt.Sprintf("%s.%s", "peerdb_test_s3", "test_simple_flow_s3") flowJobName := s.attachSuffix("test_simple_flow_s3") - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE %s ( id SERIAL PRIMARY KEY, key TEXT NOT NULL, @@ -54,7 +54,7 @@ func (s PeerFlowE2ETestSuiteS3) Test_Complete_Simple_Flow_S3() { for i := 1; i <= 20; i++ { testKey := fmt.Sprintf("test_key_%d", i) testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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) diff --git a/flow/e2e/s3/qrep_flow_s3_test.go b/flow/e2e/s3/qrep_flow_s3_test.go index 919994b570..41259e7d18 100644 --- a/flow/e2e/s3/qrep_flow_s3_test.go +++ b/flow/e2e/s3/qrep_flow_s3_test.go @@ -8,7 +8,7 @@ import ( "testing" "time" - "github.com/jackc/pgx/v5/pgxpool" + "github.com/jackc/pgx/v5" "github.com/joho/godotenv" "github.com/stretchr/testify/require" @@ -20,7 +20,7 @@ import ( type PeerFlowE2ETestSuiteS3 struct { t *testing.T - pool *pgxpool.Pool + conn *pgx.Conn s3Helper *S3TestHelper suffix string } @@ -29,8 +29,8 @@ func (s PeerFlowE2ETestSuiteS3) T() *testing.T { return s.t } -func (s PeerFlowE2ETestSuiteS3) Pool() *pgxpool.Pool { - return s.pool +func (s PeerFlowE2ETestSuiteS3) Conn() *pgx.Conn { + return s.conn } func (s PeerFlowE2ETestSuiteS3) Suffix() string { @@ -55,9 +55,9 @@ func TestPeerFlowE2ETestSuiteGCS(t *testing.T) { } func (s PeerFlowE2ETestSuiteS3) setupSourceTable(tableName string, rowCount int) { - err := e2e.CreateTableForQRep(s.pool, s.suffix, tableName) + err := e2e.CreateTableForQRep(s.conn, s.suffix, tableName) require.NoError(s.t, err) - err = e2e.PopulateSourceTable(s.pool, s.suffix, tableName, rowCount) + err = e2e.PopulateSourceTable(s.conn, s.suffix, tableName, rowCount) require.NoError(s.t, err) } @@ -72,8 +72,8 @@ func setupSuite(t *testing.T, gcs bool) PeerFlowE2ETestSuiteS3 { } suffix := "s3_" + strings.ToLower(shared.RandomString(8)) - pool, err := e2e.SetupPostgres(suffix) - if err != nil || pool == nil { + conn, err := e2e.SetupPostgres(suffix) + if err != nil || conn == nil { require.Fail(t, "failed to setup postgres", err) } @@ -84,7 +84,7 @@ func setupSuite(t *testing.T, gcs bool) PeerFlowE2ETestSuiteS3 { return PeerFlowE2ETestSuiteS3{ t: t, - pool: pool, + conn: conn, s3Helper: helper, suffix: suffix, } diff --git a/flow/e2e/snowflake/peer_flow_sf_test.go b/flow/e2e/snowflake/peer_flow_sf_test.go index 8f3cf46e7a..cdf579bd0a 100644 --- a/flow/e2e/snowflake/peer_flow_sf_test.go +++ b/flow/e2e/snowflake/peer_flow_sf_test.go @@ -11,8 +11,8 @@ import ( "time" "github.com/jackc/pgerrcode" + "github.com/jackc/pgx/v5" "github.com/jackc/pgx/v5/pgconn" - "github.com/jackc/pgx/v5/pgxpool" "github.com/joho/godotenv" "github.com/stretchr/testify/require" @@ -31,7 +31,7 @@ type PeerFlowE2ETestSuiteSF struct { t *testing.T pgSuffix string - pool *pgxpool.Pool + conn *pgx.Conn sfHelper *SnowflakeTestHelper connector *connsnowflake.SnowflakeConnector } @@ -40,8 +40,8 @@ func (s PeerFlowE2ETestSuiteSF) T() *testing.T { return s.t } -func (s PeerFlowE2ETestSuiteSF) Pool() *pgxpool.Pool { - return s.pool +func (s PeerFlowE2ETestSuiteSF) Conn() *pgx.Conn { + return s.conn } func (s PeerFlowE2ETestSuiteSF) Suffix() string { @@ -98,8 +98,8 @@ func SetupSuite(t *testing.T) PeerFlowE2ETestSuiteSF { tsSuffix := time.Now().Format("20060102150405") pgSuffix := fmt.Sprintf("sf_%s_%s", strings.ToLower(suffix), tsSuffix) - pool, err := e2e.SetupPostgres(pgSuffix) - if err != nil || pool == nil { + conn, err := e2e.SetupPostgres(pgSuffix) + if err != nil || conn == nil { slog.Error("failed to setup Postgres", slog.Any("error", err)) t.FailNow() } @@ -119,7 +119,7 @@ func SetupSuite(t *testing.T) PeerFlowE2ETestSuiteSF { suite := PeerFlowE2ETestSuiteSF{ t: t, pgSuffix: pgSuffix, - pool: pool, + conn: conn, sfHelper: sfHelper, connector: connector, } @@ -135,7 +135,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Complete_Simple_Flow_SF() { srcTableName := s.attachSchemaSuffix(tableName) dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, tableName) - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id SERIAL PRIMARY KEY, key TEXT NOT NULL, @@ -166,7 +166,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Complete_Simple_Flow_SF() { for i := 0; i < 20; i++ { testKey := fmt.Sprintf("test_key_%d", i) testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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) @@ -198,7 +198,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Flow_ReplicaIdentity_Index_No_Pkey() { dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_replica_identity_no_pkey") // Create a table without a primary key and create a named unique index - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id SERIAL, key TEXT NOT NULL, @@ -231,7 +231,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Flow_ReplicaIdentity_Index_No_Pkey() { for i := 0; i < 20; i++ { testKey := fmt.Sprintf("test_key_%d", i) testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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) @@ -261,7 +261,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Invalid_Geo_SF_Avro_CDC() { srcTableName := s.attachSchemaSuffix(tableName) dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_invalid_geo_sf_avro_cdc") - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id SERIAL PRIMARY KEY, line GEOMETRY(LINESTRING) NOT NULL, @@ -290,7 +290,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Invalid_Geo_SF_Avro_CDC() { e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) // insert 4 invalid shapes and 6 valid shapes into the source table for i := 0; i < 4; i++ { - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err = s.conn.Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s (line,poly) VALUES ($1,$2) `, srcTableName), "010200000001000000000000000000F03F0000000000000040", "0103000020e6100000010000000c0000001a8361d35dc64140afdb8d2b1bc3c9bf1b8ed4685fc641405ba64c"+ @@ -302,7 +302,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Invalid_Geo_SF_Avro_CDC() { } s.t.Log("Inserted 4 invalid geography rows into the source table") for i := 4; i < 10; i++ { - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err = s.conn.Exec(context.Background(), fmt.Sprintf(` INSERT INTO %s (line,poly) VALUES ($1,$2) `, srcTableName), "010200000002000000000000000000F03F000000000000004000000000000008400000000000001040", "010300000001000000050000000000000000000000000000000000000000000000"+ @@ -340,7 +340,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Toast_SF() { srcTableName := s.attachSchemaSuffix("test_toast_sf_1") dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_1") - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id SERIAL PRIMARY KEY, t1 text, @@ -374,7 +374,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Toast_SF() { 2. changes no toast column 2. changes 1 toast column */ - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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); @@ -405,7 +405,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Toast_Nochanges_SF() { srcTableName := s.attachSchemaSuffix("test_toast_sf_2") dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_2") - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id SERIAL PRIMARY KEY, t1 text, @@ -437,7 +437,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Toast_Nochanges_SF() { defer wg.Done() e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) /* transaction updating no rows */ - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err = s.conn.Exec(context.Background(), fmt.Sprintf(` BEGIN; UPDATE %s SET k=102 WHERE id=1; UPDATE %s SET t1='dummy' WHERE id=2; @@ -466,7 +466,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Toast_Advance_1_SF() { srcTableName := s.attachSchemaSuffix("test_toast_sf_3") dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_3") - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id SERIAL PRIMARY KEY, t1 text, @@ -495,7 +495,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Toast_Advance_1_SF() { go func() { e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) // complex transaction with random DMLs on a table with toast columns - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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); @@ -537,7 +537,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Toast_Advance_2_SF() { srcTableName := s.attachSchemaSuffix("test_toast_sf_4") dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_4") - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id SERIAL PRIMARY KEY, t1 text, @@ -565,7 +565,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Toast_Advance_2_SF() { go func() { e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) // complex transaction with random DMLs on a table with toast columns - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err = s.conn.Exec(context.Background(), fmt.Sprintf(` BEGIN; INSERT INTO %s (t1,k) SELECT random_string(9000), 1 FROM generate_series(1,1); @@ -601,7 +601,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Toast_Advance_3_SF() { srcTableName := s.attachSchemaSuffix("test_toast_sf_5") dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_5") - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id SERIAL PRIMARY KEY, t1 text, @@ -633,7 +633,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Toast_Advance_3_SF() { transaction updating a single row multiple times with changed/unchanged toast columns */ - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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); @@ -666,11 +666,11 @@ func (s PeerFlowE2ETestSuiteSF) Test_Types_SF() { dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_types_sf") createMoodEnum := "CREATE TYPE mood AS ENUM ('happy', 'sad', 'angry');" var pgErr *pgconn.PgError - _, enumErr := s.pool.Exec(context.Background(), createMoodEnum) + _, enumErr := s.conn.Exec(context.Background(), createMoodEnum) if errors.As(enumErr, &pgErr) && pgErr.Code != pgerrcode.DuplicateObject && !utils.IsUniqueError(pgErr) { require.NoError(s.t, enumErr) } - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s (id serial PRIMARY KEY,c1 BIGINT,c2 BIT,c3 VARBIT,c4 BOOLEAN, c6 BYTEA,c7 CHARACTER,c8 varchar,c9 CIDR,c11 DATE,c12 FLOAT,c13 DOUBLE PRECISION, c14 INET,c15 INTEGER,c16 INTERVAL,c17 JSON,c18 JSONB,c21 MACADDR,c22 MONEY, @@ -701,7 +701,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Types_SF() { go func() { e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) /* test inserting various types*/ - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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, @@ -767,7 +767,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Multi_Table_SF() { dstTable1Name := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test1_sf") dstTable2Name := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test2_sf") - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s (id serial primary key, c1 int, c2 text); CREATE TABLE IF NOT EXISTS %s (id serial primary key, c1 int, c2 text); `, srcTable1Name, srcTable2Name)) @@ -792,7 +792,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Multi_Table_SF() { go func() { e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) /* inserting across multiple tables*/ - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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)) @@ -821,7 +821,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Simple_Schema_Changes_SF() { srcTableName := s.attachSchemaSuffix("test_simple_schema_changes") dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_simple_schema_changes") - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id BIGINT PRIMARY KEY GENERATED ALWAYS AS IDENTITY, c1 BIGINT @@ -847,7 +847,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Simple_Schema_Changes_SF() { // and then insert and mutate schema repeatedly. go func() { e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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") @@ -876,11 +876,11 @@ func (s PeerFlowE2ETestSuiteSF) Test_Simple_Schema_Changes_SF() { e2e.EnvTrue(s.t, env, e2e.CompareTableSchemas(expectedTableSchema, output.TableNameSchemaMapping[dstTableName])) // alter source table, add column c2 and insert another row. - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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") @@ -910,11 +910,11 @@ func (s PeerFlowE2ETestSuiteSF) Test_Simple_Schema_Changes_SF() { 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.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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") @@ -946,11 +946,11 @@ func (s PeerFlowE2ETestSuiteSF) Test_Simple_Schema_Changes_SF() { e2e.EnvEqualTables(env, s, "test_simple_schema_changes", "id,c1,c3") // alter source table, drop column c3 and insert another row. - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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") @@ -994,7 +994,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Composite_PKey_SF() { srcTableName := s.attachSchemaSuffix("test_simple_cpkey") dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_simple_cpkey") - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id INT GENERATED ALWAYS AS IDENTITY, c1 INT GENERATED BY DEFAULT AS IDENTITY, @@ -1026,7 +1026,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Composite_PKey_SF() { // insert 10 rows into the source table for i := 0; i < 10; i++ { testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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) @@ -1035,10 +1035,10 @@ func (s PeerFlowE2ETestSuiteSF) Test_Composite_PKey_SF() { e2e.EnvWaitForEqualTables(env, s, "normalize table", "test_simple_cpkey", "id,c1,c2,t") - _, err := s.pool.Exec(context.Background(), + _, 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.pool.Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=$1`, srcTableName), 0) + _, 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") @@ -1055,7 +1055,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Composite_PKey_Toast_1_SF() { srcTableName := s.attachSchemaSuffix("test_cpkey_toast1") dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_cpkey_toast1") - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id INT GENERATED ALWAYS AS IDENTITY, c1 INT GENERATED BY DEFAULT AS IDENTITY, @@ -1085,7 +1085,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Composite_PKey_Toast_1_SF() { // and then insert, update and delete rows in the table. go func() { e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - rowsTx, err := s.pool.Begin(context.Background()) + rowsTx, err := s.conn.Begin(context.Background()) e2e.EnvNoError(s.t, env, err) // insert 10 rows into the source table @@ -1129,7 +1129,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Composite_PKey_Toast_2_SF() { srcTableName := s.attachSchemaSuffix(tableName) dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, tableName) - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id INT GENERATED ALWAYS AS IDENTITY, c1 INT GENERATED BY DEFAULT AS IDENTITY, @@ -1163,7 +1163,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Composite_PKey_Toast_2_SF() { // insert 10 rows into the source table for i := 0; i < 10; i++ { testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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) @@ -1171,10 +1171,10 @@ func (s PeerFlowE2ETestSuiteSF) Test_Composite_PKey_Toast_2_SF() { s.t.Log("Inserted 10 rows into the source table") e2e.EnvWaitForEqualTables(env, s, "normalize table", tableName, "id,c2,t,t2") - _, err = s.pool.Exec(context.Background(), + _, 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.pool.Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=$1`, srcTableName), 0) + _, 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") @@ -1192,7 +1192,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Column_Exclusion() { srcTableName := s.attachSchemaSuffix(tableName) dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, tableName) - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id INT GENERATED ALWAYS AS IDENTITY, c1 INT GENERATED BY DEFAULT AS IDENTITY, @@ -1236,7 +1236,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Column_Exclusion() { // insert 10 rows into the source table for i := 0; i < 10; i++ { testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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) @@ -1244,10 +1244,10 @@ func (s PeerFlowE2ETestSuiteSF) Test_Column_Exclusion() { s.t.Log("Inserted 10 rows into the source table") e2e.EnvWaitForEqualTables(env, s, "normalize table", tableName, "id,c1,t,t2") - _, err = s.pool.Exec(context.Background(), + _, 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.pool.Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=0`, srcTableName)) + _, 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") @@ -1274,7 +1274,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Soft_Delete_Basic() { srcTableName := s.attachSchemaSuffix(tableName) dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, dstName) - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id INT PRIMARY KEY GENERATED ALWAYS AS IDENTITY, c1 INT, @@ -1314,15 +1314,15 @@ func (s PeerFlowE2ETestSuiteSF) Test_Soft_Delete_Basic() { go func() { e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - _, err = s.pool.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.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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.pool.Exec(context.Background(), fmt.Sprintf(` + _, err = s.conn.Exec(context.Background(), fmt.Sprintf(` DELETE FROM %s WHERE id=1`, srcTableName)) e2e.EnvNoError(s.t, env, err) e2e.EnvWaitForEqualTablesWithNames( @@ -1354,7 +1354,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Soft_Delete_IUD_Same_Batch() { srcTableName := fmt.Sprintf("%s_src", cmpTableName) dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_softdel_iud") - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id INT PRIMARY KEY GENERATED ALWAYS AS IDENTITY, c1 INT, @@ -1394,7 +1394,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Soft_Delete_IUD_Same_Batch() { go func() { e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - insertTx, err := s.pool.Begin(context.Background()) + insertTx, err := s.conn.Begin(context.Background()) e2e.EnvNoError(s.t, env, err) _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` @@ -1438,7 +1438,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Soft_Delete_UD_Same_Batch() { srcTableName := s.attachSchemaSuffix(tableName) dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, dstName) - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id INT PRIMARY KEY GENERATED ALWAYS AS IDENTITY, c1 INT, @@ -1478,12 +1478,12 @@ func (s PeerFlowE2ETestSuiteSF) Test_Soft_Delete_UD_Same_Batch() { go func() { e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - _, err = s.pool.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") - insertTx, err := s.pool.Begin(context.Background()) + 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)) @@ -1525,7 +1525,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Soft_Delete_Insert_After_Delete() { srcTableName := s.attachSchemaSuffix(tableName) dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, tableName) - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id INT PRIMARY KEY GENERATED BY DEFAULT AS IDENTITY, c1 INT, @@ -1565,11 +1565,11 @@ func (s PeerFlowE2ETestSuiteSF) Test_Soft_Delete_Insert_After_Delete() { go func() { e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) - _, err = s.pool.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.pool.Exec(context.Background(), fmt.Sprintf(` + _, err = s.conn.Exec(context.Background(), fmt.Sprintf(` DELETE FROM %s WHERE id=1`, srcTableName)) e2e.EnvNoError(s.t, env, err) e2e.EnvWaitForEqualTablesWithNames( @@ -1581,7 +1581,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Soft_Delete_Insert_After_Delete() { "id,c1,c2,t", ) - _, err = s.pool.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") @@ -1605,7 +1605,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Supported_Mixed_Case_Table_SF() { srcTableName := s.attachSchemaSuffix("testMixedCase") dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "testMixedCase") - _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + _, err := s.conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS e2e_test_%s."%s" ( "pulseArmor" SERIAL PRIMARY KEY, "highGold" TEXT NOT NULL, @@ -1637,7 +1637,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Supported_Mixed_Case_Table_SF() { for i := 0; i < 20; i++ { testKey := fmt.Sprintf("test_key_%d", i) testValue := fmt.Sprintf("test_value_%d", i) - _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + _, 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) diff --git a/flow/e2e/snowflake/qrep_flow_sf_test.go b/flow/e2e/snowflake/qrep_flow_sf_test.go index ac90de1770..6e76353fe1 100644 --- a/flow/e2e/snowflake/qrep_flow_sf_test.go +++ b/flow/e2e/snowflake/qrep_flow_sf_test.go @@ -12,9 +12,9 @@ import ( //nolint:unparam func (s PeerFlowE2ETestSuiteSF) setupSourceTable(tableName string, numRows int) { - err := e2e.CreateTableForQRep(s.pool, s.pgSuffix, tableName) + err := e2e.CreateTableForQRep(s.conn, s.pgSuffix, tableName) require.NoError(s.t, err) - err = e2e.PopulateSourceTable(s.pool, s.pgSuffix, tableName, numRows) + err = e2e.PopulateSourceTable(s.conn, s.pgSuffix, tableName, numRows) require.NoError(s.t, err) } @@ -34,7 +34,7 @@ func (s PeerFlowE2ETestSuiteSF) checkJSONValue(tableName, colName, fieldName, va } func (s PeerFlowE2ETestSuiteSF) compareTableContentsWithDiffSelectorsSF(tableName, pgSelector, sfSelector string) { - pgRows, err := e2e.GetPgRows(s.pool, s.pgSuffix, tableName, pgSelector) + pgRows, err := e2e.GetPgRows(s.conn, s.pgSuffix, tableName, pgSelector) require.NoError(s.t, err) sfRows, err := s.GetRows(tableName, sfSelector) diff --git a/flow/e2e/sqlserver/qrep_flow_sqlserver_test.go b/flow/e2e/sqlserver/qrep_flow_sqlserver_test.go index 3cb9cda650..bbaee9f0fd 100644 --- a/flow/e2e/sqlserver/qrep_flow_sqlserver_test.go +++ b/flow/e2e/sqlserver/qrep_flow_sqlserver_test.go @@ -10,8 +10,8 @@ import ( "testing" "time" + "github.com/jackc/pgx/v5" "github.com/jackc/pgx/v5/pgtype" - "github.com/jackc/pgx/v5/pgxpool" "github.com/joho/godotenv" "github.com/stretchr/testify/require" @@ -26,7 +26,7 @@ import ( type PeerFlowE2ETestSuiteSQLServer struct { t *testing.T - pool *pgxpool.Pool + conn *pgx.Conn sqlsHelper *SQLServerHelper suffix string } @@ -35,8 +35,8 @@ func (s PeerFlowE2ETestSuiteSQLServer) T() *testing.T { return s.t } -func (s PeerFlowE2ETestSuiteSQLServer) Pool() *pgxpool.Pool { - return s.pool +func (s PeerFlowE2ETestSuiteSQLServer) Conn() *pgx.Conn { + return s.conn } func (s PeerFlowE2ETestSuiteSQLServer) Suffix() string { @@ -65,7 +65,7 @@ func SetupSuite(t *testing.T) PeerFlowE2ETestSuiteSQLServer { } suffix := "sqls_" + strings.ToLower(shared.RandomString(8)) - pool, err := e2e.SetupPostgres(suffix) + conn, err := e2e.SetupPostgres(suffix) if err != nil { require.NoError(t, err) } @@ -81,7 +81,7 @@ func SetupSuite(t *testing.T) PeerFlowE2ETestSuiteSQLServer { return PeerFlowE2ETestSuiteSQLServer{ t: t, - pool: pool, + conn: conn, sqlsHelper: sqlsHelper, suffix: suffix, } @@ -115,10 +115,10 @@ func (s PeerFlowE2ETestSuiteSQLServer) insertRowsIntoSQLServerTable(tableName st func (s PeerFlowE2ETestSuiteSQLServer) setupPGDestinationTable(tableName string) { ctx := context.Background() - _, err := s.pool.Exec(ctx, fmt.Sprintf("DROP TABLE IF EXISTS e2e_test_%s.%s", s.suffix, tableName)) + _, err := s.conn.Exec(ctx, fmt.Sprintf("DROP TABLE IF EXISTS e2e_test_%s.%s", s.suffix, tableName)) require.NoError(s.t, err) - _, err = s.pool.Exec(ctx, + _, err = s.conn.Exec(ctx, fmt.Sprintf("CREATE TABLE e2e_test_%s.%s (id TEXT, card_id TEXT, v_from TIMESTAMP, price NUMERIC, status INT)", s.suffix, tableName)) require.NoError(s.t, err) @@ -184,7 +184,7 @@ func (s PeerFlowE2ETestSuiteSQLServer) Test_Complete_QRep_Flow_SqlServer_Append( // Verify that the destination table has the same number of rows as the source table var numRowsInDest pgtype.Int8 countQuery := fmt.Sprintf("SELECT COUNT(*) FROM %s", dstTableName) - err = s.pool.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 772cdc78aa..2d4052c54a 100644 --- a/flow/e2e/test_utils.go +++ b/flow/e2e/test_utils.go @@ -15,6 +15,7 @@ import ( "github.com/google/uuid" "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" @@ -90,8 +91,8 @@ func EnvTrue(t *testing.T, env *testsuite.TestWorkflowEnvironment, val bool) { } } -func GetPgRows(pool *pgxpool.Pool, suffix string, table string, cols string) (*model.QRecordBatch, error) { - pgQueryExecutor := connpostgres.NewQRepQueryExecutor(pool, context.Background(), "testflow", "testpart") +func GetPgRows(conn *pgx.Conn, suffix string, table string, cols string) (*model.QRecordBatch, error) { + pgQueryExecutor := connpostgres.NewQRepQueryExecutor(conn, context.Background(), "testflow", "testpart") pgQueryExecutor.SetTestEnv(true) return pgQueryExecutor.ExecuteAndProcessQuery( @@ -103,9 +104,7 @@ func RequireEqualTables(suite e2eshared.RowSource, table string, cols string) { t := suite.T() t.Helper() - suffix := suite.Suffix() - pool := suite.Pool() - pgRows, err := GetPgRows(pool, suffix, table, cols) + pgRows, err := GetPgRows(suite.Conn(), suite.Suffix(), table, cols) require.NoError(t, err) rows, err := suite.GetRows(table, cols) @@ -118,9 +117,7 @@ func EnvEqualTables(env *testsuite.TestWorkflowEnvironment, suite e2eshared.RowS t := suite.T() t.Helper() - suffix := suite.Suffix() - pool := suite.Pool() - pgRows, err := GetPgRows(pool, suffix, table, cols) + pgRows, err := GetPgRows(suite.Conn(), suite.Suffix(), table, cols) EnvNoError(t, env, err) rows, err := suite.GetRows(table, cols) @@ -154,9 +151,7 @@ func EnvWaitForEqualTablesWithNames( EnvWaitFor(t, env, 3*time.Minute, reason, func() bool { t.Helper() - suffix := suite.Suffix() - pool := suite.Pool() - pgRows, err := GetPgRows(pool, suffix, srcTable, cols) + pgRows, err := GetPgRows(suite.Conn(), suite.Suffix(), srcTable, cols) if err != nil { return false } @@ -202,7 +197,7 @@ func SetupCDCFlowStatusQuery(t *testing.T, env *testsuite.TestWorkflowEnvironmen } } -func CreateTableForQRep(pool *pgxpool.Pool, suffix string, tableName string) error { +func CreateTableForQRep(conn *pgx.Conn, suffix string, tableName string) error { createMoodEnum := "CREATE TYPE mood AS ENUM ('happy', 'sad', 'angry');" tblFields := []string{ @@ -261,11 +256,11 @@ func CreateTableForQRep(pool *pgxpool.Pool, suffix string, tableName string) err } tblFieldStr := strings.Join(tblFields, ",") var pgErr *pgconn.PgError - _, enumErr := pool.Exec(context.Background(), createMoodEnum) + _, enumErr := conn.Exec(context.Background(), createMoodEnum) if errors.As(enumErr, &pgErr) && pgErr.Code != pgerrcode.DuplicateObject && !utils.IsUniqueError(pgErr) { return enumErr } - _, err := pool.Exec(context.Background(), fmt.Sprintf(` + _, err := conn.Exec(context.Background(), fmt.Sprintf(` CREATE TABLE e2e_test_%s.%s ( %s );`, suffix, tableName, tblFieldStr)) @@ -290,7 +285,7 @@ func generate20MBJson() ([]byte, error) { return v, nil } -func PopulateSourceTable(pool *pgxpool.Pool, suffix string, tableName string, rowCount int) error { +func PopulateSourceTable(conn *pgx.Conn, suffix string, tableName string, rowCount int) error { var ids []string var rows []string for i := 0; i < rowCount-1; i++ { @@ -322,7 +317,7 @@ func PopulateSourceTable(pool *pgxpool.Pool, suffix string, tableName string, ro rows = append(rows, row) } - _, err := pool.Exec(context.Background(), fmt.Sprintf(` + _, err := conn.Exec(context.Background(), fmt.Sprintf(` INSERT INTO e2e_test_%s.%s ( id, card_id, "from", price, created_at, updated_at, transaction_hash, ownerable_type, ownerable_id, @@ -330,7 +325,7 @@ func PopulateSourceTable(pool *pgxpool.Pool, suffix string, tableName string, ro deal_id, ethereum_transaction_id, ignore_price, card_eth_value, paid_eth_price, card_bought_notified, address, account_id, asset_id, status, transaction_id, settled_at, reference_id, - settle_at, settlement_delay_reason, f1, f2, f3, f4, f5, f6, f7, f8, f9, f10, f11, f12, f13, my_date, + settle_at, settlement_delay_reason, f1, f2, f3, f4, f5, f6, f7, f8, f9, f10, f11, f12, f13, my_date, my_time, my_mood, myh, "geometryPoint", geography_point,geometry_linestring, geography_linestring,geometry_polygon, geography_polygon ) VALUES %s; @@ -340,7 +335,7 @@ func PopulateSourceTable(pool *pgxpool.Pool, suffix string, tableName string, ro } // add a row where all the nullable fields are null - _, err = pool.Exec(context.Background(), fmt.Sprintf(` + _, err = conn.Exec(context.Background(), fmt.Sprintf(` INSERT INTO e2e_test_%s.%s ( id, "from", created_at, updated_at, transfer_type, blockchain, card_bought_notified, asset_id @@ -358,7 +353,7 @@ func PopulateSourceTable(pool *pgxpool.Pool, suffix string, tableName string, ro if err != nil { return err } - _, err = pool.Exec(context.Background(), fmt.Sprintf(` + _, err = conn.Exec(context.Background(), fmt.Sprintf(` UPDATE e2e_test_%s.%s SET f5 = $1 WHERE id = $2; `, suffix, tableName), v, ids[0]) if err != nil { @@ -366,7 +361,7 @@ func PopulateSourceTable(pool *pgxpool.Pool, suffix string, tableName string, ro } // update my_date to a date before 1970 - _, err = pool.Exec(context.Background(), fmt.Sprintf(` + _, err = conn.Exec(context.Background(), fmt.Sprintf(` UPDATE e2e_test_%s.%s SET old_date = '1950-01-01' WHERE id = $1; `, suffix, tableName), ids[0]) if err != nil { diff --git a/flow/e2eshared/e2eshared.go b/flow/e2eshared/e2eshared.go index a9f91dead4..ce5c25d4fe 100644 --- a/flow/e2eshared/e2eshared.go +++ b/flow/e2eshared/e2eshared.go @@ -8,7 +8,7 @@ import ( "strings" "testing" - "github.com/jackc/pgx/v5/pgxpool" + "github.com/jackc/pgx/v5" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" @@ -16,7 +16,7 @@ import ( type Suite interface { T() *testing.T - Pool() *pgxpool.Pool + Conn() *pgx.Conn Suffix() string }