diff --git a/.editorconfig b/.editorconfig index 174af3e952..004d9e96f2 100644 --- a/.editorconfig +++ b/.editorconfig @@ -10,6 +10,6 @@ trim_trailing_whitespace = true indent_style = space indent_size = 4 -[{package.json,*.yml,*.yaml}] +[{package.json,*.yml,*.yaml,*.proto}] indent_style = space indent_size = 2 diff --git a/flow/.golangci.yml b/flow/.golangci.yml index f8a051b2c6..638fab22f9 100644 --- a/flow/.golangci.yml +++ b/flow/.golangci.yml @@ -7,25 +7,40 @@ linters: - durationcheck - errcheck - forbidigo + - gci + - gocritic - gofumpt - gosec - gosimple + - ineffassign + - lll - misspell + - musttag - nakedret - nolintlint + - nonamedreturns + - perfsprint + - prealloc - staticcheck - stylecheck - sqlclosecheck + - testifylint + - thelper - unconvert - unparam - - whitespace - - prealloc - - thelper - - ineffassign - unparam - unused - - lll + - wastedassign + - whitespace linters-settings: + gci: + sections: + - standard + - 'prefix(github.com/PeerDB-io)' + - default + gocritic: + disabled-checks: + - ifElseChain stylecheck: checks: - all diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index e1ecf2bddf..902e531cb9 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -9,6 +9,14 @@ import ( "sync" "time" + "github.com/jackc/pglogrepl" + "github.com/jackc/pgx/v5" + "github.com/jackc/pgx/v5/pgtype" + "github.com/jackc/pgx/v5/pgxpool" + "go.temporal.io/sdk/activity" + "golang.org/x/sync/errgroup" + "google.golang.org/protobuf/proto" + "github.com/PeerDB-io/peer-flow/connectors" connbigquery "github.com/PeerDB-io/peer-flow/connectors/bigquery" connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" @@ -20,13 +28,6 @@ import ( "github.com/PeerDB-io/peer-flow/peerdbenv" "github.com/PeerDB-io/peer-flow/shared" "github.com/PeerDB-io/peer-flow/shared/alerting" - "github.com/jackc/pglogrepl" - "github.com/jackc/pgx/v5" - "github.com/jackc/pgx/v5/pgtype" - "github.com/jackc/pgx/v5/pgxpool" - "go.temporal.io/sdk/activity" - "golang.org/x/sync/errgroup" - "google.golang.org/protobuf/proto" ) // CheckConnectionResult is the result of a CheckConnection call. @@ -250,24 +251,6 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, hasRecords := !recordBatch.WaitAndCheckEmpty() slog.InfoContext(ctx, fmt.Sprintf("the current sync flow has records: %v", hasRecords)) - if a.CatalogPool != nil && hasRecords { - syncBatchID, err := dstConn.GetLastSyncBatchID(flowName) - if err != nil && conn.Destination.Type != protos.DBType_EVENTHUB { - return nil, err - } - - err = monitoring.AddCDCBatchForFlow(ctx, a.CatalogPool, flowName, - monitoring.CDCBatchInfo{ - BatchID: syncBatchID + 1, - RowsInBatch: 0, - BatchEndlSN: 0, - StartTime: startTime, - }) - if err != nil { - a.Alerter.LogFlowError(ctx, flowName, err) - return nil, err - } - } if !hasRecords { // wait for the pull goroutine to finish @@ -277,27 +260,46 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, return nil, fmt.Errorf("failed in pull records when: %w", err) } slog.InfoContext(ctx, "no records to push") - tableSchemaDeltas := recordBatch.WaitForSchemaDeltas(input.FlowConnectionConfigs.TableMappings) - err := dstConn.ReplayTableSchemaDeltas(flowName, tableSchemaDeltas) + err := dstConn.ReplayTableSchemaDeltas(flowName, recordBatch.SchemaDeltas) if err != nil { return nil, fmt.Errorf("failed to sync schema: %w", err) } return &model.SyncResponse{ CurrentSyncBatchID: -1, - RelationMessageMapping: <-recordBatch.RelationMessageMapping, - TableSchemaDeltas: tableSchemaDeltas, + TableSchemaDeltas: recordBatch.SchemaDeltas, + RelationMessageMapping: input.RelationMessageMapping, }, nil } + syncBatchID, err := dstConn.GetLastSyncBatchID(flowName) + if err != nil && conn.Destination.Type != protos.DBType_EVENTHUB { + return nil, err + } + syncBatchID += 1 + + err = monitoring.AddCDCBatchForFlow(ctx, a.CatalogPool, flowName, + monitoring.CDCBatchInfo{ + BatchID: syncBatchID, + RowsInBatch: 0, + BatchEndlSN: 0, + StartTime: startTime, + }) + if err != nil { + a.Alerter.LogFlowError(ctx, flowName, err) + return nil, err + } + syncStartTime := time.Now() res, err := dstConn.SyncRecords(&model.SyncRecordsRequest{ + SyncBatchID: syncBatchID, Records: recordBatch, FlowJobName: input.FlowConnectionConfigs.FlowJobName, TableMappings: input.FlowConnectionConfigs.TableMappings, StagingPath: input.FlowConnectionConfigs.CdcStagingPath, }) + res.RelationMessageMapping = input.RelationMessageMapping if err != nil { slog.Warn("failed to push records", slog.Any("error", err)) a.Alerter.LogFlowError(ctx, flowName, err) @@ -375,7 +377,7 @@ func (a *FlowableActivity) StartNormalize( if errors.Is(err, connectors.ErrUnsupportedFunctionality) { dstConn, err := connectors.GetCDCSyncConnector(ctx, conn.Destination) if err != nil { - return nil, fmt.Errorf("failed to get connector: %v", err) + return nil, fmt.Errorf("failed to get connector: %w", err) } defer connectors.CloseConnector(dstConn) @@ -570,7 +572,7 @@ func (a *FlowableActivity) replicateQRepPartition(ctx context.Context, err = monitoring.UpdatePullEndTimeAndRowsForPartition(ctx, a.CatalogPool, runUUID, partition, numRecords) if err != nil { - slog.ErrorContext(ctx, fmt.Sprintf("%v", err)) + slog.ErrorContext(ctx, err.Error()) goroutineErr = err } } @@ -582,10 +584,9 @@ func (a *FlowableActivity) replicateQRepPartition(ctx context.Context, a.Alerter.LogFlowError(ctx, config.FlowJobName, err) return fmt.Errorf("failed to pull qrep records: %w", err) } - numRecords := int64(recordBatch.NumRecords) slog.InfoContext(ctx, fmt.Sprintf("pulled %d records\n", len(recordBatch.Records))) - err = monitoring.UpdatePullEndTimeAndRowsForPartition(ctx, a.CatalogPool, runUUID, partition, numRecords) + err = monitoring.UpdatePullEndTimeAndRowsForPartition(ctx, a.CatalogPool, runUUID, partition, int64(len(recordBatch.Records))) if err != nil { return err } @@ -664,29 +665,24 @@ func (a *FlowableActivity) CleanupQRepFlow(ctx context.Context, config *protos.Q return dst.CleanupQRepFlow(config) } -func (a *FlowableActivity) DropFlow(ctx context.Context, config *protos.ShutdownRequest) error { - ctx = context.WithValue(ctx, shared.FlowNameKey, config.FlowJobName) +func (a *FlowableActivity) DropFlowSource(ctx context.Context, config *protos.ShutdownRequest) error { srcConn, err := connectors.GetCDCPullConnector(ctx, config.SourcePeer) if err != nil { return fmt.Errorf("failed to get source connector: %w", err) } defer connectors.CloseConnector(srcConn) + return srcConn.PullFlowCleanup(config.FlowJobName) +} + +func (a *FlowableActivity) DropFlowDestination(ctx context.Context, config *protos.ShutdownRequest) error { dstConn, err := connectors.GetCDCSyncConnector(ctx, config.DestinationPeer) if err != nil { return fmt.Errorf("failed to get destination connector: %w", err) } defer connectors.CloseConnector(dstConn) - err = srcConn.PullFlowCleanup(config.FlowJobName) - if err != nil { - return fmt.Errorf("failed to cleanup source: %w", err) - } - err = dstConn.SyncFlowCleanup(config.FlowJobName) - if err != nil { - return fmt.Errorf("failed to cleanup destination: %w", err) - } - return nil + return dstConn.SyncFlowCleanup(config.FlowJobName) } func (a *FlowableActivity) getPostgresPeerConfigs(ctx context.Context) ([]*protos.Peer, error) { @@ -959,7 +955,7 @@ func (a *FlowableActivity) ReplicateXminPartition(ctx context.Context, err = monitoring.UpdatePullEndTimeAndRowsForPartition( errCtx, a.CatalogPool, runUUID, partition, int64(numRecords)) if err != nil { - slog.Error(fmt.Sprintf("%v", err)) + slog.Error(err.Error()) return err } diff --git a/flow/cmd/api.go b/flow/cmd/api.go index f466a6b9f9..1e6d3c6d17 100644 --- a/flow/cmd/api.go +++ b/flow/cmd/api.go @@ -10,21 +10,20 @@ import ( "net/http" "time" - utils "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/shared" - peerflow "github.com/PeerDB-io/peer-flow/workflows" "github.com/google/uuid" "github.com/grpc-ecosystem/grpc-gateway/v2/runtime" - - "google.golang.org/grpc" - "google.golang.org/grpc/credentials/insecure" - "google.golang.org/grpc/reflection" - "go.temporal.io/api/workflowservice/v1" "go.temporal.io/sdk/client" + "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" "google.golang.org/grpc/health" "google.golang.org/grpc/health/grpc_health_v1" + "google.golang.org/grpc/reflection" + + utils "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/shared" + peerflow "github.com/PeerDB-io/peer-flow/workflows" ) type APIServerParams struct { diff --git a/flow/cmd/handler.go b/flow/cmd/handler.go index 1cee1fd5fc..93d5ff1ea6 100644 --- a/flow/cmd/handler.go +++ b/flow/cmd/handler.go @@ -8,15 +8,16 @@ import ( "strings" "time" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/shared" - peerflow "github.com/PeerDB-io/peer-flow/workflows" backoff "github.com/cenkalti/backoff/v4" "github.com/google/uuid" "github.com/jackc/pgx/v5/pgtype" "github.com/jackc/pgx/v5/pgxpool" "go.temporal.io/sdk/client" "google.golang.org/protobuf/proto" + + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/shared" + peerflow "github.com/PeerDB-io/peer-flow/workflows" ) // grpc server implementation diff --git a/flow/cmd/main.go b/flow/cmd/main.go index 79741669f4..1d924e3ccc 100644 --- a/flow/cmd/main.go +++ b/flow/cmd/main.go @@ -8,9 +8,10 @@ import ( "os/signal" "syscall" - "github.com/PeerDB-io/peer-flow/logger" "github.com/urfave/cli/v3" _ "go.uber.org/automaxprocs" + + "github.com/PeerDB-io/peer-flow/logger" ) func main() { diff --git a/flow/cmd/mirror_status.go b/flow/cmd/mirror_status.go index 1e17495085..df7862d8fe 100644 --- a/flow/cmd/mirror_status.go +++ b/flow/cmd/mirror_status.go @@ -6,11 +6,12 @@ import ( "fmt" "log/slog" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/shared" "github.com/jackc/pgx/v5/pgtype" "google.golang.org/protobuf/proto" "google.golang.org/protobuf/types/known/timestamppb" + + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/shared" ) func (h *FlowRequestHandler) MirrorStatus( diff --git a/flow/cmd/peer_data.go b/flow/cmd/peer_data.go index f91f37bf8c..110b9b5a7f 100644 --- a/flow/cmd/peer_data.go +++ b/flow/cmd/peer_data.go @@ -6,13 +6,14 @@ import ( "fmt" "log/slog" - connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" - "github.com/PeerDB-io/peer-flow/connectors/utils" - "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/jackc/pgx/v5" "github.com/jackc/pgx/v5/pgtype" "github.com/jackc/pgx/v5/pgxpool" "google.golang.org/protobuf/proto" + + connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" + "github.com/PeerDB-io/peer-flow/connectors/utils" + "github.com/PeerDB-io/peer-flow/generated/protos" ) func (h *FlowRequestHandler) getPGPeerConfig(ctx context.Context, peerName string) (*protos.PostgresConfig, error) { diff --git a/flow/cmd/snapshot_worker.go b/flow/cmd/snapshot_worker.go index 9239cf7276..d76a095ace 100644 --- a/flow/cmd/snapshot_worker.go +++ b/flow/cmd/snapshot_worker.go @@ -4,14 +4,14 @@ import ( "crypto/tls" "fmt" + "go.temporal.io/sdk/client" + "go.temporal.io/sdk/worker" + "github.com/PeerDB-io/peer-flow/activities" utils "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" "github.com/PeerDB-io/peer-flow/shared" "github.com/PeerDB-io/peer-flow/shared/alerting" peerflow "github.com/PeerDB-io/peer-flow/workflows" - - "go.temporal.io/sdk/client" - "go.temporal.io/sdk/worker" ) type SnapshotWorkerOptions struct { diff --git a/flow/cmd/worker.go b/flow/cmd/worker.go index f060230b63..aacc7d85bc 100644 --- a/flow/cmd/worker.go +++ b/flow/cmd/worker.go @@ -10,16 +10,15 @@ import ( "runtime" "syscall" + "github.com/grafana/pyroscope-go" + "go.temporal.io/sdk/client" + "go.temporal.io/sdk/worker" + "github.com/PeerDB-io/peer-flow/activities" utils "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" "github.com/PeerDB-io/peer-flow/shared" "github.com/PeerDB-io/peer-flow/shared/alerting" peerflow "github.com/PeerDB-io/peer-flow/workflows" - - "github.com/grafana/pyroscope-go" - - "go.temporal.io/sdk/client" - "go.temporal.io/sdk/worker" ) type WorkerOptions struct { diff --git a/flow/connectors/bigquery/bigquery.go b/flow/connectors/bigquery/bigquery.go index 10d44c21c3..8e6fd3e38c 100644 --- a/flow/connectors/bigquery/bigquery.go +++ b/flow/connectors/bigquery/bigquery.go @@ -12,17 +12,17 @@ import ( "cloud.google.com/go/bigquery" "cloud.google.com/go/storage" + "github.com/jackc/pgx/v5/pgxpool" + "go.temporal.io/sdk/activity" + "google.golang.org/api/iterator" + "google.golang.org/api/option" + "github.com/PeerDB-io/peer-flow/connectors/utils" cc "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" "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/shared" - "github.com/jackc/pgx/v5/pgxpool" - - "go.temporal.io/sdk/activity" - "google.golang.org/api/iterator" - "google.golang.org/api/option" ) const ( @@ -63,6 +63,7 @@ type BigQueryConnector struct { client *bigquery.Client storageClient *storage.Client datasetID string + projectID string catalogPool *pgxpool.Pool logger slog.Logger } @@ -148,13 +149,24 @@ func NewBigQueryConnector(ctx context.Context, config *protos.BigqueryConfig) (* return nil, fmt.Errorf("failed to create BigQueryServiceAccount: %v", err) } + datasetID := config.GetDatasetId() + projectID := config.GetProjectId() + projectPart, datasetPart, found := strings.Cut(datasetID, ".") + if found && strings.Contains(datasetPart, ".") { + return nil, + fmt.Errorf("invalid dataset ID: %s. Ensure that it is just a single string or string1.string2", datasetID) + } + if projectPart != "" && datasetPart != "" { + datasetID = datasetPart + projectID = projectPart + } + client, err := bqsa.CreateBigQueryClient(ctx) if err != nil { return nil, fmt.Errorf("failed to create BigQuery client: %v", err) } - datasetID := config.GetDatasetId() - _, checkErr := client.Dataset(datasetID).Metadata(ctx) + _, checkErr := client.DatasetInProject(projectID, datasetID).Metadata(ctx) if checkErr != nil { slog.ErrorContext(ctx, "failed to get dataset metadata", slog.Any("error", checkErr)) return nil, fmt.Errorf("failed to get dataset metadata: %v", checkErr) @@ -177,6 +189,7 @@ func NewBigQueryConnector(ctx context.Context, config *protos.BigqueryConfig) (* bqConfig: config, client: client, datasetID: datasetID, + projectID: projectID, storageClient: storageClient, catalogPool: catalogPool, logger: *slog.With(slog.String(string(shared.FlowNameKey), flowName)), @@ -193,7 +206,7 @@ func (c *BigQueryConnector) Close() error { // ConnectionActive returns true if the connection is active. func (c *BigQueryConnector) ConnectionActive() error { - _, err := c.client.Dataset(c.datasetID).Metadata(c.ctx) + _, err := c.client.DatasetInProject(c.projectID, c.datasetID).Metadata(c.ctx) if err != nil { return fmt.Errorf("failed to get dataset metadata: %v", err) } @@ -206,12 +219,12 @@ func (c *BigQueryConnector) ConnectionActive() error { // NeedsSetupMetadataTables returns true if the metadata tables need to be set up. func (c *BigQueryConnector) NeedsSetupMetadataTables() bool { - _, err := c.client.Dataset(c.datasetID).Table(MirrorJobsTable).Metadata(c.ctx) + _, err := c.client.DatasetInProject(c.projectID, c.datasetID).Table(MirrorJobsTable).Metadata(c.ctx) return err != nil } func (c *BigQueryConnector) waitForTableReady(datasetTable *datasetTable) error { - table := c.client.Dataset(datasetTable.dataset).Table(datasetTable.table) + table := c.client.DatasetInProject(c.projectID, datasetTable.dataset).Table(datasetTable.table) maxDuration := 5 * time.Minute deadline := time.Now().Add(maxDuration) sleepInterval := 5 * time.Second @@ -246,10 +259,13 @@ func (c *BigQueryConnector) ReplayTableSchemaDeltas(flowJobName string, for _, addedColumn := range schemaDelta.AddedColumns { dstDatasetTable, _ := c.convertToDatasetTable(schemaDelta.DstTableName) - _, err := c.client.Query(fmt.Sprintf( + query := c.client.Query(fmt.Sprintf( "ALTER TABLE %s.%s ADD COLUMN IF NOT EXISTS `%s` %s", dstDatasetTable.dataset, dstDatasetTable.table, addedColumn.ColumnName, - qValueKindToBigQueryType(addedColumn.ColumnType))).Read(c.ctx) + qValueKindToBigQueryType(addedColumn.ColumnType))) + query.DefaultProjectID = c.projectID + query.DefaultDatasetID = c.datasetID + _, err := query.Read(c.ctx) if err != nil { return fmt.Errorf("failed to add column %s for table %s: %w", addedColumn.ColumnName, schemaDelta.DstTableName, err) @@ -265,7 +281,7 @@ func (c *BigQueryConnector) ReplayTableSchemaDeltas(flowJobName string, // SetupMetadataTables sets up the metadata tables. func (c *BigQueryConnector) SetupMetadataTables() error { // check if the dataset exists - dataset := c.client.Dataset(c.datasetID) + dataset := c.client.DatasetInProject(c.projectID, c.datasetID) if _, err := dataset.Metadata(c.ctx); err != nil { // create the dataset as it doesn't exist if err := dataset.Create(c.ctx, nil); err != nil { @@ -298,6 +314,8 @@ func (c *BigQueryConnector) SetupMetadataTables() error { func (c *BigQueryConnector) GetLastOffset(jobName string) (int64, error) { query := fmt.Sprintf("SELECT offset FROM %s.%s WHERE mirror_job_name = '%s'", c.datasetID, MirrorJobsTable, jobName) q := c.client.Query(query) + q.DefaultProjectID = c.projectID + q.DefaultDatasetID = c.datasetID it, err := q.Read(c.ctx) if err != nil { err = fmt.Errorf("failed to run query %s on BigQuery:\n %w", query, err) @@ -328,6 +346,8 @@ func (c *BigQueryConnector) SetLastOffset(jobName string, lastOffset int64) erro jobName, ) q := c.client.Query(query) + q.DefaultProjectID = c.projectID + q.DefaultDatasetID = c.datasetID _, err := q.Read(c.ctx) if err != nil { return fmt.Errorf("failed to run query %s on BigQuery:\n %w", query, err) @@ -341,6 +361,8 @@ func (c *BigQueryConnector) GetLastSyncBatchID(jobName string) (int64, error) { c.datasetID, MirrorJobsTable, jobName) q := c.client.Query(query) q.DisableQueryCache = true + q.DefaultProjectID = c.projectID + q.DefaultDatasetID = c.datasetID it, err := q.Read(c.ctx) if err != nil { err = fmt.Errorf("failed to run query %s on BigQuery:\n %w", query, err) @@ -367,6 +389,8 @@ func (c *BigQueryConnector) GetLastNormalizeBatchID(jobName string) (int64, erro c.datasetID, MirrorJobsTable, jobName) q := c.client.Query(query) q.DisableQueryCache = true + q.DefaultProjectID = c.projectID + q.DefaultDatasetID = c.datasetID it, err := q.Read(c.ctx) if err != nil { err = fmt.Errorf("failed to run query %s on BigQuery:\n %w", query, err) @@ -397,6 +421,8 @@ func (c *BigQueryConnector) getDistinctTableNamesInBatch(flowJobName string, syn c.datasetID, rawTableName, normalizeBatchID, syncBatchID) // Run the query q := c.client.Query(query) + q.DefaultProjectID = c.projectID + q.DefaultDatasetID = c.datasetID it, err := q.Read(c.ctx) if err != nil { err = fmt.Errorf("failed to run query %s on BigQuery:\n %w", query, err) @@ -439,6 +465,8 @@ func (c *BigQueryConnector) getTableNametoUnchangedCols(flowJobName string, sync c.datasetID, rawTableName, normalizeBatchID, syncBatchID) // Run the query q := c.client.Query(query) + q.DefaultDatasetID = c.datasetID + q.DefaultProjectID = c.projectID it, err := q.Read(c.ctx) if err != nil { err = fmt.Errorf("failed to run query %s on BigQuery:\n %w", query, err) @@ -473,15 +501,7 @@ func (c *BigQueryConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S c.logger.Info(fmt.Sprintf("pushing records to %s.%s...", c.datasetID, rawTableName)) - // generate a sequential number for last synced batch this sequence will be - // used to keep track of records that are normalized in NormalizeFlowWorkflow - syncBatchID, err := c.GetLastSyncBatchID(req.FlowJobName) - if err != nil { - return nil, fmt.Errorf("failed to get batch for the current mirror: %v", err) - } - syncBatchID += 1 - - res, err := c.syncRecordsViaAvro(req, rawTableName, syncBatchID) + res, err := c.syncRecordsViaAvro(req, rawTableName, req.SyncBatchID) if err != nil { return nil, err } @@ -503,7 +523,7 @@ func (c *BigQueryConnector) syncRecordsViaAvro( } avroSync := NewQRepAvroSyncMethod(c, req.StagingPath, req.FlowJobName) - rawTableMetadata, err := c.client.Dataset(c.datasetID).Table(rawTableName).Metadata(c.ctx) + rawTableMetadata, err := c.client.DatasetInProject(c.projectID, c.datasetID).Table(rawTableName).Metadata(c.ctx) if err != nil { return nil, fmt.Errorf("failed to get metadata of destination table: %w", err) } @@ -588,6 +608,8 @@ func (c *BigQueryConnector) NormalizeRecords(req *model.NormalizeRecordsRequest) c.logger.Info(fmt.Sprintf("running merge statement [%d/%d] for table %s..", i+1, len(mergeStmts), tableName)) q := c.client.Query(mergeStmt) + q.DefaultProjectID = c.projectID + q.DefaultDatasetID = c.datasetID _, err = q.Read(c.ctx) if err != nil { return nil, fmt.Errorf("failed to execute merge statement %s: %v", mergeStmt, err) @@ -599,7 +621,10 @@ func (c *BigQueryConnector) NormalizeRecords(req *model.NormalizeRecordsRequest) "UPDATE %s.%s SET normalize_batch_id=%d WHERE mirror_job_name='%s';", c.datasetID, MirrorJobsTable, req.SyncBatchID, req.FlowJobName) - _, err = c.client.Query(updateMetadataStmt).Read(c.ctx) + query := c.client.Query(updateMetadataStmt) + query.DefaultProjectID = c.projectID + query.DefaultDatasetID = c.datasetID + _, err = query.Read(c.ctx) if err != nil { return nil, fmt.Errorf("failed to execute update metadata statements %s: %v", updateMetadataStmt, err) } @@ -633,7 +658,7 @@ func (c *BigQueryConnector) CreateRawTable(req *protos.CreateRawTableInput) (*pr } // create the table - table := c.client.Dataset(c.datasetID).Table(rawTableName) + table := c.client.DatasetInProject(c.projectID, c.datasetID).Table(rawTableName) // check if the table exists tableRef, err := table.Metadata(c.ctx) @@ -712,6 +737,8 @@ func (c *BigQueryConnector) metadataHasJob(jobName string) (bool, error) { c.datasetID, MirrorJobsTable, jobName) q := c.client.Query(checkStmt) + q.DefaultProjectID = c.projectID + q.DefaultDatasetID = c.datasetID it, err := q.Read(c.ctx) if err != nil { return false, fmt.Errorf("failed to check if job exists: %w", err) @@ -749,7 +776,7 @@ func (c *BigQueryConnector) SetupNormalizedTables( return nil, fmt.Errorf("invalid mirror: two tables mirror to the same BigQuery table %s", datasetTable.string()) } - dataset := c.client.Dataset(datasetTable.dataset) + dataset := c.client.DatasetInProject(c.projectID, datasetTable.dataset) _, err = dataset.Metadata(c.ctx) // just assume this means dataset don't exist, and create it if err != nil { @@ -837,7 +864,7 @@ func (c *BigQueryConnector) SetupNormalizedTables( } func (c *BigQueryConnector) SyncFlowCleanup(jobName string) error { - dataset := c.client.Dataset(c.datasetID) + dataset := c.client.DatasetInProject(c.projectID, c.datasetID) // deleting PeerDB specific tables err := dataset.Table(c.getRawTableName(jobName)).Delete(c.ctx) if err != nil { @@ -846,7 +873,10 @@ func (c *BigQueryConnector) SyncFlowCleanup(jobName string) error { // deleting job from metadata table query := fmt.Sprintf("DELETE FROM %s.%s WHERE mirror_job_name = '%s'", c.datasetID, MirrorJobsTable, jobName) - _, err = c.client.Query(query).Read(c.ctx) + queryHandler := c.client.Query(query) + queryHandler.DefaultProjectID = c.projectID + queryHandler.DefaultDatasetID = c.datasetID + _, err = queryHandler.Read(c.ctx) if err != nil { return fmt.Errorf("failed to delete job from metadata table: %w", err) } @@ -883,11 +913,15 @@ func (c *BigQueryConnector) RenameTables(req *protos.RenameTablesInput) (*protos srcDatasetTable.string(), fmt.Sprintf("%s,%s", allCols, *req.SoftDeleteColName), allCols, *req.SoftDeleteColName, dstDatasetTable.string(), pkeyCols, pkeyCols, srcDatasetTable.string())) - _, err := c.client.Query( + query := c.client.Query( fmt.Sprintf("INSERT INTO %s(%s) SELECT %s,true AS %s FROM %s WHERE (%s) NOT IN (SELECT %s FROM %s)", srcDatasetTable.string(), fmt.Sprintf("%s,%s", allCols, *req.SoftDeleteColName), allCols, *req.SoftDeleteColName, dstDatasetTable.string(), - pkeyCols, pkeyCols, srcDatasetTable.string())).Read(c.ctx) + pkeyCols, pkeyCols, srcDatasetTable.string())) + + query.DefaultProjectID = c.projectID + query.DefaultDatasetID = c.datasetID + _, err := query.Read(c.ctx) if err != nil { return nil, fmt.Errorf("unable to handle soft-deletes for table %s: %w", dstDatasetTable.string(), err) } @@ -902,9 +936,13 @@ func (c *BigQueryConnector) RenameTables(req *protos.RenameTablesInput) (*protos c.logger.InfoContext(c.ctx, fmt.Sprintf("UPDATE %s SET %s = CURRENT_TIMESTAMP WHERE %s IS NULL", srcDatasetTable.string(), *req.SyncedAtColName, *req.SyncedAtColName)) - _, err := c.client.Query( + query := c.client.Query( fmt.Sprintf("UPDATE %s SET %s = CURRENT_TIMESTAMP WHERE %s IS NULL", srcDatasetTable.string(), - *req.SyncedAtColName, *req.SyncedAtColName)).Read(c.ctx) + *req.SyncedAtColName, *req.SyncedAtColName)) + + query.DefaultProjectID = c.projectID + query.DefaultDatasetID = c.datasetID + _, err := query.Read(c.ctx) if err != nil { return nil, fmt.Errorf("unable to set synced at column for table %s: %w", srcDatasetTable.string(), err) } @@ -913,8 +951,11 @@ func (c *BigQueryConnector) RenameTables(req *protos.RenameTablesInput) (*protos c.logger.InfoContext(c.ctx, fmt.Sprintf("DROP TABLE IF EXISTS %s", dstDatasetTable.string())) // drop the dst table if exists - _, err := c.client.Query(fmt.Sprintf("DROP TABLE IF EXISTS %s", - dstDatasetTable.string())).Read(c.ctx) + dropQuery := c.client.Query(fmt.Sprintf("DROP TABLE IF EXISTS %s", + dstDatasetTable.string())) + dropQuery.DefaultProjectID = c.projectID + dropQuery.DefaultDatasetID = c.datasetID + _, err := dropQuery.Read(c.ctx) if err != nil { return nil, fmt.Errorf("unable to drop table %s: %w", dstDatasetTable.string(), err) } @@ -922,8 +963,11 @@ func (c *BigQueryConnector) RenameTables(req *protos.RenameTablesInput) (*protos c.logger.InfoContext(c.ctx, fmt.Sprintf("ALTER TABLE %s RENAME TO %s", srcDatasetTable.string(), dstDatasetTable.table)) // rename the src table to dst - _, err = c.client.Query(fmt.Sprintf("ALTER TABLE %s RENAME TO %s", - srcDatasetTable.string(), dstDatasetTable.table)).Read(c.ctx) + query := c.client.Query(fmt.Sprintf("ALTER TABLE %s RENAME TO %s", + srcDatasetTable.string(), dstDatasetTable.table)) + query.DefaultProjectID = c.projectID + query.DefaultDatasetID = c.datasetID + _, err = query.Read(c.ctx) if err != nil { return nil, fmt.Errorf("unable to rename table %s to %s: %w", srcDatasetTable.string(), dstDatasetTable.string(), err) @@ -949,8 +993,11 @@ func (c *BigQueryConnector) CreateTablesFromExisting(req *protos.CreateTablesFro activity.RecordHeartbeat(c.ctx, fmt.Sprintf("creating table '%s' similar to '%s'", newTable, existingTable)) // rename the src table to dst - _, err := c.client.Query(fmt.Sprintf("CREATE TABLE IF NOT EXISTS `%s` LIKE `%s`", - newDatasetTable.string(), existingDatasetTable.string())).Read(c.ctx) + query := c.client.Query(fmt.Sprintf("CREATE TABLE IF NOT EXISTS `%s` LIKE `%s`", + newDatasetTable.string(), existingDatasetTable.string())) + query.DefaultProjectID = c.projectID + query.DefaultDatasetID = c.datasetID + _, err := query.Read(c.ctx) if err != nil { return nil, fmt.Errorf("unable to create table %s: %w", newTable, err) } diff --git a/flow/connectors/bigquery/merge_stmt_generator.go b/flow/connectors/bigquery/merge_stmt_generator.go index e93a139a73..d87a83a290 100644 --- a/flow/connectors/bigquery/merge_stmt_generator.go +++ b/flow/connectors/bigquery/merge_stmt_generator.go @@ -5,6 +5,7 @@ import ( "strings" "cloud.google.com/go/bigquery" + "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model/qvalue" diff --git a/flow/connectors/bigquery/qrep.go b/flow/connectors/bigquery/qrep.go index 305bab01eb..5878dd2bd0 100644 --- a/flow/connectors/bigquery/qrep.go +++ b/flow/connectors/bigquery/qrep.go @@ -8,12 +8,12 @@ import ( "time" "cloud.google.com/go/bigquery" + "google.golang.org/api/iterator" + "google.golang.org/protobuf/encoding/protojson" + "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/shared" - - "google.golang.org/api/iterator" - "google.golang.org/protobuf/encoding/protojson" ) func (c *BigQueryConnector) SyncQRepRecords( @@ -54,7 +54,7 @@ func (c *BigQueryConnector) replayTableSchemaDeltasQRep(config *protos.QRepConfi srcSchema *model.QRecordSchema, ) (*bigquery.TableMetadata, error) { destDatasetTable, _ := c.convertToDatasetTable(config.DestinationTableIdentifier) - bqTable := c.client.Dataset(destDatasetTable.dataset).Table(destDatasetTable.table) + bqTable := c.client.DatasetInProject(c.projectID, destDatasetTable.dataset).Table(destDatasetTable.table) dstTableMetadata, err := bqTable.Metadata(c.ctx) if err != nil { return nil, fmt.Errorf("failed to get metadata of table %s: %w", destDatasetTable, err) @@ -134,7 +134,7 @@ func (c *BigQueryConnector) SetupQRepMetadataTables(config *protos.QRepConfig) e } // reference the table - table := c.client.Dataset(c.datasetID).Table(qRepMetadataTableName) + table := c.client.DatasetInProject(c.projectID, c.datasetID).Table(qRepMetadataTableName) // check if the table exists meta, err := table.Metadata(c.ctx) @@ -156,7 +156,10 @@ func (c *BigQueryConnector) SetupQRepMetadataTables(config *protos.QRepConfig) e } if config.WriteMode.WriteType == protos.QRepWriteType_QREP_WRITE_MODE_OVERWRITE { - _, err = c.client.Query(fmt.Sprintf("TRUNCATE TABLE %s", config.DestinationTableIdentifier)).Read(c.ctx) + query := c.client.Query(fmt.Sprintf("TRUNCATE TABLE %s", config.DestinationTableIdentifier)) + query.DefaultDatasetID = c.datasetID + query.DefaultProjectID = c.projectID + _, err = query.Read(c.ctx) if err != nil { return fmt.Errorf("failed to TRUNCATE table before query replication: %w", err) } @@ -172,6 +175,8 @@ func (c *BigQueryConnector) isPartitionSynced(partitionID string) (bool, error) ) query := c.client.Query(queryString) + query.DefaultDatasetID = c.datasetID + query.DefaultProjectID = c.projectID it, err := query.Read(c.ctx) if err != nil { return false, fmt.Errorf("failed to execute query: %w", err) diff --git a/flow/connectors/bigquery/qrep_avro_sync.go b/flow/connectors/bigquery/qrep_avro_sync.go index c9da0d6634..7e6768963a 100644 --- a/flow/connectors/bigquery/qrep_avro_sync.go +++ b/flow/connectors/bigquery/qrep_avro_sync.go @@ -5,17 +5,19 @@ import ( "fmt" "log/slog" "os" + "strconv" "strings" "time" "cloud.google.com/go/bigquery" + "go.temporal.io/sdk/activity" + "github.com/PeerDB-io/peer-flow/connectors/utils" avro "github.com/PeerDB-io/peer-flow/connectors/utils/avro" "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/shared" - "go.temporal.io/sdk/activity" ) type QRepAvroSyncMethod struct { @@ -53,8 +55,8 @@ func (s *QRepAvroSyncMethod) SyncRecords( return nil, fmt.Errorf("failed to define Avro schema: %w", err) } - stagingTable := fmt.Sprintf("%s_%s_staging", rawTableName, fmt.Sprint(syncBatchID)) - numRecords, err := s.writeToStage(fmt.Sprint(syncBatchID), rawTableName, avroSchema, + stagingTable := fmt.Sprintf("%s_%s_staging", rawTableName, strconv.FormatInt(syncBatchID, 10)) + numRecords, err := s.writeToStage(strconv.FormatInt(syncBatchID, 10), rawTableName, avroSchema, &datasetTable{ dataset: s.connector.datasetID, table: stagingTable, @@ -83,8 +85,7 @@ func (s *QRepAvroSyncMethod) SyncRecords( req.FlowJobName, rawTableName, syncBatchID), ) - tableSchemaDeltas := req.Records.WaitForSchemaDeltas(req.TableMappings) - err = s.connector.ReplayTableSchemaDeltas(req.FlowJobName, tableSchemaDeltas) + err = s.connector.ReplayTableSchemaDeltas(req.FlowJobName, req.Records.SchemaDeltas) if err != nil { return nil, fmt.Errorf("failed to sync schema changes: %w", err) } @@ -95,17 +96,20 @@ func (s *QRepAvroSyncMethod) SyncRecords( updateMetadataStmt, "COMMIT TRANSACTION;", } - _, err = bqClient.Query(strings.Join(stmts, "\n")).Read(s.connector.ctx) + query := bqClient.Query(strings.Join(stmts, "\n")) + query.DefaultDatasetID = s.connector.datasetID + query.DefaultProjectID = s.connector.projectID + _, err = query.Read(s.connector.ctx) if err != nil { return nil, fmt.Errorf("failed to execute statements in a transaction: %v", err) } // drop the staging table - if err := bqClient.Dataset(datasetID).Table(stagingTable).Delete(s.connector.ctx); err != nil { + if err := bqClient.DatasetInProject(s.connector.projectID, datasetID).Table(stagingTable).Delete(s.connector.ctx); err != nil { // just log the error this isn't fatal. slog.Error("failed to delete staging table "+stagingTable, slog.Any("error", err), - slog.String("syncBatchID", fmt.Sprint(syncBatchID)), + slog.Int64("syncBatchID", syncBatchID), slog.String("destinationTable", rawTableName)) } @@ -114,12 +118,11 @@ func (s *QRepAvroSyncMethod) SyncRecords( slog.String("dstTableName", rawTableName)) return &model.SyncResponse{ - LastSyncedCheckPointID: lastCP, + LastSyncedCheckpointID: lastCP, NumRecordsSynced: int64(numRecords), CurrentSyncBatchID: syncBatchID, TableNameRowsMapping: tableNameRowsMapping, - TableSchemaDeltas: tableSchemaDeltas, - RelationMessageMapping: <-req.Records.RelationMessageMapping, + TableSchemaDeltas: req.Records.SchemaDeltas, }, nil } @@ -209,13 +212,16 @@ func (s *QRepAvroSyncMethod) SyncQRepRecords( insertMetadataStmt, "COMMIT TRANSACTION;", } - _, err = bqClient.Query(strings.Join(stmts, "\n")).Read(s.connector.ctx) + query := bqClient.Query(strings.Join(stmts, "\n")) + query.DefaultDatasetID = s.connector.datasetID + query.DefaultProjectID = s.connector.projectID + _, err = query.Read(s.connector.ctx) if err != nil { return -1, fmt.Errorf("failed to execute statements in a transaction: %v", err) } // drop the staging table - if err := bqClient.Dataset(stagingDatasetTable.dataset). + if err := bqClient.DatasetInProject(s.connector.projectID, stagingDatasetTable.dataset). Table(stagingDatasetTable.table).Delete(s.connector.ctx); err != nil { // just log the error this isn't fatal. slog.Error("failed to delete staging table "+stagingDatasetTable.string(), @@ -335,7 +341,7 @@ func GetAvroType(bqField *bigquery.FieldSchema) (interface{}, error) { case bigquery.TimeFieldType: return qvalue.AvroSchemaField{ Type: "long", - LogicalType: "timestamp-micros", + LogicalType: "time-micros", }, nil case bigquery.DateTimeFieldType: return qvalue.AvroSchemaRecord{ @@ -458,7 +464,7 @@ func (s *QRepAvroSyncMethod) writeToStage( avroRef = localRef } - loader := bqClient.Dataset(stagingTable.dataset).Table(stagingTable.table).LoaderFrom(avroRef) + loader := bqClient.DatasetInProject(s.connector.projectID, stagingTable.dataset).Table(stagingTable.table).LoaderFrom(avroRef) loader.UseAvroLogicalTypes = true loader.WriteDisposition = bigquery.WriteTruncate job, err := loader.Run(s.connector.ctx) diff --git a/flow/connectors/bigquery/qvalue_convert.go b/flow/connectors/bigquery/qvalue_convert.go index d4e5032182..75fa2971b5 100644 --- a/flow/connectors/bigquery/qvalue_convert.go +++ b/flow/connectors/bigquery/qvalue_convert.go @@ -4,6 +4,7 @@ import ( "fmt" "cloud.google.com/go/bigquery" + "github.com/PeerDB-io/peer-flow/model/qvalue" ) diff --git a/flow/connectors/clickhouse/clickhouse.go b/flow/connectors/clickhouse/clickhouse.go index de51b0feb9..6b3740935a 100644 --- a/flow/connectors/clickhouse/clickhouse.go +++ b/flow/connectors/clickhouse/clickhouse.go @@ -8,6 +8,7 @@ import ( _ "github.com/ClickHouse/clickhouse-go/v2" _ "github.com/ClickHouse/clickhouse-go/v2/lib/driver" + "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/shared" ) @@ -37,8 +38,8 @@ func NewClickhouseConnector(ctx context.Context, } func connect(ctx context.Context, config *protos.ClickhouseConfig) (*sql.DB, error) { - dsn := fmt.Sprintf("tcp://%s:%d?username=%s&password=%s", //&database=%s" - config.Host, config.Port, config.User, config.Password) //, config.Database + dsn := fmt.Sprintf("tcp://%s:%d?username=%s&password=%s", // TODO &database=%s" + config.Host, config.Port, config.User, config.Password) // TODO , config.Database conn, err := sql.Open("clickhouse", dsn) if err != nil { diff --git a/flow/connectors/clickhouse/client.go b/flow/connectors/clickhouse/client.go index 9aa14cd57c..8bd5a0221e 100644 --- a/flow/connectors/clickhouse/client.go +++ b/flow/connectors/clickhouse/client.go @@ -4,10 +4,11 @@ import ( "context" "fmt" + "github.com/jmoiron/sqlx" + peersql "github.com/PeerDB-io/peer-flow/connectors/sql" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model/qvalue" - "github.com/jmoiron/sqlx" ) type ClickhouseClient struct { diff --git a/flow/connectors/clickhouse/qrep.go b/flow/connectors/clickhouse/qrep.go index 74ffe26524..0695d955bf 100644 --- a/flow/connectors/clickhouse/qrep.go +++ b/flow/connectors/clickhouse/qrep.go @@ -7,14 +7,14 @@ import ( "strings" "time" + "github.com/aws/aws-sdk-go-v2/aws" + "github.com/aws/aws-sdk-go-v2/service/s3" + "google.golang.org/protobuf/encoding/protojson" + "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/shared" - "github.com/aws/aws-sdk-go/aws" - "github.com/aws/aws-sdk-go/service/s3" - "github.com/aws/aws-sdk-go/service/s3/s3manager" - "google.golang.org/protobuf/encoding/protojson" ) const qRepMetadataTableName = "_peerdb_query_replication_metadata" @@ -107,7 +107,7 @@ func (c *ClickhouseConnector) isPartitionSynced(partitionID string) (bool, error } func (c *ClickhouseConnector) SetupQRepMetadataTables(config *protos.QRepConfig) error { - err := c.createQRepMetadataTable() //(createMetadataTablesTx) + err := c.createQRepMetadataTable() if err != nil { return err } @@ -122,7 +122,7 @@ func (c *ClickhouseConnector) SetupQRepMetadataTables(config *protos.QRepConfig) return nil } -func (c *ClickhouseConnector) createQRepMetadataTable() error { // createMetadataTableTx *sql.Tx +func (c *ClickhouseConnector) createQRepMetadataTable() error { // Define the schema schemaStatement := ` CREATE TABLE IF NOT EXISTS %s ( @@ -177,16 +177,27 @@ func (c *ClickhouseConnector) dropStage(stagingPath string, job string) error { } // Create a list of all objects with the defined prefix in the bucket - iter := s3manager.NewDeleteListIterator(s3svc, &s3.ListObjectsInput{ + pages := s3.NewListObjectsV2Paginator(s3svc, &s3.ListObjectsV2Input{ Bucket: aws.String(s3o.Bucket), Prefix: aws.String(fmt.Sprintf("%s/%s", s3o.Prefix, job)), }) - - // Iterate through the objects in the bucket with the prefix and delete them - s3Client := s3manager.NewBatchDeleteWithClient(s3svc) - if err := s3Client.Delete(aws.BackgroundContext(), iter); err != nil { - c.logger.Error("failed to delete objects from bucket", slog.Any("error", err)) - return fmt.Errorf("failed to delete objects from bucket: %w", err) + for pages.HasMorePages() { + page, err := pages.NextPage(c.ctx) + if err != nil { + c.logger.Error("failed to list objects from bucket", slog.Any("error", err)) + return fmt.Errorf("failed to list objects from bucket: %w", err) + } + + for _, object := range page.Contents { + _, err = s3svc.DeleteObject(c.ctx, &s3.DeleteObjectInput{ + Bucket: aws.String(s3o.Bucket), + Key: object.Key, + }) + if err != nil { + c.logger.Error("failed to delete objects from bucket", slog.Any("error", err)) + return fmt.Errorf("failed to delete objects from bucket: %w", err) + } + } } c.logger.Info(fmt.Sprintf("Deleted contents of bucket %s with prefix %s/%s", s3o.Bucket, s3o.Prefix, job)) diff --git a/flow/connectors/clickhouse/qrep_avro_sync.go b/flow/connectors/clickhouse/qrep_avro_sync.go index 2adb391457..84c5d2eb89 100644 --- a/flow/connectors/clickhouse/qrep_avro_sync.go +++ b/flow/connectors/clickhouse/qrep_avro_sync.go @@ -6,13 +6,14 @@ import ( "log/slog" "time" + "go.temporal.io/sdk/activity" + "github.com/PeerDB-io/peer-flow/connectors/utils" avro "github.com/PeerDB-io/peer-flow/connectors/utils/avro" "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/shared" - "go.temporal.io/sdk/activity" ) type ClickhouseAvroSyncMethod struct { diff --git a/flow/connectors/core.go b/flow/connectors/core.go index 8abc6f3bf8..1e28822181 100644 --- a/flow/connectors/core.go +++ b/flow/connectors/core.go @@ -6,6 +6,8 @@ import ( "fmt" "log/slog" + "github.com/jackc/pgx/v5/pgxpool" + connbigquery "github.com/PeerDB-io/peer-flow/connectors/bigquery" connclickhouse "github.com/PeerDB-io/peer-flow/connectors/clickhouse" conneventhub "github.com/PeerDB-io/peer-flow/connectors/eventhub" @@ -15,7 +17,6 @@ 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/jackc/pgx/v5/pgxpool" ) var ErrUnsupportedFunctionality = errors.New("requested connector does not support functionality") diff --git a/flow/connectors/eventhub/eventhub.go b/flow/connectors/eventhub/eventhub.go index a0d900db75..f14ca4ac1c 100644 --- a/flow/connectors/eventhub/eventhub.go +++ b/flow/connectors/eventhub/eventhub.go @@ -9,6 +9,7 @@ import ( "time" "github.com/Azure/azure-sdk-for-go/sdk/azidentity" + metadataStore "github.com/PeerDB-io/peer-flow/connectors/external_metadata" "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/generated/protos" @@ -155,7 +156,7 @@ func (c *EventHubConnector) processBatch( numRecords.Add(1) - recordLSN := record.GetCheckPointID() + recordLSN := record.GetCheckpointID() if recordLSN > lastSeenLSN { lastSeenLSN = recordLSN } @@ -185,10 +186,8 @@ func (c *EventHubConnector) processBatch( partitionColumn := destination.PartitionKeyColumn partitionValue := record.GetItems().GetColumnValue(partitionColumn).Value var partitionKey string - if partitionValue == nil { - partitionKey = "" - } else { - partitionKey = fmt.Sprintf("%v", partitionValue) + if partitionValue != nil { + partitionKey = fmt.Sprint(partitionValue) } partitionKey = utils.HashedPartitionKey(partitionKey, numPartitions) destination.SetPartitionValue(partitionKey) @@ -249,19 +248,12 @@ func (c *EventHubConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S return nil, err } - rowsSynced := int64(numRecords) - syncBatchID, err := c.GetLastSyncBatchID(req.FlowJobName) - if err != nil { - c.logger.Error("failed to get last sync batch id", slog.Any("error", err)) - } - return &model.SyncResponse{ - CurrentSyncBatchID: syncBatchID, - LastSyncedCheckPointID: lastCheckpoint, - NumRecordsSynced: rowsSynced, + CurrentSyncBatchID: req.SyncBatchID, + LastSyncedCheckpointID: lastCheckpoint, + NumRecordsSynced: int64(numRecords), TableNameRowsMapping: make(map[string]uint32), - TableSchemaDeltas: req.Records.WaitForSchemaDeltas(req.TableMappings), - RelationMessageMapping: <-req.Records.RelationMessageMapping, + TableSchemaDeltas: req.Records.SchemaDeltas, }, nil } @@ -308,9 +300,5 @@ func (c *EventHubConnector) SetupNormalizedTables( } func (c *EventHubConnector) SyncFlowCleanup(jobName string) error { - err := c.pgMetadata.DropMetadata(jobName) - if err != nil { - return err - } - return nil + return c.pgMetadata.DropMetadata(jobName) } diff --git a/flow/connectors/eventhub/hub_batches.go b/flow/connectors/eventhub/hub_batches.go index 5634173faf..e96d0e6896 100644 --- a/flow/connectors/eventhub/hub_batches.go +++ b/flow/connectors/eventhub/hub_batches.go @@ -9,6 +9,7 @@ import ( "time" azeventhubs "github.com/Azure/azure-sdk-for-go/sdk/messaging/azeventhubs" + "github.com/PeerDB-io/peer-flow/shared" ) diff --git a/flow/connectors/eventhub/hubmanager.go b/flow/connectors/eventhub/hubmanager.go index 3635544d3c..7f6d206728 100644 --- a/flow/connectors/eventhub/hubmanager.go +++ b/flow/connectors/eventhub/hubmanager.go @@ -12,9 +12,10 @@ import ( "github.com/Azure/azure-sdk-for-go/sdk/azidentity" "github.com/Azure/azure-sdk-for-go/sdk/messaging/azeventhubs" "github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/eventhub/armeventhub" + cmap "github.com/orcaman/concurrent-map/v2" + "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/generated/protos" - cmap "github.com/orcaman/concurrent-map/v2" ) type EventHubManager struct { diff --git a/flow/connectors/external_metadata/store.go b/flow/connectors/external_metadata/store.go index b26aacf637..e1858680cf 100644 --- a/flow/connectors/external_metadata/store.go +++ b/flow/connectors/external_metadata/store.go @@ -5,12 +5,14 @@ import ( "fmt" "log/slog" + "github.com/jackc/pgx/v5/pgtype" + "github.com/jackc/pgx/v5/pgxpool" + + connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" "github.com/PeerDB-io/peer-flow/connectors/utils" cc "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/shared" - "github.com/jackc/pgx/v5/pgtype" - "github.com/jackc/pgx/v5/pgxpool" ) const ( @@ -66,6 +68,10 @@ func (p *PostgresMetadataStore) Close() error { return nil } +func (p *PostgresMetadataStore) QualifyTable(table string) string { + return connpostgres.QuoteIdentifier(p.schemaName) + "." + connpostgres.QuoteIdentifier(table) +} + func (p *PostgresMetadataStore) Ping() error { if p.pool == nil { return fmt.Errorf("metadata db ping failed as pool does not exist") @@ -106,7 +112,7 @@ func (p *PostgresMetadataStore) SetupMetadata() error { // create the last sync state table _, err = p.pool.Exec(p.ctx, ` - CREATE TABLE IF NOT EXISTS `+p.schemaName+`.`+lastSyncStateTableName+` ( + CREATE TABLE IF NOT EXISTS `+p.QualifyTable(lastSyncStateTableName)+` ( job_name TEXT PRIMARY KEY NOT NULL, last_offset BIGINT NOT NULL, updated_at TIMESTAMP NOT NULL DEFAULT NOW(), @@ -125,7 +131,7 @@ func (p *PostgresMetadataStore) SetupMetadata() error { func (p *PostgresMetadataStore) FetchLastOffset(jobName string) (int64, error) { rows := p.pool.QueryRow(p.ctx, ` SELECT last_offset - FROM `+p.schemaName+`.`+lastSyncStateTableName+` + FROM `+p.QualifyTable(lastSyncStateTableName)+` WHERE job_name = $1 `, jobName) var offset pgtype.Int8 @@ -147,7 +153,7 @@ func (p *PostgresMetadataStore) FetchLastOffset(jobName string) (int64, error) { func (p *PostgresMetadataStore) GetLastBatchID(jobName string) (int64, error) { rows := p.pool.QueryRow(p.ctx, ` SELECT sync_batch_id - FROM `+p.schemaName+`.`+lastSyncStateTableName+` + FROM `+p.QualifyTable(lastSyncStateTableName)+` WHERE job_name = $1 `, jobName) @@ -179,10 +185,10 @@ func (p *PostgresMetadataStore) UpdateLastOffset(jobName string, offset int64) e // update the last offset p.logger.Info("updating last offset", slog.Int64("offset", offset)) _, err = tx.Exec(p.ctx, ` - INSERT INTO `+p.schemaName+`.`+lastSyncStateTableName+` (job_name, last_offset, sync_batch_id) + INSERT INTO `+p.QualifyTable(lastSyncStateTableName)+` (job_name, last_offset, sync_batch_id) VALUES ($1, $2, $3) ON CONFLICT (job_name) - DO UPDATE SET last_offset = GREATEST(`+lastSyncStateTableName+`.last_offset, excluded.last_offset), + DO UPDATE SET last_offset = GREATEST(`+connpostgres.QuoteIdentifier(lastSyncStateTableName)+`.last_offset, excluded.last_offset), updated_at = NOW() `, jobName, offset, 0) @@ -205,7 +211,7 @@ func (p *PostgresMetadataStore) UpdateLastOffset(jobName string, offset int64) e func (p *PostgresMetadataStore) IncrementID(jobName string) error { p.logger.Info("incrementing sync batch id for job") _, err := p.pool.Exec(p.ctx, ` - UPDATE `+p.schemaName+`.`+lastSyncStateTableName+` + UPDATE `+p.QualifyTable(lastSyncStateTableName)+` SET sync_batch_id=sync_batch_id+1 WHERE job_name=$1 `, jobName) if err != nil { @@ -218,7 +224,7 @@ func (p *PostgresMetadataStore) IncrementID(jobName string) error { func (p *PostgresMetadataStore) DropMetadata(jobName string) error { _, err := p.pool.Exec(p.ctx, ` - DELETE FROM `+p.schemaName+`.`+lastSyncStateTableName+` + DELETE FROM `+p.QualifyTable(lastSyncStateTableName)+` WHERE job_name = $1 `, jobName) return err diff --git a/flow/connectors/postgres/cdc.go b/flow/connectors/postgres/cdc.go index 8de328d0f6..8029b4ae60 100644 --- a/flow/connectors/postgres/cdc.go +++ b/flow/connectors/postgres/cdc.go @@ -3,19 +3,11 @@ package connpostgres import ( "context" "crypto/sha256" - "encoding/json" "fmt" "log/slog" "regexp" "time" - "github.com/PeerDB-io/peer-flow/connectors/utils" - "github.com/PeerDB-io/peer-flow/connectors/utils/cdc_records" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/geo" - "github.com/PeerDB-io/peer-flow/model" - "github.com/PeerDB-io/peer-flow/model/qvalue" - "github.com/PeerDB-io/peer-flow/shared" "github.com/jackc/pglogrepl" "github.com/jackc/pgx/v5" "github.com/jackc/pgx/v5/pgconn" @@ -24,6 +16,14 @@ import ( "github.com/jackc/pgx/v5/pgxpool" "github.com/lib/pq/oid" "go.temporal.io/sdk/activity" + + "github.com/PeerDB-io/peer-flow/connectors/utils" + "github.com/PeerDB-io/peer-flow/connectors/utils/cdc_records" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/geo" + "github.com/PeerDB-io/peer-flow/model" + "github.com/PeerDB-io/peer-flow/model/qvalue" + "github.com/PeerDB-io/peer-flow/shared" ) const maxRetriesForWalSegmentRemoved = 5 @@ -236,7 +236,6 @@ func (p *PostgresCDCSource) consumeStream( if cdcRecordsStorage.IsEmpty() { records.SignalAsEmpty() } - records.RelationMessageMapping <- p.relationMessageMapping p.logger.Info(fmt.Sprintf("[finished] PullRecords streamed %d records", cdcRecordsStorage.Len())) err := cdcRecordsStorage.Close() if err != nil { @@ -508,7 +507,7 @@ func (p *PostgresCDCSource) consumeStream( if len(tableSchemaDelta.AddedColumns) > 0 { p.logger.Info(fmt.Sprintf("Detected schema change for table %s, addedColumns: %v", tableSchemaDelta.SrcTableName, tableSchemaDelta.AddedColumns)) - records.SchemaDeltas <- tableSchemaDelta + records.AddSchemaDelta(req.TableNameMapping, tableSchemaDelta) } } } @@ -600,7 +599,7 @@ func (p *PostgresCDCSource) processInsertMessage( } return &model.InsertRecord{ - CheckPointID: int64(lsn), + CheckpointID: int64(lsn), Items: items, DestinationTableName: p.TableNameMapping[tableName].Name, SourceTableName: tableName, @@ -641,7 +640,7 @@ func (p *PostgresCDCSource) processUpdateMessage( } return &model.UpdateRecord{ - CheckPointID: int64(lsn), + CheckpointID: int64(lsn), OldItems: oldItems, NewItems: newItems, DestinationTableName: p.TableNameMapping[tableName].Name, @@ -678,7 +677,7 @@ func (p *PostgresCDCSource) processDeleteMessage( } return &model.DeleteRecord{ - CheckPointID: int64(lsn), + CheckpointID: int64(lsn), Items: items, DestinationTableName: p.TableNameMapping[tableName].Name, SourceTableName: tableName, @@ -810,16 +809,12 @@ func (p *PostgresCDCSource) auditSchemaDelta(flowJobName string, rec *model.Rela activityInfo := activity.GetInfo(p.ctx) workflowID := activityInfo.WorkflowExecution.ID runID := activityInfo.WorkflowExecution.RunID - recJSON, err := json.Marshal(rec) - if err != nil { - return fmt.Errorf("failed to marshal schema delta to JSON: %w", err) - } - _, err = p.catalogPool.Exec(p.ctx, + _, err := p.catalogPool.Exec(p.ctx, `INSERT INTO peerdb_stats.schema_deltas_audit_log(flow_job_name,workflow_id,run_id,delta_info) VALUES($1,$2,$3,$4)`, - flowJobName, workflowID, runID, recJSON) + flowJobName, workflowID, runID, rec) if err != nil { return fmt.Errorf("failed to insert row into table: %w", err) } @@ -886,7 +881,7 @@ func (p *PostgresCDCSource) processRelationMessage( p.relationMessageMapping[currRel.RelationId] = currRel rec := &model.RelationRecord{ TableSchemaDelta: schemaDelta, - CheckPointID: int64(lsn), + CheckpointID: int64(lsn), } return rec, p.auditSchemaDelta(p.flowJobName, rec) } diff --git a/flow/connectors/postgres/client.go b/flow/connectors/postgres/client.go index 52152cabd5..5ef8f70b7a 100644 --- a/flow/connectors/postgres/client.go +++ b/flow/connectors/postgres/client.go @@ -3,18 +3,18 @@ package connpostgres import ( "errors" "fmt" - "log" "regexp" "strconv" "strings" - "github.com/PeerDB-io/peer-flow/connectors/utils" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/model" "github.com/jackc/pglogrepl" "github.com/jackc/pgx/v5" "github.com/jackc/pgx/v5/pgtype" "github.com/lib/pq/oid" + + "github.com/PeerDB-io/peer-flow/connectors/utils" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/model" ) type PGVersion int @@ -121,13 +121,18 @@ func (c *PostgresConnector) getReplicaIdentityType(schemaTable *utils.SchemaTabl if err != nil { return ReplicaIdentityDefault, fmt.Errorf("error getting replica identity for table %s: %w", schemaTable, err) } + if replicaIdentity == rune(ReplicaIdentityNothing) { + return ReplicaIdentityType(replicaIdentity), fmt.Errorf("table %s has replica identity 'n'/NOTHING", schemaTable) + } return ReplicaIdentityType(replicaIdentity), nil } -// getPrimaryKeyColumns returns the primary key columns for a given table. -// Errors if there is no primary key column or if there is more than one primary key column. -func (c *PostgresConnector) getPrimaryKeyColumns( +// getUniqueColumns returns the unique columns (used to select in MERGE statement) for a given table. +// For replica identity 'd'/default, these are the primary key columns +// For replica identity 'i'/index, these are the columns in the selected index (indisreplident set) +// For replica identity 'f'/full, if there is a primary key we use that, else we return all columns +func (c *PostgresConnector) getUniqueColumns( replicaIdentity ReplicaIdentityType, schemaTable *utils.SchemaTable, ) ([]string, error) { @@ -140,12 +145,16 @@ func (c *PostgresConnector) getPrimaryKeyColumns( return c.getReplicaIdentityIndexColumns(relID, schemaTable) } - // Find the primary key index OID + // Find the primary key index OID, for replica identity 'd'/default or 'f'/full var pkIndexOID oid.Oid err = c.pool.QueryRow(c.ctx, `SELECT indexrelid FROM pg_index WHERE indrelid = $1 AND indisprimary`, relID).Scan(&pkIndexOID) if err != nil { + // don't error out if no pkey index, this would happen in EnsurePullability or UI. + if err == pgx.ErrNoRows { + return []string{}, nil + } return nil, fmt.Errorf("error finding primary key index for table %s: %w", schemaTable, err) } @@ -158,7 +167,7 @@ func (c *PostgresConnector) getReplicaIdentityIndexColumns(relID uint32, schemaT // Fetch the OID of the index used as the replica identity err := c.pool.QueryRow(c.ctx, `SELECT indexrelid FROM pg_index - WHERE indrelid = $1 AND indisreplident = true`, + WHERE indrelid=$1 AND indisreplident=true`, relID).Scan(&indexRelID) if err != nil { return nil, fmt.Errorf("error finding replica identity index for table %s: %w", schemaTable, err) @@ -246,7 +255,7 @@ func (c *PostgresConnector) checkSlotAndPublication(slot string, publication str // 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) { - whereClause := "" + var whereClause string if slotName != "" { whereClause = fmt.Sprintf(" WHERE slot_name = %s", QuoteLiteral(slotName)) } else { @@ -408,18 +417,18 @@ func generateCreateTableSQLForNormalizedTable( ) string { createTableSQLArray := make([]string, 0, utils.TableSchemaColumns(sourceTableSchema)+2) utils.IterColumns(sourceTableSchema, func(columnName, genericColumnType string) { - createTableSQLArray = append(createTableSQLArray, fmt.Sprintf("\"%s\" %s,", columnName, - qValueKindToPostgresType(genericColumnType))) + createTableSQLArray = append(createTableSQLArray, + fmt.Sprintf("%s %s", QuoteIdentifier(columnName), qValueKindToPostgresType(genericColumnType))) }) if softDeleteColName != "" { createTableSQLArray = append(createTableSQLArray, - fmt.Sprintf(`%s BOOL DEFAULT FALSE,`, QuoteIdentifier(softDeleteColName))) + fmt.Sprintf(`%s BOOL DEFAULT FALSE`, QuoteIdentifier(softDeleteColName))) } if syncedAtColName != "" { createTableSQLArray = append(createTableSQLArray, - fmt.Sprintf(`%s TIMESTAMP DEFAULT CURRENT_TIMESTAMP,`, QuoteIdentifier(syncedAtColName))) + fmt.Sprintf(`%s TIMESTAMP DEFAULT CURRENT_TIMESTAMP`, QuoteIdentifier(syncedAtColName))) } // add composite primary key to the table @@ -428,12 +437,11 @@ func generateCreateTableSQLForNormalizedTable( for _, primaryKeyCol := range sourceTableSchema.PrimaryKeyColumns { primaryKeyColsQuoted = append(primaryKeyColsQuoted, QuoteIdentifier(primaryKeyCol)) } - createTableSQLArray = append(createTableSQLArray, fmt.Sprintf("PRIMARY KEY(%s),", - strings.TrimSuffix(strings.Join(primaryKeyColsQuoted, ","), ","))) + createTableSQLArray = append(createTableSQLArray, fmt.Sprintf("PRIMARY KEY(%s)", + strings.Join(primaryKeyColsQuoted, ","))) } - return fmt.Sprintf(createNormalizedTableSQL, sourceTableIdentifier, - strings.TrimSuffix(strings.Join(createTableSQLArray, ""), ",")) + return fmt.Sprintf(createNormalizedTableSQL, sourceTableIdentifier, strings.Join(createTableSQLArray, ",")) } func (c *PostgresConnector) GetLastSyncBatchID(jobName string) (int64, error) { @@ -554,12 +562,12 @@ func (c *PostgresConnector) getTableNametoUnchangedCols(flowJobName string, sync for rows.Next() { err := rows.Scan(&destinationTableName, &unchangedToastColumns) if err != nil { - log.Fatalf("Failed to scan row: %v", err) + return nil, fmt.Errorf("failed to scan row: %w", err) } resultMap[destinationTableName.String] = unchangedToastColumns } if err := rows.Err(); err != nil { - log.Fatalf("Error iterating over rows: %v", err) + return nil, fmt.Errorf("error iterating over rows: %w", err) } return resultMap, nil } @@ -585,26 +593,31 @@ func (c *PostgresConnector) CheckSourceTables(tableNames []string, pubName strin } // Check that we can select from all tables - for _, tableName := range tableNames { + tableArr := make([]string, 0, len(tableNames)) + for _, table := range tableNames { var row pgx.Row - err := c.pool.QueryRow(c.ctx, fmt.Sprintf("SELECT * FROM %s LIMIT 0;", tableName)).Scan(&row) + schemaName, tableName, found := strings.Cut(table, ".") + if !found { + return fmt.Errorf("invalid source table identifier: %s", table) + } + + tableArr = append(tableArr, fmt.Sprintf(`(%s::text, %s::text)`, QuoteLiteral(schemaName), QuoteLiteral(tableName))) + err := c.pool.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 } } // Check if tables belong to publication - tableArr := make([]string, 0, len(tableNames)) - for _, tableName := range tableNames { - tableArr = append(tableArr, fmt.Sprintf("'%s'", tableName)) - } - tableStr := strings.Join(tableArr, ",") - if pubName != "" { var pubTableCount int - err := c.pool.QueryRow(c.ctx, fmt.Sprintf("select COUNT(DISTINCT(schemaname||'.'||tablename)) from pg_publication_tables "+ - "where schemaname||'.'||tablename in (%s) and pubname=$1;", tableStr), pubName).Scan(&pubTableCount) + err := c.pool.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 + ON schemaname=stc.sname and tablename=stc.tname where pubname=$1;`, tableStr), pubName).Scan(&pubTableCount) if err != nil { return err } diff --git a/flow/connectors/postgres/escape.go b/flow/connectors/postgres/escape.go index 280d108338..783c0cd7c7 100644 --- a/flow/connectors/postgres/escape.go +++ b/flow/connectors/postgres/escape.go @@ -21,14 +21,14 @@ func QuoteLiteral(literal string) string { // https://git.postgresql.org/gitweb/?p=postgresql.git;a=blob;f=src/interfaces/libpq/fe-exec.c // // substitute any single-quotes (') with two single-quotes ('') - literal = strings.Replace(literal, `'`, `''`, -1) + literal = strings.ReplaceAll(literal, `'`, `''`) // determine if the string has any backslashes (\) in it. // if it does, replace any backslashes (\) with two backslashes (\\) // then, we need to wrap the entire string with a PostgreSQL // C-style escape. Per how "PQEscapeStringInternal" handles this case, we // also add a space before the "E" if strings.Contains(literal, `\`) { - literal = strings.Replace(literal, `\`, `\\`, -1) + literal = strings.ReplaceAll(literal, `\`, `\\`) literal = ` E'` + literal + `'` } else { // otherwise, we can just wrap the literal with a pair of single quotes @@ -53,5 +53,5 @@ func QuoteIdentifier(name string) string { if end > -1 { name = name[:end] } - return `"` + strings.Replace(name, `"`, `""`, -1) + `"` + return `"` + strings.ReplaceAll(name, `"`, `""`) + `"` } diff --git a/flow/connectors/postgres/normalize_stmt_generator.go b/flow/connectors/postgres/normalize_stmt_generator.go index 083021926a..3792c188af 100644 --- a/flow/connectors/postgres/normalize_stmt_generator.go +++ b/flow/connectors/postgres/normalize_stmt_generator.go @@ -6,10 +6,11 @@ import ( "slices" "strings" + "golang.org/x/exp/maps" + "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model/qvalue" - "golang.org/x/exp/maps" ) type normalizeStmtGenerator struct { diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index a9b9117af6..891e3e2925 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -8,17 +8,18 @@ import ( "strings" "time" + "github.com/google/uuid" + "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/connectors/utils" "github.com/PeerDB-io/peer-flow/connectors/utils/monitoring" "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/shared" - "github.com/google/uuid" - "github.com/jackc/pgx/v5" - "github.com/jackc/pgx/v5/pgconn" - "github.com/jackc/pgx/v5/pgtype" - "github.com/jackc/pgx/v5/pgxpool" ) // PostgresConnector is a Connector implementation for Postgres. @@ -272,11 +273,6 @@ func (c *PostgresConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S rawTableIdentifier := getRawTableIdentifier(req.FlowJobName) c.logger.Info(fmt.Sprintf("pushing records to Postgres table %s via COPY", rawTableIdentifier)) - syncBatchID, err := c.GetLastSyncBatchID(req.FlowJobName) - if err != nil { - return nil, fmt.Errorf("failed to get previous syncBatchID: %w", err) - } - syncBatchID += 1 records := make([][]interface{}, 0) tableNameRowsMapping := make(map[string]uint32) @@ -298,7 +294,7 @@ func (c *PostgresConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S itemsJSON, 0, "{}", - syncBatchID, + req.SyncBatchID, "", }) tableNameRowsMapping[typedRecord.DestinationTableName] += 1 @@ -325,7 +321,7 @@ func (c *PostgresConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S newItemsJSON, 1, oldItemsJSON, - syncBatchID, + req.SyncBatchID, utils.KeysToString(typedRecord.UnchangedToastColumns), }) tableNameRowsMapping[typedRecord.DestinationTableName] += 1 @@ -345,7 +341,7 @@ func (c *PostgresConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S itemsJSON, 2, itemsJSON, - syncBatchID, + req.SyncBatchID, "", }) tableNameRowsMapping[typedRecord.DestinationTableName] += 1 @@ -354,15 +350,14 @@ func (c *PostgresConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S } } - tableSchemaDeltas := req.Records.WaitForSchemaDeltas(req.TableMappings) - err = c.ReplayTableSchemaDeltas(req.FlowJobName, tableSchemaDeltas) + err := c.ReplayTableSchemaDeltas(req.FlowJobName, req.Records.SchemaDeltas) if err != nil { return nil, fmt.Errorf("failed to sync schema changes: %w", err) } if len(records) == 0 { return &model.SyncResponse{ - LastSyncedCheckPointID: 0, + LastSyncedCheckpointID: 0, NumRecordsSynced: 0, TableSchemaDeltas: tableSchemaDeltas, RelationMessageMapping: <-req.Records.RelationMessageMapping, @@ -403,7 +398,7 @@ func (c *PostgresConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S } // updating metadata with new offset and syncBatchID - err = c.updateSyncMetadata(req.FlowJobName, lastCP, syncBatchID, syncRecordsTx) + err = c.updateSyncMetadata(req.FlowJobName, lastCP, req.SyncBatchID, syncRecordsTx) if err != nil { return nil, err } @@ -414,12 +409,11 @@ func (c *PostgresConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S } return &model.SyncResponse{ - LastSyncedCheckPointID: lastCP, + LastSyncedCheckpointID: lastCP, NumRecordsSynced: int64(len(records)), - CurrentSyncBatchID: syncBatchID, + CurrentSyncBatchID: req.SyncBatchID, TableNameRowsMapping: tableNameRowsMapping, - TableSchemaDeltas: tableSchemaDeltas, - RelationMessageMapping: <-req.Records.RelationMessageMapping, + TableSchemaDeltas: req.Records.SchemaDeltas, }, nil } @@ -587,7 +581,7 @@ func (c *PostgresConnector) GetTableSchema( ) (*protos.GetTableSchemaBatchOutput, error) { res := make(map[string]*protos.TableSchema) for _, tableName := range req.TableIdentifiers { - tableSchema, err := c.getTableSchemaForTable(tableName, req.SkipPkeyAndReplicaCheck) + tableSchema, err := c.getTableSchemaForTable(tableName) if err != nil { return nil, err } @@ -603,27 +597,19 @@ func (c *PostgresConnector) GetTableSchema( func (c *PostgresConnector) getTableSchemaForTable( tableName string, - skipPkeyAndReplicaCheck bool, ) (*protos.TableSchema, error) { schemaTable, err := utils.ParseSchemaTable(tableName) if err != nil { return nil, err } - var pKeyCols []string - var replicaIdentityType ReplicaIdentityType - if !skipPkeyAndReplicaCheck { - var replErr error - replicaIdentityType, replErr = c.getReplicaIdentityType(schemaTable) - if replErr != nil { - return nil, fmt.Errorf("[getTableSchema]:error getting replica identity for table %s: %w", schemaTable, replErr) - } - - var err error - pKeyCols, err = c.getPrimaryKeyColumns(replicaIdentityType, schemaTable) - if err != nil { - return nil, fmt.Errorf("[getTableSchema]:error getting primary key column for table %s: %w", schemaTable, err) - } + replicaIdentityType, err := c.getReplicaIdentityType(schemaTable) + if err != nil { + return nil, fmt.Errorf("[getTableSchema] error getting replica identity for table %s: %w", schemaTable, err) + } + pKeyCols, err := c.getUniqueColumns(replicaIdentityType, schemaTable) + if err != nil { + return nil, fmt.Errorf("[getTableSchema] error getting primary key column for table %s: %w", schemaTable, err) } // Get the column names and types @@ -656,6 +642,10 @@ func (c *PostgresConnector) getTableSchemaForTable( if err = rows.Err(); err != nil { return nil, fmt.Errorf("error iterating over table schema: %w", err) } + // if we have no pkey, we will use all columns as the pkey for the MERGE statement + if replicaIdentityType == ReplicaIdentityFull && len(pKeyCols) == 0 { + pKeyCols = columnNames + } return &protos.TableSchema{ TableIdentifier: tableName, @@ -751,8 +741,8 @@ func (c *PostgresConnector) ReplayTableSchemaDeltas( for _, addedColumn := range schemaDelta.AddedColumns { _, err = tableSchemaModifyTx.Exec(c.ctx, fmt.Sprintf( - "ALTER TABLE %s ADD COLUMN IF NOT EXISTS \"%s\" %s", - schemaDelta.DstTableName, addedColumn.ColumnName, + "ALTER TABLE %s ADD COLUMN IF NOT EXISTS %s %s", + schemaDelta.DstTableName, QuoteIdentifier(addedColumn.ColumnName), qValueKindToPostgresType(addedColumn.ColumnType))) if err != nil { return fmt.Errorf("failed to add column %s for table %s: %w", addedColumn.ColumnName, @@ -807,14 +797,14 @@ func (c *PostgresConnector) EnsurePullability( return nil, fmt.Errorf("error getting replica identity for table %s: %w", schemaTable, replErr) } - pKeyCols, err := c.getPrimaryKeyColumns(replicaIdentity, schemaTable) + pKeyCols, err := c.getUniqueColumns(replicaIdentity, schemaTable) if err != nil { return nil, fmt.Errorf("error getting primary key column for table %s: %w", schemaTable, err) } // we only allow no primary key if the table has REPLICA IDENTITY FULL // this is ok for replica identity index as we populate the primary key columns - if len(pKeyCols) == 0 && !(replicaIdentity == ReplicaIdentityFull) { + if len(pKeyCols) == 0 && replicaIdentity != ReplicaIdentityFull { return nil, fmt.Errorf("table %s has no primary keys and does not have REPLICA IDENTITY FULL", schemaTable) } diff --git a/flow/connectors/postgres/postgres_schema_delta_test.go b/flow/connectors/postgres/postgres_schema_delta_test.go index 8b72b46f20..4c3b012243 100644 --- a/flow/connectors/postgres/postgres_schema_delta_test.go +++ b/flow/connectors/postgres/postgres_schema_delta_test.go @@ -6,13 +6,14 @@ import ( "strings" "testing" + "github.com/jackc/pgx/v5" + "github.com/stretchr/testify/require" + "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/e2eshared" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/PeerDB-io/peer-flow/shared" - "github.com/jackc/pgx/v5" - "github.com/stretchr/testify/require" ) type PostgresSchemaDeltaTestSuite struct { @@ -250,9 +251,9 @@ func TestPostgresSchemaDeltaTestSuite(t *testing.T) { err = teardownTx.Commit(context.Background()) require.NoError(s.t, err) - require.True(s.t, s.connector.ConnectionActive() == nil) + require.NoError(s.t, s.connector.ConnectionActive()) err = s.connector.Close() require.NoError(s.t, err) - require.False(s.t, s.connector.ConnectionActive() == nil) + require.Error(s.t, s.connector.ConnectionActive()) }) } diff --git a/flow/connectors/postgres/qrep.go b/flow/connectors/postgres/qrep.go index f34e0a13bd..fc706c920a 100644 --- a/flow/connectors/postgres/qrep.go +++ b/flow/connectors/postgres/qrep.go @@ -8,14 +8,15 @@ import ( "text/template" "time" + "github.com/google/uuid" + "github.com/jackc/pgx/v5" + "github.com/jackc/pgx/v5/pgtype" + "github.com/PeerDB-io/peer-flow/connectors/utils" partition_utils "github.com/PeerDB-io/peer-flow/connectors/utils/partition" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/shared" - "github.com/google/uuid" - "github.com/jackc/pgx/v5" - "github.com/jackc/pgx/v5/pgtype" ) const qRepMetadataTableName = "_peerdb_query_replication_metadata" @@ -83,7 +84,7 @@ func (c *PostgresConnector) getNumRowsPartitions( ) ([]*protos.QRepPartition, error) { var err error numRowsPerPartition := int64(config.NumRowsPerPartition) - quotedWatermarkColumn := fmt.Sprintf("\"%s\"", config.WatermarkColumn) + quotedWatermarkColumn := QuoteIdentifier(config.WatermarkColumn) whereClause := "" if last != nil && last.Range != nil { @@ -197,7 +198,7 @@ func (c *PostgresConnector) getMinMaxValues( last *protos.QRepPartition, ) (interface{}, interface{}, error) { var minValue, maxValue interface{} - quotedWatermarkColumn := fmt.Sprintf("\"%s\"", config.WatermarkColumn) + quotedWatermarkColumn := QuoteIdentifier(config.WatermarkColumn) parsedWatermarkTable, err := utils.ParseSchemaTable(config.WatermarkTable) if err != nil { diff --git a/flow/connectors/postgres/qrep_partition_test.go b/flow/connectors/postgres/qrep_partition_test.go index 73278ce531..f99b248ec4 100644 --- a/flow/connectors/postgres/qrep_partition_test.go +++ b/flow/connectors/postgres/qrep_partition_test.go @@ -7,10 +7,11 @@ import ( "testing" "time" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/shared" "github.com/jackc/pgx/v5/pgxpool" "github.com/stretchr/testify/assert" + + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/shared" ) type testCase struct { @@ -184,7 +185,7 @@ func TestGetQRepPartitions(t *testing.T) { // for now, but ideally we should check that the partition ranges // are correct as well. if tc.expectedNumPartitions != 0 { - assert.Equal(t, tc.expectedNumPartitions, len(got)) + assert.Len(t, got, tc.expectedNumPartitions) return } diff --git a/flow/connectors/postgres/qrep_query_executor.go b/flow/connectors/postgres/qrep_query_executor.go index 506668c9be..b45d291957 100644 --- a/flow/connectors/postgres/qrep_query_executor.go +++ b/flow/connectors/postgres/qrep_query_executor.go @@ -5,17 +5,17 @@ import ( "fmt" "log/slog" - "github.com/PeerDB-io/peer-flow/connectors/utils" - "github.com/PeerDB-io/peer-flow/geo" - "github.com/PeerDB-io/peer-flow/model" - "github.com/PeerDB-io/peer-flow/model/qvalue" - "github.com/PeerDB-io/peer-flow/shared" "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" + "github.com/PeerDB-io/peer-flow/geo" + "github.com/PeerDB-io/peer-flow/model" + "github.com/PeerDB-io/peer-flow/model/qvalue" + "github.com/PeerDB-io/peer-flow/shared" ) type QRepQueryExecutor struct { @@ -130,7 +130,7 @@ func (qe *QRepQueryExecutor) ProcessRows( fieldDescriptions []pgconn.FieldDescription, ) (*model.QRecordBatch, error) { // Initialize the record slice - records := make([]model.QRecord, 0) + records := make([][]qvalue.QValue, 0) qe.logger.Info("Processing rows") // Iterate over the rows for rows.Next() { @@ -148,12 +148,11 @@ func (qe *QRepQueryExecutor) ProcessRows( } batch := &model.QRecordBatch{ - NumRecords: uint32(len(records)), - Records: records, - Schema: qe.fieldDescriptionsToSchema(fieldDescriptions), + Records: records, + Schema: qe.fieldDescriptionsToSchema(fieldDescriptions), } - qe.logger.Info(fmt.Sprintf("[postgres] pulled %d records", batch.NumRecords)) + qe.logger.Info(fmt.Sprintf("[postgres] pulled %d records", len(batch.Records))) return batch, nil } @@ -283,9 +282,8 @@ func (qe *QRepQueryExecutor) ExecuteAndProcessQuery( return nil, fmt.Errorf("failed to get schema from stream: %w", schema.Err) } batch := &model.QRecordBatch{ - NumRecords: 0, - Records: make([]model.QRecord, 0), - Schema: schema.Schema, + Records: make([][]qvalue.QValue, 0), + Schema: schema.Schema, } for record := range stream.Records { if record.Err == nil { @@ -294,7 +292,6 @@ func (qe *QRepQueryExecutor) ExecuteAndProcessQuery( return nil, fmt.Errorf("[pg] failed to get record from stream: %w", record.Err) } } - batch.NumRecords = uint32(len(batch.Records)) return batch, nil } } @@ -437,14 +434,14 @@ func (qe *QRepQueryExecutor) ExecuteAndProcessQueryStreamWithTx( func mapRowToQRecord(row pgx.Rows, fds []pgconn.FieldDescription, customTypeMap map[uint32]string, -) (model.QRecord, error) { +) ([]qvalue.QValue, error) { // make vals an empty array of QValue of size len(fds) - record := model.NewQRecord(len(fds)) + record := make([]qvalue.QValue, len(fds)) values, err := row.Values() if err != nil { slog.Error("[pg_query_executor] failed to get values from row", slog.Any("error", err)) - return model.QRecord{}, fmt.Errorf("failed to scan row: %w", err) + return nil, fmt.Errorf("failed to scan row: %w", err) } for i, fd := range fds { @@ -454,9 +451,9 @@ func mapRowToQRecord(row pgx.Rows, fds []pgconn.FieldDescription, tmp, err := parseFieldFromPostgresOID(fd.DataTypeOID, values[i]) if err != nil { slog.Error("[pg_query_executor] failed to parse field", slog.Any("error", err)) - return model.QRecord{}, fmt.Errorf("failed to parse field: %w", err) + return nil, fmt.Errorf("failed to parse field: %w", err) } - record.Set(i, tmp) + record[i] = tmp } else { customQKind := customTypeToQKind(typeName) if customQKind == qvalue.QValueKindGeography || customQKind == qvalue.QValueKindGeometry { @@ -468,11 +465,10 @@ func mapRowToQRecord(row pgx.Rows, fds []pgconn.FieldDescription, values[i] = wkt } } - customTypeVal := qvalue.QValue{ + record[i] = qvalue.QValue{ Kind: customQKind, Value: values[i], } - record.Set(i, customTypeVal) } } diff --git a/flow/connectors/postgres/qrep_query_executor_test.go b/flow/connectors/postgres/qrep_query_executor_test.go index db5a04d93f..fc336e2070 100644 --- a/flow/connectors/postgres/qrep_query_executor_test.go +++ b/flow/connectors/postgres/qrep_query_executor_test.go @@ -95,8 +95,8 @@ func TestExecuteAndProcessQuery(t *testing.T) { t.Fatalf("expected 1 record, got %v", len(batch.Records)) } - if batch.Records[0].Entries[1].Value != "testdata" { - t.Fatalf("expected 'testdata', got %v", batch.Records[0].Entries[0].Value) + if batch.Records[0][1].Value != "testdata" { + t.Fatalf("expected 'testdata', got %v", batch.Records[0][0].Value) } } @@ -212,52 +212,52 @@ func TestAllDataTypes(t *testing.T) { record := batch.Records[0] expectedBool := true - if record.Entries[0].Value.(bool) != expectedBool { - t.Fatalf("expected %v, got %v", expectedBool, record.Entries[0].Value) + if record[0].Value.(bool) != expectedBool { + t.Fatalf("expected %v, got %v", expectedBool, record[0].Value) } expectedInt4 := int32(2) - if record.Entries[1].Value.(int32) != expectedInt4 { - t.Fatalf("expected %v, got %v", expectedInt4, record.Entries[1].Value) + if record[1].Value.(int32) != expectedInt4 { + t.Fatalf("expected %v, got %v", expectedInt4, record[1].Value) } expectedInt8 := int64(3) - if record.Entries[2].Value.(int64) != expectedInt8 { - t.Fatalf("expected %v, got %v", expectedInt8, record.Entries[2].Value) + if record[2].Value.(int64) != expectedInt8 { + t.Fatalf("expected %v, got %v", expectedInt8, record[2].Value) } expectedFloat4 := float32(1.1) - if record.Entries[3].Value.(float32) != expectedFloat4 { - t.Fatalf("expected %v, got %v", expectedFloat4, record.Entries[3].Value) + if record[3].Value.(float32) != expectedFloat4 { + t.Fatalf("expected %v, got %v", expectedFloat4, record[3].Value) } expectedFloat8 := float64(2.2) - if record.Entries[4].Value.(float64) != expectedFloat8 { - t.Fatalf("expected %v, got %v", expectedFloat8, record.Entries[4].Value) + if record[4].Value.(float64) != expectedFloat8 { + t.Fatalf("expected %v, got %v", expectedFloat8, record[4].Value) } expectedText := "text" - if record.Entries[5].Value.(string) != expectedText { - t.Fatalf("expected %v, got %v", expectedText, record.Entries[5].Value) + if record[5].Value.(string) != expectedText { + t.Fatalf("expected %v, got %v", expectedText, record[5].Value) } expectedBytea := []byte("bytea") - if !bytes.Equal(record.Entries[6].Value.([]byte), expectedBytea) { - t.Fatalf("expected %v, got %v", expectedBytea, record.Entries[6].Value) + if !bytes.Equal(record[6].Value.([]byte), expectedBytea) { + t.Fatalf("expected %v, got %v", expectedBytea, record[6].Value) } expectedJSON := `{"key":"value"}` - if record.Entries[7].Value.(string) != expectedJSON { - t.Fatalf("expected %v, got %v", expectedJSON, record.Entries[7].Value) + if record[7].Value.(string) != expectedJSON { + t.Fatalf("expected %v, got %v", expectedJSON, record[7].Value) } - actualUUID := record.Entries[8].Value.([16]uint8) + actualUUID := record[8].Value.([16]uint8) if !bytes.Equal(actualUUID[:], savedUUID[:]) { t.Fatalf("expected %v, got %v", savedUUID, actualUUID) } expectedNumeric := "123.456" - actualNumeric := record.Entries[10].Value.(*big.Rat).FloatString(3) + actualNumeric := record[10].Value.(*big.Rat).FloatString(3) if actualNumeric != expectedNumeric { t.Fatalf("expected %v, got %v", expectedNumeric, actualNumeric) } diff --git a/flow/connectors/postgres/qrep_sql_sync.go b/flow/connectors/postgres/qrep_sql_sync.go index b8bcf38687..e75e38a970 100644 --- a/flow/connectors/postgres/qrep_sql_sync.go +++ b/flow/connectors/postgres/qrep_sql_sync.go @@ -7,14 +7,14 @@ import ( "strings" "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" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/shared" - "github.com/jackc/pgx/v5" - "github.com/jackc/pgx/v5/pgxpool" - - "google.golang.org/protobuf/encoding/protojson" ) type QRepSyncMethod interface { diff --git a/flow/connectors/postgres/qvalue_convert.go b/flow/connectors/postgres/qvalue_convert.go index d357495810..d80baf69b1 100644 --- a/flow/connectors/postgres/qvalue_convert.go +++ b/flow/connectors/postgres/qvalue_convert.go @@ -10,9 +10,10 @@ import ( "strings" "time" - "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/jackc/pgx/v5/pgtype" "github.com/lib/pq/oid" + + "github.com/PeerDB-io/peer-flow/model/qvalue" ) func postgresOIDToQValueKind(recvOID uint32) qvalue.QValueKind { @@ -262,8 +263,7 @@ func parseFieldFromQValueKind(qvalueKind qvalue.QValueKind, value interface{}) ( val = qvalue.QValue{Kind: qvalue.QValueKindFloat64, Value: floatVal} case qvalue.QValueKindString: // handling all unsupported types with strings as well for now. - textVal := value - val = qvalue.QValue{Kind: qvalue.QValueKindString, Value: fmt.Sprint(textVal)} + val = qvalue.QValue{Kind: qvalue.QValueKindString, Value: fmt.Sprint(value)} case qvalue.QValueKindUUID: switch value.(type) { case string: diff --git a/flow/connectors/postgres/ssh_wrapped_pool.go b/flow/connectors/postgres/ssh_wrapped_pool.go index a82356a721..4f17116ea4 100644 --- a/flow/connectors/postgres/ssh_wrapped_pool.go +++ b/flow/connectors/postgres/ssh_wrapped_pool.go @@ -8,10 +8,11 @@ import ( "sync" "time" - "github.com/PeerDB-io/peer-flow/connectors/utils" - "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/jackc/pgx/v5/pgxpool" "golang.org/x/crypto/ssh" + + "github.com/PeerDB-io/peer-flow/connectors/utils" + "github.com/PeerDB-io/peer-flow/generated/protos" ) type SSHWrappedPostgresPool struct { @@ -39,11 +40,7 @@ func NewSSHWrappedPostgresPool( if sshConfig != nil { sshServer = fmt.Sprintf("%s:%d", sshConfig.Host, sshConfig.Port) var err error - clientConfig, err = utils.GetSSHClientConfig( - sshConfig.User, - sshConfig.Password, - sshConfig.PrivateKey, - ) + clientConfig, err = utils.GetSSHClientConfig(sshConfig) if err != nil { slog.Error("Failed to get SSH client config", slog.Any("error", err)) cancel() @@ -137,18 +134,21 @@ func (swpp *SSHWrappedPostgresPool) Close() { type retryFunc func() error -func retryWithBackoff(fn retryFunc, maxRetries int, backoff time.Duration) (err error) { - for i := 0; i < maxRetries; i++ { - err = fn() +func retryWithBackoff(fn retryFunc, maxRetries int, backoff time.Duration) error { + i := 0 + for { + err := fn() if err == nil { return nil } - if i < maxRetries-1 { + i += 1 + if i < maxRetries { slog.Info(fmt.Sprintf("Attempt #%d failed, retrying in %s", i+1, backoff)) time.Sleep(backoff) + } else { + return err } } - return err } // see: https://github.com/jackc/pgx/issues/382#issuecomment-1496586216 diff --git a/flow/connectors/s3/s3.go b/flow/connectors/s3/s3.go index 7933936bd7..19e8cbafe4 100644 --- a/flow/connectors/s3/s3.go +++ b/flow/connectors/s3/s3.go @@ -4,16 +4,18 @@ import ( "context" "fmt" "log/slog" + "strconv" "strings" "time" + "github.com/aws/aws-sdk-go-v2/aws" + "github.com/aws/aws-sdk-go-v2/service/s3" + metadataStore "github.com/PeerDB-io/peer-flow/connectors/external_metadata" "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/shared" - "github.com/aws/aws-sdk-go/aws" - "github.com/aws/aws-sdk-go/service/s3" ) const ( @@ -24,7 +26,7 @@ type S3Connector struct { ctx context.Context url string pgMetadata *metadataStore.PostgresMetadataStore - client s3.S3 + client s3.Client creds utils.S3PeerCredentials logger slog.Logger } @@ -91,8 +93,8 @@ func (c *S3Connector) Close() error { return c.pgMetadata.Close() } -func ValidCheck(s3Client *s3.S3, bucketURL string, metadataDB *metadataStore.PostgresMetadataStore) error { - _, listErr := s3Client.ListBuckets(nil) +func ValidCheck(ctx context.Context, s3Client *s3.Client, bucketURL string, metadataDB *metadataStore.PostgresMetadataStore) error { + _, listErr := s3Client.ListBuckets(ctx, nil) if listErr != nil { return fmt.Errorf("failed to list buckets: %w", listErr) } @@ -107,7 +109,7 @@ func ValidCheck(s3Client *s3.S3, bucketURL string, metadataDB *metadataStore.Pos // Write an empty file and then delete it // to check if we have write permissions bucketName := aws.String(bucketPrefix.Bucket) - _, putErr := s3Client.PutObject(&s3.PutObjectInput{ + _, putErr := s3Client.PutObject(ctx, &s3.PutObjectInput{ Bucket: bucketName, Key: aws.String(_peerDBCheck), Body: reader, @@ -116,7 +118,7 @@ func ValidCheck(s3Client *s3.S3, bucketURL string, metadataDB *metadataStore.Pos return fmt.Errorf("failed to write to bucket: %w", putErr) } - _, delErr := s3Client.DeleteObject(&s3.DeleteObjectInput{ + _, delErr := s3Client.DeleteObject(ctx, &s3.DeleteObjectInput{ Bucket: bucketName, Key: aws.String(_peerDBCheck), }) @@ -134,12 +136,12 @@ func ValidCheck(s3Client *s3.S3, bucketURL string, metadataDB *metadataStore.Pos } func (c *S3Connector) ConnectionActive() error { - _, listErr := c.client.ListBuckets(nil) + _, listErr := c.client.ListBuckets(c.ctx, nil) if listErr != nil { return listErr } - validErr := ValidCheck(&c.client, c.url, c.pgMetadata) + validErr := ValidCheck(c.ctx, &c.client, c.url, c.pgMetadata) if validErr != nil { c.logger.Error("failed to validate s3 connector:", slog.Any("error", validErr)) return validErr @@ -182,14 +184,8 @@ func (c *S3Connector) SetLastOffset(jobName string, offset int64) error { } func (c *S3Connector) SyncRecords(req *model.SyncRecordsRequest) (*model.SyncResponse, error) { - syncBatchID, err := c.GetLastSyncBatchID(req.FlowJobName) - if err != nil { - return nil, fmt.Errorf("failed to get previous syncBatchID: %w", err) - } - syncBatchID += 1 - tableNameRowsMapping := make(map[string]uint32) - streamReq := model.NewRecordsToStreamRequest(req.Records.GetRecords(), tableNameRowsMapping, syncBatchID) + streamReq := model.NewRecordsToStreamRequest(req.Records.GetRecords(), tableNameRowsMapping, req.SyncBatchID) streamRes, err := utils.RecordsToRawTableStream(streamReq) if err != nil { return nil, fmt.Errorf("failed to convert records to raw table stream: %w", err) @@ -200,7 +196,7 @@ func (c *S3Connector) SyncRecords(req *model.SyncRecordsRequest) (*model.SyncRes DestinationTableIdentifier: fmt.Sprintf("raw_table_%s", req.FlowJobName), } partition := &protos.QRepPartition{ - PartitionId: fmt.Sprint(syncBatchID), + PartitionId: strconv.FormatInt(req.SyncBatchID, 10), } numRecords, err := c.SyncQRepRecords(qrepConfig, partition, recordStream) if err != nil { @@ -225,11 +221,10 @@ func (c *S3Connector) SyncRecords(req *model.SyncRecordsRequest) (*model.SyncRes } return &model.SyncResponse{ - LastSyncedCheckPointID: lastCheckpoint, + LastSyncedCheckpointID: lastCheckpoint, NumRecordsSynced: int64(numRecords), TableNameRowsMapping: tableNameRowsMapping, - TableSchemaDeltas: req.Records.WaitForSchemaDeltas(req.TableMappings), - RelationMessageMapping: <-req.Records.RelationMessageMapping, + TableSchemaDeltas: req.Records.SchemaDeltas, }, nil } @@ -247,9 +242,5 @@ func (c *S3Connector) SetupNormalizedTables(req *protos.SetupNormalizedTableBatc } func (c *S3Connector) SyncFlowCleanup(jobName string) error { - err := c.pgMetadata.DropMetadata(jobName) - if err != nil { - return err - } - return nil + return c.pgMetadata.DropMetadata(jobName) } diff --git a/flow/connectors/snowflake/avro_file_writer_test.go b/flow/connectors/snowflake/avro_file_writer_test.go index 8a5753680c..7dca611828 100644 --- a/flow/connectors/snowflake/avro_file_writer_test.go +++ b/flow/connectors/snowflake/avro_file_writer_test.go @@ -8,11 +8,12 @@ import ( "testing" "time" + "github.com/google/uuid" + "github.com/stretchr/testify/require" + avro "github.com/PeerDB-io/peer-flow/connectors/utils/avro" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" - "github.com/google/uuid" - "github.com/stretchr/testify/require" ) // createQValue creates a QValue of the appropriate kind for a given placeholder. @@ -96,9 +97,8 @@ func generateRecords( // Create sample records records := &model.QRecordBatch{ - NumRecords: numRows, - Records: make([]model.QRecord, numRows), - Schema: schema, + Records: make([][]qvalue.QValue, numRows), + Schema: schema, } for i, kind := range allQValueKinds { @@ -120,9 +120,7 @@ func generateRecords( } } - records.Records[row] = model.QRecord{ - Entries: entries, - } + records.Records[row] = entries } stream, err := records.ToQRecordStream(1024) diff --git a/flow/connectors/snowflake/client.go b/flow/connectors/snowflake/client.go index 089965d573..82a09cb1ed 100644 --- a/flow/connectors/snowflake/client.go +++ b/flow/connectors/snowflake/client.go @@ -6,6 +6,7 @@ import ( "strings" "time" + "github.com/jackc/pgx/v5/pgtype" "github.com/jmoiron/sqlx" "github.com/snowflakedb/gosnowflake" @@ -14,7 +15,6 @@ import ( "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/PeerDB-io/peer-flow/shared" - "github.com/jackc/pgx/v5/pgtype" ) type SnowflakeClient struct { diff --git a/flow/connectors/snowflake/merge_stmt_generator.go b/flow/connectors/snowflake/merge_stmt_generator.go index a26d981b9b..291b3314d9 100644 --- a/flow/connectors/snowflake/merge_stmt_generator.go +++ b/flow/connectors/snowflake/merge_stmt_generator.go @@ -41,31 +41,31 @@ func (m *mergeStmtGenerator) generateMergeStmt() (string, error) { switch qvalue.QValueKind(genericColumnType) { case qvalue.QValueKindBytes, qvalue.QValueKindBit: flattenedCastsSQLArray = append(flattenedCastsSQLArray, fmt.Sprintf("BASE64_DECODE_BINARY(%s:\"%s\") "+ - "AS %s,", toVariantColumnName, columnName, targetColumnName)) + "AS %s", toVariantColumnName, columnName, targetColumnName)) case qvalue.QValueKindGeography: flattenedCastsSQLArray = append(flattenedCastsSQLArray, - fmt.Sprintf("TO_GEOGRAPHY(CAST(%s:\"%s\" AS STRING),true) AS %s,", + fmt.Sprintf("TO_GEOGRAPHY(CAST(%s:\"%s\" AS STRING),true) AS %s", toVariantColumnName, columnName, targetColumnName)) case qvalue.QValueKindGeometry: flattenedCastsSQLArray = append(flattenedCastsSQLArray, - fmt.Sprintf("TO_GEOMETRY(CAST(%s:\"%s\" AS STRING),true) AS %s,", + fmt.Sprintf("TO_GEOMETRY(CAST(%s:\"%s\" AS STRING),true) AS %s", toVariantColumnName, columnName, targetColumnName)) case qvalue.QValueKindJSON, qvalue.QValueKindHStore: flattenedCastsSQLArray = append(flattenedCastsSQLArray, - fmt.Sprintf("PARSE_JSON(CAST(%s:\"%s\" AS STRING)) AS %s,", + fmt.Sprintf("PARSE_JSON(CAST(%s:\"%s\" AS STRING)) AS %s", toVariantColumnName, columnName, targetColumnName)) // TODO: https://github.com/PeerDB-io/peerdb/issues/189 - handle time types and interval types // case model.ColumnTypeTime: // flattenedCastsSQLArray = append(flattenedCastsSQLArray, fmt.Sprintf("TIME_FROM_PARTS(0,0,0,%s:%s:"+ // "Microseconds*1000) "+ - // "AS %s,", toVariantColumnName, columnName, columnName)) + // "AS %s", toVariantColumnName, columnName, columnName)) default: if qvKind == qvalue.QValueKindNumeric { flattenedCastsSQLArray = append(flattenedCastsSQLArray, - fmt.Sprintf("TRY_CAST((%s:\"%s\")::text AS %s) AS %s,", + fmt.Sprintf("TRY_CAST((%s:\"%s\")::text AS %s) AS %s", toVariantColumnName, columnName, sfType, targetColumnName)) } else { - flattenedCastsSQLArray = append(flattenedCastsSQLArray, fmt.Sprintf("CAST(%s:\"%s\" AS %s) AS %s,", + flattenedCastsSQLArray = append(flattenedCastsSQLArray, fmt.Sprintf("CAST(%s:\"%s\" AS %s) AS %s", toVariantColumnName, columnName, sfType, targetColumnName)) } } @@ -74,7 +74,7 @@ func (m *mergeStmtGenerator) generateMergeStmt() (string, error) { if err != nil { return "", err } - flattenedCastsSQL := strings.TrimSuffix(strings.Join(flattenedCastsSQLArray, ""), ",") + flattenedCastsSQL := strings.Join(flattenedCastsSQLArray, ",") quotedUpperColNames := make([]string, 0, len(columnNames)) for _, columnName := range columnNames { @@ -85,7 +85,7 @@ func (m *mergeStmtGenerator) generateMergeStmt() (string, error) { fmt.Sprintf(`"%s"`, strings.ToUpper(m.peerdbCols.SyncedAtColName)), ) - insertColumnsSQL := strings.TrimSuffix(strings.Join(quotedUpperColNames, ","), ",") + insertColumnsSQL := strings.Join(quotedUpperColNames, ",") insertValuesSQLArray := make([]string, 0, len(columnNames)) for _, columnName := range columnNames { diff --git a/flow/connectors/snowflake/qrep.go b/flow/connectors/snowflake/qrep.go index 264cd3ea0e..b4c2642f57 100644 --- a/flow/connectors/snowflake/qrep.go +++ b/flow/connectors/snowflake/qrep.go @@ -7,15 +7,15 @@ import ( "strings" "time" + "github.com/aws/aws-sdk-go-v2/aws" + "github.com/aws/aws-sdk-go-v2/service/s3" + "github.com/jackc/pgx/v5/pgtype" + "google.golang.org/protobuf/encoding/protojson" + "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/shared" - "github.com/aws/aws-sdk-go/aws" - "github.com/aws/aws-sdk-go/service/s3" - "github.com/aws/aws-sdk-go/service/s3/s3manager" - "github.com/jackc/pgx/v5/pgtype" - "google.golang.org/protobuf/encoding/protojson" ) const qRepMetadataTableName = "_peerdb_query_replication_metadata" @@ -338,16 +338,27 @@ func (c *SnowflakeConnector) dropStage(stagingPath string, job string) error { } // Create a list of all objects with the defined prefix in the bucket - iter := s3manager.NewDeleteListIterator(s3svc, &s3.ListObjectsInput{ + pages := s3.NewListObjectsV2Paginator(s3svc, &s3.ListObjectsV2Input{ Bucket: aws.String(s3o.Bucket), Prefix: aws.String(fmt.Sprintf("%s/%s", s3o.Prefix, job)), }) - - // Iterate through the objects in the bucket with the prefix and delete them - s3Client := s3manager.NewBatchDeleteWithClient(s3svc) - if err := s3Client.Delete(aws.BackgroundContext(), iter); err != nil { - c.logger.Error("failed to delete objects from bucket", slog.Any("error", err)) - return fmt.Errorf("failed to delete objects from bucket: %w", err) + for pages.HasMorePages() { + page, err := pages.NextPage(c.ctx) + if err != nil { + c.logger.Error("failed to list objects from bucket", slog.Any("error", err)) + return fmt.Errorf("failed to list objects from bucket: %w", err) + } + + for _, object := range page.Contents { + _, err = s3svc.DeleteObject(c.ctx, &s3.DeleteObjectInput{ + Bucket: aws.String(s3o.Bucket), + Key: object.Key, + }) + if err != nil { + c.logger.Error("failed to delete objects from bucket", slog.Any("error", err)) + return fmt.Errorf("failed to delete objects from bucket: %w", err) + } + } } c.logger.Info(fmt.Sprintf("Deleted contents of bucket %s with prefix %s/%s", s3o.Bucket, s3o.Prefix, job)) diff --git a/flow/connectors/snowflake/qrep_avro_consolidate.go b/flow/connectors/snowflake/qrep_avro_consolidate.go index a77767eb53..17b8246c23 100644 --- a/flow/connectors/snowflake/qrep_avro_consolidate.go +++ b/flow/connectors/snowflake/qrep_avro_consolidate.go @@ -94,6 +94,12 @@ func getTransformSQL(colNames []string, colTypes []string, syncedAtCol string) ( case "NUMBER": transformations = append(transformations, fmt.Sprintf("$1:\"%s\" AS %s", avroColName, normalizedColName)) + case "DATE": + transformations = append(transformations, + fmt.Sprintf("TO_DATE($1:\"%s\") AS %s", avroColName, normalizedColName)) + case "TIME": + transformations = append(transformations, + fmt.Sprintf("TO_TIME(SPLIT($1:\"%s\",'+')[0]) AS %s", avroColName, normalizedColName)) case "VARIANT": transformations = append(transformations, fmt.Sprintf("PARSE_JSON($1:\"%s\") AS %s", avroColName, normalizedColName)) diff --git a/flow/connectors/snowflake/qrep_avro_sync.go b/flow/connectors/snowflake/qrep_avro_sync.go index 8355ab8bb2..3c330d636c 100644 --- a/flow/connectors/snowflake/qrep_avro_sync.go +++ b/flow/connectors/snowflake/qrep_avro_sync.go @@ -8,14 +8,15 @@ import ( "strings" "time" + _ "github.com/snowflakedb/gosnowflake" + "go.temporal.io/sdk/activity" + "github.com/PeerDB-io/peer-flow/connectors/utils" avro "github.com/PeerDB-io/peer-flow/connectors/utils/avro" "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/shared" - _ "github.com/snowflakedb/gosnowflake" - "go.temporal.io/sdk/activity" ) type SnowflakeAvroSyncHandler struct { diff --git a/flow/connectors/snowflake/snowflake.go b/flow/connectors/snowflake/snowflake.go index 221db2c374..e60bf8993e 100644 --- a/flow/connectors/snowflake/snowflake.go +++ b/flow/connectors/snowflake/snowflake.go @@ -6,22 +6,22 @@ import ( "encoding/json" "errors" "fmt" - "log" "log/slog" "regexp" "strings" "sync/atomic" "time" + "github.com/jackc/pgx/v5/pgtype" + "github.com/snowflakedb/gosnowflake" + "go.temporal.io/sdk/activity" + "golang.org/x/sync/errgroup" + "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/PeerDB-io/peer-flow/shared" - "github.com/jackc/pgx/v5/pgtype" - "github.com/snowflakedb/gosnowflake" - "go.temporal.io/sdk/activity" - "golang.org/x/sync/errgroup" ) const ( @@ -34,6 +34,7 @@ const ( _PEERDB_TIMESTAMP INT NOT NULL,_PEERDB_DESTINATION_TABLE_NAME STRING NOT NULL,_PEERDB_DATA STRING NOT NULL, _PEERDB_RECORD_TYPE INTEGER NOT NULL, _PEERDB_MATCH_DATA STRING,_PEERDB_BATCH_ID INT, _PEERDB_UNCHANGED_TOAST_COLUMNS STRING)` + createDummyTableSQL = "CREATE TABLE IF NOT EXISTS %s.%s(_PEERDB_DUMMY_COL STRING)" rawTableMultiValueInsertSQL = "INSERT INTO %s.%s VALUES%s" createNormalizedTableSQL = "CREATE TABLE IF NOT EXISTS %s(%s)" toVariantColumnName = "VAR_COLS" @@ -76,6 +77,7 @@ const ( getLastNormalizeBatchID_SQL = "SELECT NORMALIZE_BATCH_ID FROM %s.%s WHERE MIRROR_JOB_NAME=?" dropTableIfExistsSQL = "DROP TABLE IF EXISTS %s.%s" deleteJobMetadataSQL = "DELETE FROM %s.%s WHERE MIRROR_JOB_NAME=?" + dropSchemaIfExistsSQL = "DROP SCHEMA IF EXISTS %s" checkSchemaExistsSQL = "SELECT TO_BOOLEAN(COUNT(1)) FROM INFORMATION_SCHEMA.SCHEMATA WHERE SCHEMA_NAME=?" ) @@ -105,6 +107,64 @@ type UnchangedToastColumnResult struct { UnchangedToastColumns ArrayString } +func TableCheck(ctx context.Context, database *sql.DB) error { + dummySchema := "PEERDB_DUMMY_SCHEMA_" + shared.RandomString(4) + dummyTable := "PEERDB_DUMMY_TABLE_" + shared.RandomString(4) + + // In a transaction, create a table, insert a row into the table and then drop the table + // If any of these steps fail, the transaction will be rolled back + tx, err := database.BeginTx(ctx, nil) + if err != nil { + return fmt.Errorf("failed to begin transaction: %w", err) + } + // in case we return after error, ensure transaction is rolled back + defer func() { + deferErr := tx.Rollback() + if deferErr != sql.ErrTxDone && deferErr != nil { + activity.GetLogger(ctx).Error("error while rolling back transaction for table check", + slog.Any("error", deferErr)) + } + }() + + // create schema + _, err = tx.ExecContext(ctx, fmt.Sprintf(createSchemaSQL, dummySchema)) + if err != nil { + return fmt.Errorf("failed to create schema: %w", err) + } + + // create table + _, err = tx.ExecContext(ctx, fmt.Sprintf(createDummyTableSQL, dummySchema, dummyTable)) + if err != nil { + return fmt.Errorf("failed to create table: %w", err) + } + + // insert row + _, err = tx.ExecContext(ctx, fmt.Sprintf("INSERT INTO %s.%s VALUES ('dummy')", dummySchema, dummyTable)) + if err != nil { + return fmt.Errorf("failed to insert row: %w", err) + } + + // drop table + _, err = tx.ExecContext(ctx, fmt.Sprintf(dropTableIfExistsSQL, dummySchema, dummyTable)) + if err != nil { + return fmt.Errorf("failed to drop table: %w", err) + } + + // drop schema + _, err = tx.ExecContext(ctx, fmt.Sprintf(dropSchemaIfExistsSQL, dummySchema)) + if err != nil { + return fmt.Errorf("failed to drop schema: %w", err) + } + + // commit transaction + err = tx.Commit() + if err != nil { + return fmt.Errorf("failed to commit transaction: %w", err) + } + + return nil +} + func NewSnowflakeConnector(ctx context.Context, snowflakeProtoConfig *protos.SnowflakeConfig, ) (*SnowflakeConnector, error) { @@ -141,6 +201,11 @@ func NewSnowflakeConnector(ctx context.Context, return nil, fmt.Errorf("failed to open connection to Snowflake peer: %w", err) } + err = TableCheck(ctx, database) + if err != nil { + return nil, fmt.Errorf("could not validate snowflake peer: %w", err) + } + metadataSchema := "_PEERDB_INTERNAL" if snowflakeProtoConfig.MetadataSchema != nil { metadataSchema = *snowflakeProtoConfig.MetadataSchema @@ -379,12 +444,12 @@ func (c *SnowflakeConnector) getTableNametoUnchangedCols(flowJobName string, syn var r UnchangedToastColumnResult err := rows.Scan(&r.TableName, &r.UnchangedToastColumns) if err != nil { - log.Fatalf("Failed to scan row: %v", err) + return nil, fmt.Errorf("failed to scan row: %w", err) } resultMap[r.TableName] = r.UnchangedToastColumns } if err := rows.Err(); err != nil { - log.Fatalf("Error iterating over rows: %v", err) + return nil, fmt.Errorf("error iterating over rows: %w", err) } return resultMap, nil } @@ -481,13 +546,7 @@ func (c *SnowflakeConnector) SyncRecords(req *model.SyncRecordsRequest) (*model. rawTableIdentifier := getRawTableIdentifier(req.FlowJobName) c.logger.Info(fmt.Sprintf("pushing records to Snowflake table %s", rawTableIdentifier)) - syncBatchID, err := c.GetLastSyncBatchID(req.FlowJobName) - if err != nil { - return nil, fmt.Errorf("failed to get previous syncBatchID: %w", err) - } - syncBatchID += 1 - - res, err := c.syncRecordsViaAvro(req, rawTableIdentifier, syncBatchID) + res, err := c.syncRecordsViaAvro(req, rawTableIdentifier, req.SyncBatchID) if err != nil { return nil, err } @@ -502,12 +561,12 @@ func (c *SnowflakeConnector) SyncRecords(req *model.SyncRecordsRequest) (*model. deferErr := syncRecordsTx.Rollback() if deferErr != sql.ErrTxDone && deferErr != nil { c.logger.Error("error while rolling back transaction for SyncRecords: %v", - slog.Any("error", deferErr), slog.Int64("syncBatchID", syncBatchID)) + slog.Any("error", deferErr), slog.Int64("syncBatchID", req.SyncBatchID)) } }() // updating metadata with new offset and syncBatchID - err = c.updateSyncMetadata(req.FlowJobName, res.LastSyncedCheckPointID, syncBatchID, syncRecordsTx) + err = c.updateSyncMetadata(req.FlowJobName, res.LastSyncedCheckpointID, req.SyncBatchID, syncRecordsTx) if err != nil { return nil, err } @@ -549,8 +608,7 @@ func (c *SnowflakeConnector) syncRecordsViaAvro( return nil, err } - tableSchemaDeltas := req.Records.WaitForSchemaDeltas(req.TableMappings) - err = c.ReplayTableSchemaDeltas(req.FlowJobName, tableSchemaDeltas) + err = c.ReplayTableSchemaDeltas(req.FlowJobName, req.Records.SchemaDeltas) if err != nil { return nil, fmt.Errorf("failed to sync schema changes: %w", err) } @@ -561,12 +619,11 @@ func (c *SnowflakeConnector) syncRecordsViaAvro( } return &model.SyncResponse{ - LastSyncedCheckPointID: lastCheckpoint, + LastSyncedCheckpointID: lastCheckpoint, NumRecordsSynced: int64(numRecords), CurrentSyncBatchID: syncBatchID, TableNameRowsMapping: tableNameRowsMapping, - TableSchemaDeltas: tableSchemaDeltas, - RelationMessageMapping: <-req.Records.RelationMessageMapping, + TableSchemaDeltas: req.Records.SchemaDeltas, }, nil } @@ -783,14 +840,14 @@ func generateCreateTableSQLForNormalizedTable( slog.Any("error", err)) return } - createTableSQLArray = append(createTableSQLArray, fmt.Sprintf(`%s %s,`, normalizedColName, sfColType)) + createTableSQLArray = append(createTableSQLArray, fmt.Sprintf(`%s %s`, normalizedColName, sfColType)) }) // add a _peerdb_is_deleted column to the normalized table // this is boolean default false, and is used to mark records as deleted if softDeleteColName != "" { createTableSQLArray = append(createTableSQLArray, - fmt.Sprintf(`%s BOOLEAN DEFAULT FALSE,`, softDeleteColName)) + fmt.Sprintf(`%s BOOLEAN DEFAULT FALSE`, softDeleteColName)) } // add a _peerdb_synced column to the normalized table @@ -798,7 +855,7 @@ func generateCreateTableSQLForNormalizedTable( // default value is the current timestamp (snowflake) if syncedAtColName != "" { createTableSQLArray = append(createTableSQLArray, - fmt.Sprintf(`%s TIMESTAMP DEFAULT CURRENT_TIMESTAMP,`, syncedAtColName)) + fmt.Sprintf(`%s TIMESTAMP DEFAULT CURRENT_TIMESTAMP`, syncedAtColName)) } // add composite primary key to the table @@ -808,12 +865,12 @@ func generateCreateTableSQLForNormalizedTable( normalizedPrimaryKeyCols = append(normalizedPrimaryKeyCols, SnowflakeIdentifierNormalize(primaryKeyCol)) } - createTableSQLArray = append(createTableSQLArray, fmt.Sprintf("PRIMARY KEY(%s),", - strings.TrimSuffix(strings.Join(normalizedPrimaryKeyCols, ","), ","))) + createTableSQLArray = append(createTableSQLArray, + fmt.Sprintf("PRIMARY KEY(%s)", strings.Join(normalizedPrimaryKeyCols, ","))) } return fmt.Sprintf(createNormalizedTableSQL, snowflakeSchemaTableNormalize(dstSchemaTable), - strings.TrimSuffix(strings.Join(createTableSQLArray, ""), ",")) + strings.Join(createTableSQLArray, ",")) } func getRawTableIdentifier(jobName string) string { diff --git a/flow/connectors/sql/query_executor.go b/flow/connectors/sql/query_executor.go index 751230a643..58e90b6894 100644 --- a/flow/connectors/sql/query_executor.go +++ b/flow/connectors/sql/query_executor.go @@ -9,14 +9,14 @@ import ( "math/big" "strings" - "github.com/PeerDB-io/peer-flow/model" - "github.com/PeerDB-io/peer-flow/model/qvalue" - "github.com/PeerDB-io/peer-flow/shared" "github.com/google/uuid" "github.com/jackc/pgx/v5/pgtype" "github.com/jmoiron/sqlx" - "go.temporal.io/sdk/activity" + + "github.com/PeerDB-io/peer-flow/model" + "github.com/PeerDB-io/peer-flow/model/qvalue" + "github.com/PeerDB-io/peer-flow/shared" ) type SQLQueryExecutor interface { @@ -173,7 +173,7 @@ func (g *GenericSQLQueryExecutor) processRows(rows *sqlx.Rows) (*model.QRecordBa qfields[i] = qfield } - var records []model.QRecord + var records [][]qvalue.QValue totalRowsProcessed := 0 const heartBeatNumRows = 25000 @@ -237,13 +237,7 @@ func (g *GenericSQLQueryExecutor) processRows(rows *sqlx.Rows) (*model.QRecordBa qValues[i] = qv } - // Create a QRecord - record := model.NewQRecord(len(qValues)) - for i, qv := range qValues { - record.Set(i, qv) - } - - records = append(records, record) + records = append(records, qValues) totalRowsProcessed += 1 if totalRowsProcessed%heartBeatNumRows == 0 { @@ -258,9 +252,8 @@ func (g *GenericSQLQueryExecutor) processRows(rows *sqlx.Rows) (*model.QRecordBa // Return a QRecordBatch return &model.QRecordBatch{ - NumRecords: uint32(len(records)), - Records: records, - Schema: model.NewQRecordSchema(qfields), + Records: records, + Schema: model.NewQRecordSchema(qfields), }, nil } diff --git a/flow/connectors/sqlserver/qrep.go b/flow/connectors/sqlserver/qrep.go index 6a75373597..33474ef490 100644 --- a/flow/connectors/sqlserver/qrep.go +++ b/flow/connectors/sqlserver/qrep.go @@ -6,12 +6,13 @@ import ( "log/slog" "text/template" - utils "github.com/PeerDB-io/peer-flow/connectors/utils/partition" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/model" "github.com/google/uuid" "github.com/jackc/pgx/v5/pgtype" "github.com/jmoiron/sqlx" + + utils "github.com/PeerDB-io/peer-flow/connectors/utils/partition" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/model" ) func (c *SQLServerConnector) GetQRepPartitions( diff --git a/flow/connectors/sqlserver/sqlserver.go b/flow/connectors/sqlserver/sqlserver.go index 6e9235431b..805dcfb849 100644 --- a/flow/connectors/sqlserver/sqlserver.go +++ b/flow/connectors/sqlserver/sqlserver.go @@ -5,11 +5,12 @@ import ( "fmt" "log/slog" + "github.com/jmoiron/sqlx" + _ "github.com/microsoft/go-mssqldb" + peersql "github.com/PeerDB-io/peer-flow/connectors/sql" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/shared" - "github.com/jmoiron/sqlx" - _ "github.com/microsoft/go-mssqldb" ) type SQLServerConnector struct { diff --git a/flow/connectors/utils/avro/avro_writer.go b/flow/connectors/utils/avro/avro_writer.go index c2f3af94dd..6148ae2a6a 100644 --- a/flow/connectors/utils/avro/avro_writer.go +++ b/flow/connectors/utils/avro/avro_writer.go @@ -9,16 +9,17 @@ import ( "os" "sync/atomic" - "github.com/PeerDB-io/peer-flow/connectors/utils" - "github.com/PeerDB-io/peer-flow/model" - "github.com/PeerDB-io/peer-flow/model/qvalue" - - "github.com/aws/aws-sdk-go/aws" - "github.com/aws/aws-sdk-go/service/s3/s3manager" + "github.com/aws/aws-sdk-go-v2/aws" + "github.com/aws/aws-sdk-go-v2/feature/s3/manager" + "github.com/aws/aws-sdk-go-v2/service/s3" "github.com/klauspost/compress/flate" "github.com/klauspost/compress/snappy" "github.com/klauspost/compress/zstd" "github.com/linkedin/goavro/v2" + + "github.com/PeerDB-io/peer-flow/connectors/utils" + "github.com/PeerDB-io/peer-flow/model" + "github.com/PeerDB-io/peer-flow/model/qvalue" ) type ( @@ -201,11 +202,7 @@ func (p *peerDBOCFWriter) WriteRecordsToS3(bucketName, key string, s3Creds utils return nil, fmt.Errorf("failed to create S3 client: %w", err) } - // Create an uploader with the session and default options - uploader := s3manager.NewUploaderWithClient(s3svc) - - // Upload the file to S3. - result, err := uploader.Upload(&s3manager.UploadInput{ + _, err = manager.NewUploader(s3svc).Upload(p.ctx, &s3.PutObjectInput{ Bucket: aws.String(bucketName), Key: aws.String(key), Body: r, @@ -216,7 +213,7 @@ func (p *peerDBOCFWriter) WriteRecordsToS3(bucketName, key string, s3Creds utils return nil, fmt.Errorf("failed to upload file to path %s: %w", s3Path, err) } - slog.Info("file uploaded to" + result.Location) + slog.Info("file uploaded to " + fmt.Sprintf("%s/%s", bucketName, key)) return &AvroFile{ NumRecords: <-numRowsWritten, diff --git a/flow/connectors/utils/aws.go b/flow/connectors/utils/aws.go index 473efd0ce5..63adfa8330 100644 --- a/flow/connectors/utils/aws.go +++ b/flow/connectors/utils/aws.go @@ -2,13 +2,15 @@ package utils import ( "fmt" + "net/http" "os" "strings" + "time" - "github.com/aws/aws-sdk-go/aws" - "github.com/aws/aws-sdk-go/aws/credentials" - "github.com/aws/aws-sdk-go/aws/session" - "github.com/aws/aws-sdk-go/service/s3" + "github.com/aws/aws-sdk-go-v2/aws" + v4 "github.com/aws/aws-sdk-go-v2/aws/signer/v4" + "github.com/aws/aws-sdk-go-v2/credentials" + "github.com/aws/aws-sdk-go-v2/service/s3" ) type AWSSecrets struct { @@ -20,11 +22,11 @@ type AWSSecrets struct { } type S3PeerCredentials struct { - AccessKeyID string - SecretAccessKey string - AwsRoleArn string - Region string - Endpoint string + AccessKeyID string `json:"accessKeyId"` + SecretAccessKey string `json:"secretAccessKey"` + AwsRoleArn string `json:"awsRoleArn"` + Region string `json:"region"` + Endpoint string `json:"endpoint"` } func GetAWSSecrets(creds S3PeerCredentials) (*AWSSecrets, error) { @@ -89,23 +91,65 @@ func NewS3BucketAndPrefix(s3Path string) (*S3BucketAndPrefix, error) { }, nil } -func CreateS3Client(s3Creds S3PeerCredentials) (*s3.S3, error) { +func CreateS3Client(s3Creds S3PeerCredentials) (*s3.Client, error) { awsSecrets, err := GetAWSSecrets(s3Creds) if err != nil { return nil, fmt.Errorf("failed to get AWS secrets: %w", err) } - - config := &aws.Config{ - Region: aws.String(awsSecrets.Region), - Endpoint: aws.String(awsSecrets.Endpoint), + options := s3.Options{ + Region: awsSecrets.Region, + Credentials: credentials.NewStaticCredentialsProvider(awsSecrets.AccessKeyID, awsSecrets.SecretAccessKey, ""), } - - if s3Creds.AccessKeyID != "" && s3Creds.SecretAccessKey != "" { - config.Credentials = credentials.NewStaticCredentials(s3Creds.AccessKeyID, s3Creds.SecretAccessKey, "") + if awsSecrets.Endpoint != "" { + options.BaseEndpoint = &awsSecrets.Endpoint + if strings.Contains(awsSecrets.Endpoint, "storage.googleapis.com") { + // Assign custom client with our own transport + options.HTTPClient = &http.Client{ + Transport: &RecalculateV4Signature{ + next: http.DefaultTransport, + signer: v4.NewSigner(), + credentials: options.Credentials, + region: options.Region, + }, + } + } } - sess := session.Must(session.NewSession(config)) + return s3.New(options), nil +} + +// RecalculateV4Signature allow GCS over S3, removing Accept-Encoding header from sign +// https://stackoverflow.com/a/74382598/1204665 +// https://github.com/aws/aws-sdk-go-v2/issues/1816 +type RecalculateV4Signature struct { + next http.RoundTripper + signer *v4.Signer + credentials aws.CredentialsProvider + region string +} + +func (lt *RecalculateV4Signature) RoundTrip(req *http.Request) (*http.Response, error) { + // store for later use + acceptEncodingValue := req.Header.Get("Accept-Encoding") + + // delete the header so the header doesn't account for in the signature + req.Header.Del("Accept-Encoding") + + // sign with the same date + timeString := req.Header.Get("X-Amz-Date") + timeDate, _ := time.Parse("20060102T150405Z", timeString) + + creds, err := lt.credentials.Retrieve(req.Context()) + if err != nil { + return nil, err + } + err = lt.signer.SignHTTP(req.Context(), creds, req, v4.GetPayloadHash(req.Context()), "s3", lt.region, timeDate) + if err != nil { + return nil, err + } + // Reset Accept-Encoding if desired + req.Header.Set("Accept-Encoding", acceptEncodingValue) - s3svc := s3.New(sess) - return s3svc, nil + // follows up the original round tripper + return lt.next.RoundTrip(req) } diff --git a/flow/connectors/utils/catalog/env.go b/flow/connectors/utils/catalog/env.go index f5c8e0507d..5a12172022 100644 --- a/flow/connectors/utils/catalog/env.go +++ b/flow/connectors/utils/catalog/env.go @@ -5,10 +5,11 @@ import ( "fmt" "sync" + "github.com/jackc/pgx/v5/pgxpool" + "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/peerdbenv" - "github.com/jackc/pgx/v5/pgxpool" ) var ( diff --git a/flow/connectors/utils/cdc_records/cdc_records_storage.go b/flow/connectors/utils/cdc_records/cdc_records_storage.go index c6dece9eb9..a1bde8d614 100644 --- a/flow/connectors/utils/cdc_records/cdc_records_storage.go +++ b/flow/connectors/utils/cdc_records/cdc_records_storage.go @@ -10,10 +10,11 @@ import ( "os" "time" + "github.com/cockroachdb/pebble" + "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/peerdbenv" "github.com/PeerDB-io/peer-flow/shared" - "github.com/cockroachdb/pebble" ) func encVal(val any) ([]byte, error) { diff --git a/flow/connectors/utils/cdc_records/cdc_records_storage_test.go b/flow/connectors/utils/cdc_records/cdc_records_storage_test.go index 9aea00ae18..9f0b7a9f6e 100644 --- a/flow/connectors/utils/cdc_records/cdc_records_storage_test.go +++ b/flow/connectors/utils/cdc_records/cdc_records_storage_test.go @@ -6,9 +6,10 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" - "github.com/stretchr/testify/require" ) func getTimeForTesting(t *testing.T) time.Time { @@ -47,7 +48,7 @@ func genKeyAndRec(t *testing.T) (model.TableWithPkey, model.Record) { rec := &model.InsertRecord{ SourceTableName: "test_src_tbl", DestinationTableName: "test_dst_tbl", - CheckPointID: 1, + CheckpointID: 1, CommitID: 2, Items: &model.RecordItems{ ColToValIdx: map[string]int{ @@ -83,7 +84,7 @@ func TestSingleRecord(t *testing.T) { err := cdcRecordsStore.Set(key, rec) require.NoError(t, err) // should not spill into DB - require.Equal(t, 1, len(cdcRecordsStore.inMemoryRecords)) + require.Len(t, cdcRecordsStore.inMemoryRecords, 1) require.Nil(t, cdcRecordsStore.pebbleDB) reck, ok, err := cdcRecordsStore.Get(key) @@ -100,11 +101,11 @@ func TestRecordsTillSpill(t *testing.T) { cdcRecordsStore.numRecordsSwitchThreshold = 10 // add records upto set limit - for i := 0; i < 10; i++ { + for i := 1; i <= 10; i++ { key, rec := genKeyAndRec(t) err := cdcRecordsStore.Set(key, rec) require.NoError(t, err) - require.Equal(t, i+1, len(cdcRecordsStore.inMemoryRecords)) + require.Len(t, cdcRecordsStore.inMemoryRecords, i) require.Nil(t, cdcRecordsStore.pebbleDB) } diff --git a/flow/connectors/utils/monitoring/monitoring.go b/flow/connectors/utils/monitoring/monitoring.go index 7e9263cf26..5412323bfc 100644 --- a/flow/connectors/utils/monitoring/monitoring.go +++ b/flow/connectors/utils/monitoring/monitoring.go @@ -4,16 +4,17 @@ import ( "context" "fmt" "log/slog" + "strconv" "time" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/shared" "github.com/jackc/pglogrepl" "github.com/jackc/pgx/v5" "github.com/jackc/pgx/v5/pgtype" "github.com/jackc/pgx/v5/pgxpool" - "google.golang.org/protobuf/proto" + + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/shared" ) type CDCBatchInfo struct { @@ -233,8 +234,8 @@ func addPartitionToQRepRun(ctx context.Context, pool *pgxpool.Pool, flowJobName var rangeStart, rangeEnd string switch x := partition.Range.Range.(type) { case *protos.PartitionRange_IntRange: - rangeStart = fmt.Sprint(x.IntRange.Start) - rangeEnd = fmt.Sprint(x.IntRange.End) + rangeStart = strconv.FormatInt(x.IntRange.Start, 10) + rangeEnd = strconv.FormatInt(x.IntRange.End, 10) case *protos.PartitionRange_TimestampRange: rangeStart = x.TimestampRange.Start.AsTime().String() rangeEnd = x.TimestampRange.End.AsTime().String() diff --git a/flow/connectors/utils/partition/partition.go b/flow/connectors/utils/partition/partition.go index cb2f326a66..7d4d2c4298 100644 --- a/flow/connectors/utils/partition/partition.go +++ b/flow/connectors/utils/partition/partition.go @@ -5,11 +5,11 @@ import ( "log/slog" "time" - "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/google/uuid" "github.com/jackc/pgx/v5/pgtype" - "google.golang.org/protobuf/types/known/timestamppb" + + "github.com/PeerDB-io/peer-flow/generated/protos" ) // Function to compare two values @@ -45,14 +45,12 @@ func compareValues(prevEnd interface{}, start interface{}) int { return -1 } else if pe.BlockNumber > v.BlockNumber { return 1 + } else if pe.OffsetNumber < v.OffsetNumber { + return -1 + } else if pe.OffsetNumber > v.OffsetNumber { + return 1 } else { - if pe.OffsetNumber < v.OffsetNumber { - return -1 - } else if pe.OffsetNumber > v.OffsetNumber { - return 1 - } else { - return 0 - } + return 0 } case uint32: // xmin if prevEnd.(uint32) < v { diff --git a/flow/connectors/utils/partition_hash.go b/flow/connectors/utils/partition_hash.go index 14de3ae943..e22b1a27d3 100644 --- a/flow/connectors/utils/partition_hash.go +++ b/flow/connectors/utils/partition_hash.go @@ -1,8 +1,8 @@ package utils import ( - "fmt" "hash/fnv" + "strconv" ) func hashString(s string) uint32 { @@ -14,5 +14,5 @@ func hashString(s string) uint32 { func HashedPartitionKey(s string, numPartitions uint32) string { hashValue := hashString(s) partition := hashValue % numPartitions - return fmt.Sprintf("%d", partition) + return strconv.FormatUint(uint64(partition), 10) } diff --git a/flow/connectors/utils/postgres.go b/flow/connectors/utils/postgres.go index c3cc48659f..ed97364417 100644 --- a/flow/connectors/utils/postgres.go +++ b/flow/connectors/utils/postgres.go @@ -6,12 +6,13 @@ import ( "fmt" "net/url" - "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/jackc/pgerrcode" "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" ) func IsUniqueError(err error) bool { diff --git a/flow/connectors/utils/ssh.go b/flow/connectors/utils/ssh.go index 511eea672a..c4580e870f 100644 --- a/flow/connectors/utils/ssh.go +++ b/flow/connectors/utils/ssh.go @@ -5,6 +5,8 @@ import ( "fmt" "golang.org/x/crypto/ssh" + + "github.com/PeerDB-io/peer-flow/generated/protos" ) // getSSHClientConfig returns an *ssh.ClientConfig based on provided credentials. @@ -13,17 +15,17 @@ import ( // user: SSH username // password: SSH password (can be empty if using a private key) // privateKeyString: Private key as a string (can be empty if using a password) -func GetSSHClientConfig(user, password, privateKeyString string) (*ssh.ClientConfig, error) { +func GetSSHClientConfig(config *protos.SSHConfig) (*ssh.ClientConfig, error) { var authMethods []ssh.AuthMethod // Password-based authentication - if password != "" { - authMethods = append(authMethods, ssh.Password(password)) + if config.Password != "" { + authMethods = append(authMethods, ssh.Password(config.Password)) } // Private key-based authentication - if privateKeyString != "" { - pkey, err := base64.StdEncoding.DecodeString(privateKeyString) + if config.PrivateKey != "" { + pkey, err := base64.StdEncoding.DecodeString(config.PrivateKey) if err != nil { return nil, fmt.Errorf("failed to base64 decode private key: %w", err) } @@ -40,10 +42,21 @@ func GetSSHClientConfig(user, password, privateKeyString string) (*ssh.ClientCon return nil, fmt.Errorf("no authentication methods provided") } - return &ssh.ClientConfig{ - User: user, - Auth: authMethods, + var hostKeyCallback ssh.HostKeyCallback + if config.HostKey != "" { + pubKey, err := ssh.ParsePublicKey([]byte(config.HostKey)) + if err != nil { + return nil, fmt.Errorf("failed to parse host key: %w", err) + } + hostKeyCallback = ssh.FixedHostKey(pubKey) + } else { //nolint:gosec - HostKeyCallback: ssh.InsecureIgnoreHostKey(), + hostKeyCallback = ssh.InsecureIgnoreHostKey() + } + + return &ssh.ClientConfig{ + User: config.User, + Auth: authMethods, + HostKeyCallback: hostKeyCallback, }, nil } diff --git a/flow/connectors/utils/stream.go b/flow/connectors/utils/stream.go index bbfe0a2fa6..898acd03b5 100644 --- a/flow/connectors/utils/stream.go +++ b/flow/connectors/utils/stream.go @@ -4,9 +4,10 @@ import ( "fmt" "time" + "github.com/google/uuid" + "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" - "github.com/google/uuid" ) func RecordsToRawTableStream(req *model.RecordsToStreamRequest) (*model.RecordsToStreamResponse, error) { @@ -182,9 +183,6 @@ func recordToQRecordOrError(tableMapping map[string]uint32, batchID int64, recor } return model.QRecordOrError{ - Record: model.QRecord{ - NumEntries: 8, - Entries: entries[:], - }, + Record: entries[:], } } diff --git a/flow/dynamicconf/dynamicconf.go b/flow/dynamicconf/dynamicconf.go index 5a22ba4058..d08ece7078 100644 --- a/flow/dynamicconf/dynamicconf.go +++ b/flow/dynamicconf/dynamicconf.go @@ -6,9 +6,10 @@ import ( "strconv" "time" - utils "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" "github.com/jackc/pgx/v5/pgtype" "github.com/jackc/pgx/v5/pgxpool" + + utils "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" ) func dynamicConfKeyExists(ctx context.Context, conn *pgxpool.Pool, key string) bool { diff --git a/flow/e2e/bigquery/bigquery_helper.go b/flow/e2e/bigquery/bigquery_helper.go index 445488266e..14878a5c6f 100644 --- a/flow/e2e/bigquery/bigquery_helper.go +++ b/flow/e2e/bigquery/bigquery_helper.go @@ -11,13 +11,14 @@ import ( "cloud.google.com/go/bigquery" "cloud.google.com/go/civil" + "google.golang.org/api/iterator" + peer_bq "github.com/PeerDB-io/peer-flow/connectors/bigquery" "github.com/PeerDB-io/peer-flow/e2eshared" "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/shared" - "google.golang.org/api/iterator" ) type BigQueryTestHelper struct { @@ -218,7 +219,9 @@ func toQValue(bqValue bigquery.Value) (qvalue.QValue, error) { case bool: return qvalue.QValue{Kind: qvalue.QValueKindBoolean, Value: v}, nil case civil.Date: - return qvalue.QValue{Kind: qvalue.QValueKindDate, Value: bqValue.(civil.Date).In(time.UTC)}, nil + return qvalue.QValue{Kind: qvalue.QValueKindDate, Value: v.In(time.UTC)}, nil + case civil.Time: + return qvalue.QValue{Kind: qvalue.QValueKindTime, Value: v}, nil case time.Time: return qvalue.QValue{Kind: qvalue.QValueKindTimestamp, Value: v}, nil case *big.Rat: @@ -333,7 +336,7 @@ func (b *BigQueryTestHelper) ExecuteAndProcessQuery(query string) (*model.QRecor return nil, fmt.Errorf("failed to run command: %w", err) } - var records []model.QRecord + var records [][]qvalue.QValue for { var row []bigquery.Value err := it.Next(&row) @@ -354,13 +357,7 @@ func (b *BigQueryTestHelper) ExecuteAndProcessQuery(query string) (*model.QRecor qValues[i] = qv } - // Create a QRecord - record := model.NewQRecord(len(qValues)) - for i, qv := range qValues { - record.Set(i, qv) - } - - records = append(records, record) + records = append(records, qValues) } // Now you should fill the column names as well. Here we assume the schema is @@ -375,18 +372,17 @@ func (b *BigQueryTestHelper) ExecuteAndProcessQuery(query string) (*model.QRecor // Return a QRecordBatch return &model.QRecordBatch{ - NumRecords: uint32(len(records)), - Records: records, - Schema: schema, + Records: records, + Schema: schema, }, nil } // returns whether the function errors or there are nulls -func (b *BigQueryTestHelper) CheckNull(tableName string, ColName []string) (bool, error) { - if len(ColName) == 0 { +func (b *BigQueryTestHelper) CheckNull(tableName string, colName []string) (bool, error) { + if len(colName) == 0 { return true, nil } - joinedString := strings.Join(ColName, " is null or ") + " is null" + joinedString := strings.Join(colName, " is null or ") + " is null" command := fmt.Sprintf("SELECT COUNT(*) FROM `%s.%s` WHERE %s", b.Config.DatasetId, tableName, joinedString) q := b.client.Query(command) @@ -419,8 +415,8 @@ func (b *BigQueryTestHelper) CheckNull(tableName string, ColName []string) (bool } // check if NaN, Inf double values are null -func (b *BigQueryTestHelper) CheckDoubleValues(tableName string, ColName []string) (bool, error) { - csep := strings.Join(ColName, ",") +func (b *BigQueryTestHelper) CheckDoubleValues(tableName string, colName []string) (bool, error) { + csep := strings.Join(colName, ",") command := fmt.Sprintf("SELECT %s FROM `%s.%s`", csep, b.Config.DatasetId, tableName) q := b.client.Query(command) @@ -513,9 +509,9 @@ func (b *BigQueryTestHelper) RunInt64Query(query string) (int64, error) { if err != nil { return 0, fmt.Errorf("could not execute query: %w", err) } - if recordBatch.NumRecords != 1 { - return 0, fmt.Errorf("expected only 1 record, got %d", recordBatch.NumRecords) + if len(recordBatch.Records) != 1 { + return 0, fmt.Errorf("expected only 1 record, got %d", len(recordBatch.Records)) } - return recordBatch.Records[0].Entries[0].Value.(int64), nil + return recordBatch.Records[0][0].Value.(int64), nil } diff --git a/flow/e2e/bigquery/peer_flow_bq_test.go b/flow/e2e/bigquery/peer_flow_bq_test.go index efa47366db..651f5551a4 100644 --- a/flow/e2e/bigquery/peer_flow_bq_test.go +++ b/flow/e2e/bigquery/peer_flow_bq_test.go @@ -10,6 +10,12 @@ import ( "testing" "time" + "github.com/jackc/pgerrcode" + "github.com/jackc/pgx/v5/pgconn" + "github.com/jackc/pgx/v5/pgxpool" + "github.com/joho/godotenv" + "github.com/stretchr/testify/require" + "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/e2e" "github.com/PeerDB-io/peer-flow/e2eshared" @@ -18,11 +24,6 @@ import ( "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/PeerDB-io/peer-flow/shared" peerflow "github.com/PeerDB-io/peer-flow/workflows" - "github.com/jackc/pgerrcode" - "github.com/jackc/pgx/v5/pgconn" - "github.com/jackc/pgx/v5/pgxpool" - "github.com/joho/godotenv" - "github.com/stretchr/testify/require" ) type PeerFlowE2ETestSuiteBQ struct { @@ -81,7 +82,7 @@ func (s PeerFlowE2ETestSuiteBQ) checkJSONValue(tableName, colName, fieldName, va return fmt.Errorf("json value check failed: %v", err) } - jsonVal := res.Records[0].Entries[0].Value + jsonVal := res.Records[0][0].Value if jsonVal != value { return fmt.Errorf("bad json value in field %s of column %s: %v. expected: %v", fieldName, colName, jsonVal, value) } @@ -113,7 +114,7 @@ func (s *PeerFlowE2ETestSuiteBQ) checkPeerdbColumns(dstQualified string, softDel recordCount := 0 for _, record := range recordBatch.Records { - for _, entry := range record.Entries { + for _, entry := range record { if entry.Kind == qvalue.QValueKindBoolean { isDeleteVal, ok := entry.Value.(bool) if !(ok && isDeleteVal) { diff --git a/flow/e2e/bigquery/qrep_flow_bq_test.go b/flow/e2e/bigquery/qrep_flow_bq_test.go index 6ac9f3be5d..4302b222e5 100644 --- a/flow/e2e/bigquery/qrep_flow_bq_test.go +++ b/flow/e2e/bigquery/qrep_flow_bq_test.go @@ -3,8 +3,9 @@ package e2e_bigquery import ( "fmt" - "github.com/PeerDB-io/peer-flow/e2e" "github.com/stretchr/testify/require" + + "github.com/PeerDB-io/peer-flow/e2e" ) func (s PeerFlowE2ETestSuiteBQ) setupSourceTable(tableName string, rowCount int) { diff --git a/flow/e2e/congen.go b/flow/e2e/congen.go index f324a8e8f5..cb3c35bc29 100644 --- a/flow/e2e/congen.go +++ b/flow/e2e/congen.go @@ -6,13 +6,14 @@ import ( "log/slog" "time" - "github.com/PeerDB-io/peer-flow/connectors/utils" - "github.com/PeerDB-io/peer-flow/e2eshared" - "github.com/PeerDB-io/peer-flow/generated/protos" "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" + "github.com/PeerDB-io/peer-flow/e2eshared" + "github.com/PeerDB-io/peer-flow/generated/protos" ) const ( diff --git a/flow/e2e/postgres/peer_flow_pg_test.go b/flow/e2e/postgres/peer_flow_pg_test.go index b02d03df05..27be975acb 100644 --- a/flow/e2e/postgres/peer_flow_pg_test.go +++ b/flow/e2e/postgres/peer_flow_pg_test.go @@ -7,16 +7,17 @@ import ( "strings" "time" - "github.com/PeerDB-io/peer-flow/connectors/utils" - "github.com/PeerDB-io/peer-flow/e2e" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/model/qvalue" - peerflow "github.com/PeerDB-io/peer-flow/workflows" "github.com/jackc/pgerrcode" "github.com/jackc/pgx/v5/pgconn" "github.com/jackc/pgx/v5/pgtype" "github.com/stretchr/testify/require" "go.temporal.io/sdk/testsuite" + + "github.com/PeerDB-io/peer-flow/connectors/utils" + "github.com/PeerDB-io/peer-flow/e2e" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/model/qvalue" + peerflow "github.com/PeerDB-io/peer-flow/workflows" ) func (s PeerFlowE2ETestSuitePG) attachSchemaSuffix(tableName string) string { diff --git a/flow/e2e/postgres/qrep_flow_pg_test.go b/flow/e2e/postgres/qrep_flow_pg_test.go index 2943e201b5..dcb32c7fd9 100644 --- a/flow/e2e/postgres/qrep_flow_pg_test.go +++ b/flow/e2e/postgres/qrep_flow_pg_test.go @@ -8,16 +8,17 @@ import ( "testing" "time" - connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" - "github.com/PeerDB-io/peer-flow/e2e" - "github.com/PeerDB-io/peer-flow/e2eshared" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/shared" "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" + + connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" + "github.com/PeerDB-io/peer-flow/e2e" + "github.com/PeerDB-io/peer-flow/e2eshared" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/shared" ) type PeerFlowE2ETestSuitePG struct { diff --git a/flow/e2e/s3/cdc_s3_test.go b/flow/e2e/s3/cdc_s3_test.go index 26eb42d38e..69c26b6809 100644 --- a/flow/e2e/s3/cdc_s3_test.go +++ b/flow/e2e/s3/cdc_s3_test.go @@ -5,9 +5,10 @@ import ( "fmt" "time" + "github.com/stretchr/testify/require" + "github.com/PeerDB-io/peer-flow/e2e" peerflow "github.com/PeerDB-io/peer-flow/workflows" - "github.com/stretchr/testify/require" ) func (s PeerFlowE2ETestSuiteS3) attachSchemaSuffix(tableName string) string { diff --git a/flow/e2e/s3/qrep_flow_s3_test.go b/flow/e2e/s3/qrep_flow_s3_test.go index 46dd16ef4c..919994b570 100644 --- a/flow/e2e/s3/qrep_flow_s3_test.go +++ b/flow/e2e/s3/qrep_flow_s3_test.go @@ -8,12 +8,13 @@ import ( "testing" "time" - "github.com/PeerDB-io/peer-flow/e2e" - "github.com/PeerDB-io/peer-flow/e2eshared" - "github.com/PeerDB-io/peer-flow/shared" "github.com/jackc/pgx/v5/pgxpool" "github.com/joho/godotenv" "github.com/stretchr/testify/require" + + "github.com/PeerDB-io/peer-flow/e2e" + "github.com/PeerDB-io/peer-flow/e2eshared" + "github.com/PeerDB-io/peer-flow/shared" ) type PeerFlowE2ETestSuiteS3 struct { @@ -39,7 +40,7 @@ func (s PeerFlowE2ETestSuiteS3) Suffix() string { func tearDownSuite(s PeerFlowE2ETestSuiteS3) { e2e.TearDownPostgres(s) - err := s.s3Helper.CleanUp() + err := s.s3Helper.CleanUp(context.Background()) if err != nil { require.Fail(s.t, "failed to clean up s3", err) } @@ -143,7 +144,7 @@ func (s PeerFlowE2ETestSuiteS3) Test_Complete_QRep_Flow_S3() { require.NoError(s.t, err) - require.Equal(s.t, 1, len(files)) + require.Len(s.t, files, 1) } func (s PeerFlowE2ETestSuiteS3) Test_Complete_QRep_Flow_S3_CTID() { @@ -192,5 +193,5 @@ func (s PeerFlowE2ETestSuiteS3) Test_Complete_QRep_Flow_S3_CTID() { require.NoError(s.t, err) - require.Equal(s.t, 10, len(files)) + require.Len(s.t, files, 10) } diff --git a/flow/e2e/s3/s3_helper.go b/flow/e2e/s3/s3_helper.go index ef9020e278..1be1765927 100644 --- a/flow/e2e/s3/s3_helper.go +++ b/flow/e2e/s3/s3_helper.go @@ -8,12 +8,13 @@ import ( "os" "time" + "github.com/aws/aws-sdk-go-v2/service/s3" + s3types "github.com/aws/aws-sdk-go-v2/service/s3/types" + "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/e2eshared" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/shared" - "github.com/aws/aws-sdk-go/aws" - "github.com/aws/aws-sdk-go/service/s3" ) const ( @@ -21,7 +22,7 @@ const ( ) type S3TestHelper struct { - client *s3.S3 + client *s3.Client s3Config *protos.S3Config bucketName string prefix string @@ -90,10 +91,10 @@ func (h *S3TestHelper) GetPeer() *protos.Peer { func (h *S3TestHelper) ListAllFiles( ctx context.Context, jobName string, -) ([]*s3.Object, error) { +) ([]s3types.Object, error) { Bucket := h.bucketName Prefix := fmt.Sprintf("%s/%s/", h.prefix, jobName) - files, err := h.client.ListObjects(&s3.ListObjectsInput{ + files, err := h.client.ListObjects(ctx, &s3.ListObjectsInput{ Bucket: &Bucket, Prefix: &Prefix, }) @@ -106,10 +107,10 @@ func (h *S3TestHelper) ListAllFiles( } // Delete all generated objects during the test -func (h *S3TestHelper) CleanUp() error { +func (h *S3TestHelper) CleanUp(ctx context.Context) error { Bucket := h.bucketName Prefix := h.prefix - files, err := h.client.ListObjects(&s3.ListObjectsInput{ + files, err := h.client.ListObjects(ctx, &s3.ListObjectsInput{ Bucket: &Bucket, Prefix: &Prefix, }) @@ -121,11 +122,11 @@ func (h *S3TestHelper) CleanUp() error { // Delete each object for _, obj := range files.Contents { deleteInput := &s3.DeleteObjectInput{ - Bucket: aws.String(Bucket), + Bucket: &Bucket, Key: obj.Key, } - _, err := h.client.DeleteObject(deleteInput) + _, err := h.client.DeleteObject(ctx, deleteInput) if err != nil { return err } diff --git a/flow/e2e/snowflake/peer_flow_sf_test.go b/flow/e2e/snowflake/peer_flow_sf_test.go index bb4d64636a..8f3cf46e7a 100644 --- a/flow/e2e/snowflake/peer_flow_sf_test.go +++ b/flow/e2e/snowflake/peer_flow_sf_test.go @@ -10,6 +10,12 @@ import ( "testing" "time" + "github.com/jackc/pgerrcode" + "github.com/jackc/pgx/v5/pgconn" + "github.com/jackc/pgx/v5/pgxpool" + "github.com/joho/godotenv" + "github.com/stretchr/testify/require" + connsnowflake "github.com/PeerDB-io/peer-flow/connectors/snowflake" "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/e2e" @@ -19,11 +25,6 @@ import ( "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/PeerDB-io/peer-flow/shared" peerflow "github.com/PeerDB-io/peer-flow/workflows" - "github.com/jackc/pgerrcode" - "github.com/jackc/pgx/v5/pgconn" - "github.com/jackc/pgx/v5/pgxpool" - "github.com/joho/godotenv" - "github.com/stretchr/testify/require" ) type PeerFlowE2ETestSuiteSF struct { @@ -754,7 +755,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Types_SF() { require.NoError(s.t, err) // Make sure that there are no nulls - require.Equal(s.t, noNulls, true) + require.True(s.t, noNulls) } func (s PeerFlowE2ETestSuiteSF) Test_Multi_Table_SF() { @@ -1259,9 +1260,9 @@ func (s PeerFlowE2ETestSuiteSF) Test_Column_Exclusion() { require.NoError(s.t, err) for _, field := range sfRows.Schema.Fields { - require.NotEqual(s.t, field.Name, "c2") + require.NotEqual(s.t, "c2", field.Name) } - require.Equal(s.t, 5, len(sfRows.Schema.Fields)) + require.Len(s.t, sfRows.Schema.Fields, 5) } func (s PeerFlowE2ETestSuiteSF) Test_Soft_Delete_Basic() { diff --git a/flow/e2e/snowflake/qrep_flow_sf_test.go b/flow/e2e/snowflake/qrep_flow_sf_test.go index b23ec8bb67..ac90de1770 100644 --- a/flow/e2e/snowflake/qrep_flow_sf_test.go +++ b/flow/e2e/snowflake/qrep_flow_sf_test.go @@ -3,10 +3,11 @@ package e2e_snowflake import ( "fmt" - "github.com/PeerDB-io/peer-flow/e2e" - "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/google/uuid" "github.com/stretchr/testify/require" + + "github.com/PeerDB-io/peer-flow/e2e" + "github.com/PeerDB-io/peer-flow/generated/protos" ) //nolint:unparam @@ -25,7 +26,7 @@ func (s PeerFlowE2ETestSuiteSF) checkJSONValue(tableName, colName, fieldName, va return fmt.Errorf("json value check failed: %v", err) } - jsonVal := res.Records[0].Entries[0].Value + jsonVal := res.Records[0][0].Value if jsonVal != value { return fmt.Errorf("bad json value in field %s of column %s: %v. expected: %v", fieldName, colName, jsonVal, value) } diff --git a/flow/e2e/snowflake/snowflake_helper.go b/flow/e2e/snowflake/snowflake_helper.go index 88ce61e60d..1c4b9d2bb9 100644 --- a/flow/e2e/snowflake/snowflake_helper.go +++ b/flow/e2e/snowflake/snowflake_helper.go @@ -159,21 +159,21 @@ func (s *SnowflakeTestHelper) RunIntQuery(query string) (int, error) { } rec := rows.Records[0] - if rec.NumEntries != 1 { - return 0, fmt.Errorf("failed to execute query: %s, returned %d != 1 columns", query, rec.NumEntries) + if len(rec) != 1 { + return 0, fmt.Errorf("failed to execute query: %s, returned %d != 1 columns", query, len(rec)) } - switch rec.Entries[0].Kind { + switch rec[0].Kind { case qvalue.QValueKindInt32: - return int(rec.Entries[0].Value.(int32)), nil + return int(rec[0].Value.(int32)), nil case qvalue.QValueKindInt64: - return int(rec.Entries[0].Value.(int64)), nil + return int(rec[0].Value.(int64)), nil case qvalue.QValueKindNumeric: // get big.Rat and convert to int - rat := rec.Entries[0].Value.(*big.Rat) + rat := rec[0].Value.(*big.Rat) return int(rat.Num().Int64() / rat.Denom().Int64()), nil default: - return 0, fmt.Errorf("failed to execute query: %s, returned value of type %s", query, rec.Entries[0].Kind) + return 0, fmt.Errorf("failed to execute query: %s, returned value of type %s", query, rec[0].Kind) } } @@ -185,7 +185,7 @@ func (s *SnowflakeTestHelper) checkSyncedAt(query string) error { } for _, record := range recordBatch.Records { - for _, entry := range record.Entries { + for _, entry := range record { if entry.Kind != qvalue.QValueKindTimestamp { return fmt.Errorf("synced_at column check failed: _PEERDB_SYNCED_AT is not timestamp") } diff --git a/flow/e2e/snowflake/snowflake_schema_delta_test.go b/flow/e2e/snowflake/snowflake_schema_delta_test.go index f9f0c1ceca..52f02b005e 100644 --- a/flow/e2e/snowflake/snowflake_schema_delta_test.go +++ b/flow/e2e/snowflake/snowflake_schema_delta_test.go @@ -6,12 +6,13 @@ import ( "log/slog" "testing" + "github.com/stretchr/testify/require" + connsnowflake "github.com/PeerDB-io/peer-flow/connectors/snowflake" "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/e2eshared" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model/qvalue" - "github.com/stretchr/testify/require" ) const schemaDeltaTestSchemaName = "PUBLIC" diff --git a/flow/e2e/sqlserver/qrep_flow_sqlserver_test.go b/flow/e2e/sqlserver/qrep_flow_sqlserver_test.go index f7378ea2ae..3cb9cda650 100644 --- a/flow/e2e/sqlserver/qrep_flow_sqlserver_test.go +++ b/flow/e2e/sqlserver/qrep_flow_sqlserver_test.go @@ -5,20 +5,22 @@ import ( "fmt" "log/slog" "os" + "strconv" "strings" "testing" "time" + "github.com/jackc/pgx/v5/pgtype" + "github.com/jackc/pgx/v5/pgxpool" + "github.com/joho/godotenv" + "github.com/stretchr/testify/require" + "github.com/PeerDB-io/peer-flow/e2e" "github.com/PeerDB-io/peer-flow/e2eshared" "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/shared" - "github.com/jackc/pgx/v5/pgtype" - "github.com/jackc/pgx/v5/pgxpool" - "github.com/joho/godotenv" - "github.com/stretchr/testify/require" ) type PeerFlowE2ETestSuiteSQLServer struct { @@ -95,8 +97,8 @@ func (s PeerFlowE2ETestSuiteSQLServer) insertRowsIntoSQLServerTable(tableName st schemaQualified := fmt.Sprintf("%s.%s", s.sqlsHelper.SchemaName, tableName) for i := 0; i < numRows; i++ { params := make(map[string]interface{}) - params["id"] = "test_id_" + fmt.Sprintf("%d", i) - params["card_id"] = "test_card_id_" + fmt.Sprintf("%d", i) + params["id"] = "test_id_" + strconv.Itoa(i) + params["card_id"] = "test_card_id_" + strconv.Itoa(i) params["v_from"] = time.Now() params["price"] = 100.00 params["status"] = 1 diff --git a/flow/e2e/test_utils.go b/flow/e2e/test_utils.go index 0aa5aaebbb..0638ecae25 100644 --- a/flow/e2e/test_utils.go +++ b/flow/e2e/test_utils.go @@ -13,6 +13,13 @@ import ( "testing" "time" + "github.com/google/uuid" + "github.com/jackc/pgerrcode" + "github.com/jackc/pgx/v5/pgconn" + "github.com/jackc/pgx/v5/pgxpool" + "github.com/stretchr/testify/require" + "go.temporal.io/sdk/testsuite" + "github.com/PeerDB-io/peer-flow/activities" connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" connsnowflake "github.com/PeerDB-io/peer-flow/connectors/snowflake" @@ -25,12 +32,6 @@ import ( "github.com/PeerDB-io/peer-flow/shared" "github.com/PeerDB-io/peer-flow/shared/alerting" peerflow "github.com/PeerDB-io/peer-flow/workflows" - "github.com/google/uuid" - "github.com/jackc/pgerrcode" - "github.com/jackc/pgx/v5/pgconn" - "github.com/jackc/pgx/v5/pgxpool" - "github.com/stretchr/testify/require" - "go.temporal.io/sdk/testsuite" ) func RegisterWorkflowsAndActivities(t *testing.T, env *testsuite.TestWorkflowEnvironment) { @@ -247,6 +248,8 @@ func CreateTableForQRep(pool *pgxpool.Pool, suffix string, tableName string) err "f12 boolean[]", "f13 smallint[]", "my_date DATE", + "old_date DATE", + "my_time TIME", "my_mood mood", "myh HSTORE", `"geometryPoint" geometry(point)`, @@ -309,7 +312,7 @@ func PopulateSourceTable(pool *pgxpool.Pool, suffix string, tableName string, ro '{"2026-01-17 10:00:00","2026-01-18 13:45:00"}', '{true, false}', '{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}', - CURRENT_DATE, 'happy', '"a"=>"b"','POINT(1 2)','POINT(40.7128 -74.0060)', + CURRENT_DATE, CURRENT_TIME,'happy', '"a"=>"b"','POINT(1 2)','POINT(40.7128 -74.0060)', 'LINESTRING(0 0, 1 1, 2 2)', 'LINESTRING(-74.0060 40.7128, -73.9352 40.7306, -73.9123 40.7831)', 'POLYGON((0 0, 0 1, 1 1, 1 0, 0 0))','POLYGON((0 0, 0 1, 1 1, 1 0, 0 0))' @@ -327,7 +330,8 @@ 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, my_mood, myh, + 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; `, suffix, tableName, strings.Join(rows, ","))) @@ -361,6 +365,14 @@ func PopulateSourceTable(pool *pgxpool.Pool, suffix string, tableName string, ro return err } + // update my_date to a date before 1970 + _, err = pool.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 { + return err + } + return nil } @@ -450,6 +462,8 @@ func GetOwnersSchema() *model.QRecordSchema { {Name: "f8", Type: qvalue.QValueKindInt16, Nullable: true}, {Name: "f13", Type: qvalue.QValueKindArrayInt16, Nullable: true}, {Name: "my_date", Type: qvalue.QValueKindDate, Nullable: true}, + {Name: "old_date", Type: qvalue.QValueKindDate, Nullable: true}, + {Name: "my_time", Type: qvalue.QValueKindTime, Nullable: true}, {Name: "my_mood", Type: qvalue.QValueKindString, Nullable: true}, {Name: "geometryPoint", Type: qvalue.QValueKindGeometry, Nullable: true}, {Name: "geometry_linestring", Type: qvalue.QValueKindGeometry, Nullable: true}, diff --git a/flow/e2eshared/e2eshared.go b/flow/e2eshared/e2eshared.go index 176564d342..a9f91dead4 100644 --- a/flow/e2eshared/e2eshared.go +++ b/flow/e2eshared/e2eshared.go @@ -8,8 +8,10 @@ import ( "strings" "testing" - "github.com/PeerDB-io/peer-flow/model" "github.com/jackc/pgx/v5/pgxpool" + + "github.com/PeerDB-io/peer-flow/model" + "github.com/PeerDB-io/peer-flow/model/qvalue" ) type Suite interface { @@ -67,16 +69,16 @@ func ReadFileToBytes(path string) ([]byte, error) { } // checks if two QRecords are identical -func CheckQRecordEquality(t *testing.T, q model.QRecord, other model.QRecord) bool { +func CheckQRecordEquality(t *testing.T, q []qvalue.QValue, other []qvalue.QValue) bool { t.Helper() - if q.NumEntries != other.NumEntries { - t.Logf("unequal entry count: %d != %d", q.NumEntries, other.NumEntries) + if len(q) != len(other) { + t.Logf("unequal entry count: %d != %d", len(q), len(other)) return false } - for i, entry := range q.Entries { - otherEntry := other.Entries[i] + for i, entry := range q { + otherEntry := other[i] if !entry.Equals(otherEntry) { t.Logf("entry %d: %v != %v", i, entry, otherEntry) return false @@ -96,10 +98,10 @@ func CheckEqualRecordBatches(t *testing.T, q *model.QRecordBatch, other *model.Q } // First check simple attributes - if q.NumRecords != other.NumRecords { + if len(q.Records) != len(other.Records) { // print num records - t.Logf("q.NumRecords: %d", q.NumRecords) - t.Logf("other.NumRecords: %d", other.NumRecords) + t.Logf("q.NumRecords: %d", len(q.Records)) + t.Logf("other.NumRecords: %d", len(other.Records)) return false } diff --git a/flow/go.mod b/flow/go.mod index 5239fe48b1..8548882df5 100644 --- a/flow/go.mod +++ b/flow/go.mod @@ -4,13 +4,16 @@ go 1.21 require ( cloud.google.com/go v0.112.0 - cloud.google.com/go/bigquery v1.57.1 + cloud.google.com/go/bigquery v1.58.0 cloud.google.com/go/storage v1.36.0 - github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.4.0 - github.com/Azure/azure-sdk-for-go/sdk/messaging/azeventhubs v1.0.2 + github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.5.1 + github.com/Azure/azure-sdk-for-go/sdk/messaging/azeventhubs v1.0.3 github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/eventhub/armeventhub v1.2.0 github.com/ClickHouse/clickhouse-go/v2 v2.17.1 - github.com/aws/aws-sdk-go v1.49.20 + github.com/aws/aws-sdk-go-v2 v1.24.1 + github.com/aws/aws-sdk-go-v2/credentials v1.16.16 + github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.15.13 + github.com/aws/aws-sdk-go-v2/service/s3 v1.48.0 github.com/cenkalti/backoff/v4 v4.2.1 github.com/cockroachdb/pebble v0.0.0-20231210175914-b4d301aeb46a github.com/google/uuid v1.5.0 @@ -18,7 +21,7 @@ require ( github.com/grpc-ecosystem/grpc-gateway/v2 v2.19.0 github.com/jackc/pgerrcode v0.0.0-20220416144525-469b46aa5efa github.com/jackc/pglogrepl v0.0.0-20231111135425-1627ab1b5780 - github.com/jackc/pgx/v5 v5.5.1 + github.com/jackc/pgx/v5 v5.5.2 github.com/jmoiron/sqlx v1.3.5 github.com/joho/godotenv v1.5.1 github.com/klauspost/compress v1.17.4 @@ -27,77 +30,82 @@ require ( github.com/microsoft/go-mssqldb v1.6.0 github.com/orcaman/concurrent-map/v2 v2.0.1 github.com/slack-go/slack v0.12.3 - github.com/snowflakedb/gosnowflake v1.7.1 + github.com/snowflakedb/gosnowflake v1.7.2 github.com/stretchr/testify v1.8.4 - github.com/twpayne/go-geos v0.15.0 + github.com/twpayne/go-geos v0.16.1 github.com/urfave/cli/v3 v3.0.0-alpha8 github.com/youmark/pkcs8 v0.0.0-20201027041543-1326539a0a0a go.temporal.io/api v1.26.0 go.temporal.io/sdk v1.25.1 go.uber.org/automaxprocs v1.5.3 + golang.org/x/crypto v0.18.0 golang.org/x/sync v0.6.0 - google.golang.org/api v0.156.0 - google.golang.org/genproto/googleapis/api v0.0.0-20240108191215-35c7eff3a6b1 + google.golang.org/api v0.157.0 + google.golang.org/genproto/googleapis/api v0.0.0-20240116215550-a9fa1716bcac google.golang.org/grpc v1.60.1 google.golang.org/protobuf v1.32.0 ) require ( - github.com/ClickHouse/ch-go v0.58.2 // indirect + github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4 // indirect + github.com/99designs/keyring v1.2.2 // indirect + github.com/ClickHouse/ch-go v0.61.1 // indirect github.com/DataDog/zstd v1.5.5 // indirect + github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c // indirect + github.com/apache/arrow/go/v14 v14.0.2 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect github.com/cockroachdb/errors v1.11.1 // indirect github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b // indirect github.com/cockroachdb/redact v1.1.5 // indirect github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06 // indirect + github.com/danieljoos/wincred v1.2.1 // indirect + github.com/dvsekhvalnov/jose2go v1.6.0 // indirect github.com/felixge/httpsnoop v1.0.4 // indirect + github.com/gabriel-vasile/mimetype v1.4.3 // indirect github.com/getsentry/sentry-go v0.26.0 // indirect github.com/go-faster/city v1.0.1 // indirect - github.com/go-faster/errors v0.6.1 // indirect + github.com/go-faster/errors v0.7.1 // indirect github.com/go-logr/logr v1.4.1 // indirect github.com/go-logr/stdr v1.2.2 // indirect + github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2 // indirect github.com/gorilla/websocket v1.5.1 // indirect + github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c // indirect github.com/kr/pretty v0.3.1 // indirect github.com/kr/text v0.2.0 // indirect - github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0 // indirect - github.com/paulmach/orb v0.10.0 // indirect + github.com/mtibben/percent v0.2.1 // indirect + github.com/paulmach/orb v0.11.0 // indirect github.com/pkg/errors v0.9.1 // indirect github.com/prometheus/client_golang v1.18.0 // indirect github.com/prometheus/client_model v0.5.0 // indirect - github.com/prometheus/common v0.45.0 // indirect + github.com/prometheus/common v0.46.0 // indirect github.com/prometheus/procfs v0.12.0 // indirect github.com/rogpeppe/go-internal v1.12.0 // indirect github.com/segmentio/asm v1.2.0 // indirect github.com/shopspring/decimal v1.3.1 // indirect github.com/sirupsen/logrus v1.9.3 // indirect - go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.46.1 // indirect - go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.46.1 // indirect - go.opentelemetry.io/otel v1.21.0 // indirect - go.opentelemetry.io/otel/metric v1.21.0 // indirect - go.opentelemetry.io/otel/trace v1.21.0 // indirect + go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.47.0 // indirect + go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.47.0 // indirect + go.opentelemetry.io/otel v1.22.0 // indirect + go.opentelemetry.io/otel/metric v1.22.0 // indirect + go.opentelemetry.io/otel/trace v1.22.0 // indirect go.uber.org/atomic v1.11.0 // indirect + golang.org/x/term v0.16.0 // indirect ) require ( cloud.google.com/go/compute v1.23.3 // indirect cloud.google.com/go/compute/metadata v0.2.3 // indirect cloud.google.com/go/iam v1.1.5 // indirect - github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4 // indirect - github.com/99designs/keyring v1.2.2 // indirect github.com/Azure/azure-sdk-for-go/sdk/azcore v1.9.1 // indirect github.com/Azure/azure-sdk-for-go/sdk/internal v1.5.1 // indirect github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.2.1 // indirect - github.com/Azure/go-amqp v1.0.3 // indirect + github.com/Azure/go-amqp v1.0.4 // indirect github.com/AzureAD/microsoft-authentication-library-for-go v1.2.1 // indirect - github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c // indirect github.com/andybalholm/brotli v1.1.0 // indirect github.com/apache/arrow/go/v12 v12.0.1 // indirect github.com/apache/thrift v0.19.0 // indirect - github.com/aws/aws-sdk-go-v2 v1.24.1 // indirect github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.4 // indirect - github.com/aws/aws-sdk-go-v2/credentials v1.16.14 // indirect - github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.15.11 // indirect github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.10 // indirect github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.10 // indirect github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.10 // indirect @@ -105,16 +113,11 @@ require ( github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.10 // indirect github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.10 // indirect github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.10 // indirect - github.com/aws/aws-sdk-go-v2/service/s3 v1.48.0 // indirect github.com/aws/smithy-go v1.19.0 // indirect - github.com/danieljoos/wincred v1.2.1 // indirect github.com/davecgh/go-spew v1.1.1 // indirect - github.com/dvsekhvalnov/jose2go v1.6.0 // indirect github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a // indirect github.com/form3tech-oss/jwt-go v3.2.5+incompatible // indirect - github.com/gabriel-vasile/mimetype v1.4.3 // indirect github.com/goccy/go-json v0.10.2 // indirect - github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2 // indirect github.com/gogo/googleapis v1.4.1 // indirect github.com/gogo/protobuf v1.3.2 // indirect github.com/gogo/status v1.1.1 // indirect @@ -132,7 +135,6 @@ require ( github.com/grafana/pyroscope-go/godeltaprof v0.1.6 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway v1.16.0 // indirect - github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c // indirect github.com/jackc/pgio v1.0.0 // indirect github.com/jackc/pgpassfile v1.0.0 // indirect github.com/jackc/pgservicefile v0.0.0-20231201235250-de7065d80cb9 // indirect @@ -143,7 +145,6 @@ require ( github.com/kylelemons/godebug v1.1.0 // indirect github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8 // indirect github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3 // indirect - github.com/mtibben/percent v0.2.1 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pierrec/lz4/v4 v4.1.21 // indirect github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c // indirect @@ -153,19 +154,17 @@ require ( github.com/xrash/smetrics v0.0.0-20231213231151-1d8dd44e695e // indirect github.com/zeebo/xxh3 v1.0.2 // indirect go.opencensus.io v0.24.0 // indirect - golang.org/x/crypto v0.18.0 - golang.org/x/exp v0.0.0-20240112132812-db7319d0e0e3 + golang.org/x/exp v0.0.0-20240119083558-1b970713d09a golang.org/x/mod v0.14.0 // indirect golang.org/x/net v0.20.0 // indirect golang.org/x/oauth2 v0.16.0 // indirect golang.org/x/sys v0.16.0 // indirect - golang.org/x/term v0.16.0 // indirect golang.org/x/text v0.14.0 // indirect golang.org/x/time v0.5.0 // indirect golang.org/x/tools v0.17.0 // indirect golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 // indirect google.golang.org/appengine v1.6.8 // indirect - google.golang.org/genproto v0.0.0-20240108191215-35c7eff3a6b1 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240108191215-35c7eff3a6b1 // indirect + google.golang.org/genproto v0.0.0-20240116215550-a9fa1716bcac // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240116215550-a9fa1716bcac // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/flow/go.sum b/flow/go.sum index 7e8d745158..911bb2e9b2 100644 --- a/flow/go.sum +++ b/flow/go.sum @@ -2,8 +2,8 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMT cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.112.0 h1:tpFCD7hpHFlQ8yPwT3x+QeXqc2T6+n6T+hmABHfDUSM= cloud.google.com/go v0.112.0/go.mod h1:3jEEVwZ/MHU4djK5t5RHuKOA/GbLddgTdVubX1qnPD4= -cloud.google.com/go/bigquery v1.57.1 h1:FiULdbbzUxWD0Y4ZGPSVCDLvqRSyCIO6zKV7E2nf5uA= -cloud.google.com/go/bigquery v1.57.1/go.mod h1:iYzC0tGVWt1jqSzBHqCr3lrRn0u13E8e+AqowBsDgug= +cloud.google.com/go/bigquery v1.58.0 h1:drSd9RcPVLJP2iFMimvOB9SCSIrcl+9HD4II03Oy7A0= +cloud.google.com/go/bigquery v1.58.0/go.mod h1:0eh4mWNY0KrBTjUzLjoYImapGORq9gEPT7MWjCy9lik= cloud.google.com/go/compute v1.23.3 h1:6sVlXXBmbd7jNX0Ipq0trII3e4n1/MsADLK6a+aiVlk= cloud.google.com/go/compute v1.23.3/go.mod h1:VCgBUoMnIVIR0CscqQiPJLAG25E3ZRZMzcFZeQ+h8CI= cloud.google.com/go/compute/metadata v0.2.3 h1:mg4jlk7mCAj6xXp9UJ4fjI9VUI5rubuGBW5aJ7UnBMY= @@ -22,12 +22,12 @@ github.com/99designs/keyring v1.2.2 h1:pZd3neh/EmUzWONb35LxQfvuY7kiSXAq3HQd97+XB github.com/99designs/keyring v1.2.2/go.mod h1:wes/FrByc8j7lFOAGLGSNEg8f/PaI3cgTBqhFkHUrPk= github.com/Azure/azure-sdk-for-go/sdk/azcore v1.9.1 h1:lGlwhPtrX6EVml1hO0ivjkUxsSyl4dsiw9qcA1k/3IQ= github.com/Azure/azure-sdk-for-go/sdk/azcore v1.9.1/go.mod h1:RKUqNu35KJYcVG/fqTRqmuXJZYNhYkBrnC/hX7yGbTA= -github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.4.0 h1:BMAjVKJM0U/CYF27gA0ZMmXGkOcvfFtD0oHVZ1TIPRI= -github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.4.0/go.mod h1:1fXstnBMas5kzG+S3q8UoJcmyU6nUeunJcMDHcRYHhs= +github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.5.1 h1:sO0/P7g68FrryJzljemN+6GTssUXdANk6aJ7T1ZxnsQ= +github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.5.1/go.mod h1:h8hyGFDsU5HMivxiS2iYFZsgDbU9OnnJ163x5UGVKYo= github.com/Azure/azure-sdk-for-go/sdk/internal v1.5.1 h1:6oNBlSdi1QqM1PNW7FPA6xOGA5UNsXnkaYZz9vdPGhA= github.com/Azure/azure-sdk-for-go/sdk/internal v1.5.1/go.mod h1:s4kgfzA0covAXNicZHDMN58jExvcng2mC/DepXiF1EI= -github.com/Azure/azure-sdk-for-go/sdk/messaging/azeventhubs v1.0.2 h1:ujuMdFIUqhfohvpjjt7YmWn6Wk5Vlw9cwtGC0/BEwLU= -github.com/Azure/azure-sdk-for-go/sdk/messaging/azeventhubs v1.0.2/go.mod h1:P39PnDHXbDhUV+BVw/8Nb7wQnM76jKUA7qx5T7eS+BU= +github.com/Azure/azure-sdk-for-go/sdk/messaging/azeventhubs v1.0.3 h1:zkAs5JZZm1Yr4lxLUj3xt2FLgKmvcwGt3a94iJ8rgew= +github.com/Azure/azure-sdk-for-go/sdk/messaging/azeventhubs v1.0.3/go.mod h1:P39PnDHXbDhUV+BVw/8Nb7wQnM76jKUA7qx5T7eS+BU= github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/eventhub/armeventhub v1.2.0 h1:+dggnR89/BIIlRlQ6d19dkhhdd/mQUiQbXhyHUFiB4w= github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/eventhub/armeventhub v1.2.0/go.mod h1:tI9M2Q/ueFi287QRkdrhb9LHm6ZnXgkVYLRC3FhYkPw= github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/internal/v2 v2.0.0 h1:PTFGRSlMKCQelWwxUyYVEUqseBJVemLyqWJjvMyt0do= @@ -42,13 +42,13 @@ github.com/Azure/azure-sdk-for-go/sdk/security/keyvault/internal v0.8.0 h1:T028g github.com/Azure/azure-sdk-for-go/sdk/security/keyvault/internal v0.8.0/go.mod h1:cw4zVQgBby0Z5f2v0itn6se2dDP17nTjbZFXW5uPyHA= github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.2.1 h1:AMf7YbZOZIW5b66cXNHMWWT/zkjhz5+a+k/3x40EO7E= github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.2.1/go.mod h1:uwfk06ZBcvL/g4VHNjurPfVln9NMbsk2XIZxJ+hu81k= -github.com/Azure/go-amqp v1.0.3 h1:wEXQyG2tromTkSdTTOhpYSmosOlJjtVX5UVk2pxyC0Y= -github.com/Azure/go-amqp v1.0.3/go.mod h1:vZAogwdrkbyK3Mla8m/CxSc/aKdnTZ4IbPxl51Y5WZE= +github.com/Azure/go-amqp v1.0.4 h1:GX5OFOs706UjuFRD5PDKm3aOuLQ92F7DMbua+DKAYCc= +github.com/Azure/go-amqp v1.0.4/go.mod h1:vZAogwdrkbyK3Mla8m/CxSc/aKdnTZ4IbPxl51Y5WZE= github.com/AzureAD/microsoft-authentication-library-for-go v1.2.1 h1:DzHpqpoJVaCgOUdVHxE8QB52S6NiVdDQvGlny1qvPqA= github.com/AzureAD/microsoft-authentication-library-for-go v1.2.1/go.mod h1:wP83P5OoQ5p6ip3ScPr0BAq0BvuPAvacpEuSzyouqAI= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/ClickHouse/ch-go v0.58.2 h1:jSm2szHbT9MCAB1rJ3WuCJqmGLi5UTjlNu+f530UTS0= -github.com/ClickHouse/ch-go v0.58.2/go.mod h1:Ap/0bEmiLa14gYjCiRkYGbXvbe8vwdrfTYWhsuQ99aw= +github.com/ClickHouse/ch-go v0.61.1 h1:j5rx3qnvcnYjhnP1IdXE/vdIRQiqgwAzyqOaasA6QCw= +github.com/ClickHouse/ch-go v0.61.1/go.mod h1:myxt/JZgy2BYHFGQqzmaIpbfr5CMbs3YHVULaWQj5YU= github.com/ClickHouse/clickhouse-go/v2 v2.17.1 h1:ZCmAYWpu75IyEi7+Yrs/uaAjiCGY5wfW5kXo64exkX4= github.com/ClickHouse/clickhouse-go/v2 v2.17.1/go.mod h1:rkGTvFDTLqLIm0ma+13xmcCfr/08Gvs7KmFt1tgiWHQ= github.com/DataDog/zstd v1.5.5 h1:oWf5W7GtOLgp6bciQYDmhHHjdhYkALu6S/5Ni9ZgSvQ= @@ -64,22 +64,22 @@ github.com/andybalholm/brotli v1.1.0/go.mod h1:sms7XGricyQI9K10gOSf56VKKWS4oLer5 github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= github.com/apache/arrow/go/v12 v12.0.1 h1:JsR2+hzYYjgSUkBSaahpqCetqZMr76djX80fF/DiJbg= github.com/apache/arrow/go/v12 v12.0.1/go.mod h1:weuTY7JvTG/HDPtMQxEUp7pU73vkLWMLpY67QwZ/WWw= +github.com/apache/arrow/go/v14 v14.0.2 h1:N8OkaJEOfI3mEZt07BIkvo4sC6XDbL+48MBPWO5IONw= +github.com/apache/arrow/go/v14 v14.0.2/go.mod h1:u3fgh3EdgN/YQ8cVQRguVW3R+seMybFg8QBQ5LU+eBY= github.com/apache/thrift v0.19.0 h1:sOqkWPzMj7w6XaYbJQG7m4sGqVolaW/0D28Ln7yPzMk= github.com/apache/thrift v0.19.0/go.mod h1:SUALL216IiaOw2Oy+5Vs9lboJ/t9g40C+G07Dc0QC1I= -github.com/aws/aws-sdk-go v1.49.20 h1:VgEUq2/ZbUkLbqPyDcxrirfXB+PgiZUUF5XbsgWe2S0= -github.com/aws/aws-sdk-go v1.49.20/go.mod h1:LF8svs817+Nz+DmiMQKTO3ubZ/6IaTpq3TjupRn3Eqk= github.com/aws/aws-sdk-go-v2 v1.24.1 h1:xAojnj+ktS95YZlDf0zxWBkbFtymPeDP+rvUQIH3uAU= github.com/aws/aws-sdk-go-v2 v1.24.1/go.mod h1:LNh45Br1YAkEKaAqvmE1m8FUx6a5b/V0oAKV7of29b4= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.4 h1:OCs21ST2LrepDfD3lwlQiOqIGp6JiEUqG84GzTDoyJs= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.4/go.mod h1:usURWEKSNNAcAZuzRn/9ZYPT8aZQkR7xcCtunK/LkJo= -github.com/aws/aws-sdk-go-v2/config v1.26.3 h1:dKuc2jdp10y13dEEvPqWxqLoc0vF3Z9FC45MvuQSxOA= -github.com/aws/aws-sdk-go-v2/config v1.26.3/go.mod h1:Bxgi+DeeswYofcYO0XyGClwlrq3DZEXli0kLf4hkGA0= -github.com/aws/aws-sdk-go-v2/credentials v1.16.14 h1:mMDTwwYO9A0/JbOCOG7EOZHtYM+o7OfGWfu0toa23VE= -github.com/aws/aws-sdk-go-v2/credentials v1.16.14/go.mod h1:cniAUh3ErQPHtCQGPT5ouvSAQ0od8caTO9OOuufZOAE= +github.com/aws/aws-sdk-go-v2/config v1.26.5 h1:lodGSevz7d+kkFJodfauThRxK9mdJbyutUxGq1NNhvw= +github.com/aws/aws-sdk-go-v2/config v1.26.5/go.mod h1:DxHrz6diQJOc9EwDslVRh84VjjrE17g+pVZXUeSxaDU= +github.com/aws/aws-sdk-go-v2/credentials v1.16.16 h1:8q6Rliyv0aUFAVtzaldUEcS+T5gbadPbWdV1WcAddK8= +github.com/aws/aws-sdk-go-v2/credentials v1.16.16/go.mod h1:UHVZrdUsv63hPXFo1H7c5fEneoVo9UXiz36QG1GEPi0= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.11 h1:c5I5iH+DZcH3xOIMlz3/tCKJDaHFwYEmxvlh2fAcFo8= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.11/go.mod h1:cRrYDYAMUohBJUtUnOhydaMHtiK/1NZ0Otc9lIb6O0Y= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.15.11 h1:I6lAa3wBWfCz/cKkOpAcumsETRkFAl70sWi8ItcMEsM= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.15.11/go.mod h1:be1NIO30kJA23ORBLqPo1LttEM6tPNSEcjkd1eKzNW0= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.15.13 h1:8Nt4LBUEKV0FxLBO2BmRzDKax3hp2LRMKySMBwL4vMc= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.15.13/go.mod h1:t5QEDu/FBJJM4kslbQlTSpYtnhoWDNmHSsgQojIxE0o= github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.10 h1:vF+Zgd9s+H4vOXd5BMaPWykta2a6Ih0AKLq/X6NYKn4= github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.10/go.mod h1:6BkRjejp/GR4411UGqkX8+wFMbFbqsUIimfK4XjOKR4= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.10 h1:nYPe006ktcqUji8S2mqXf9c/7NdiKriOwMvWQHgYztw= @@ -98,10 +98,10 @@ github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.10 h1:KOxnQeWy5sXyS github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.10/go.mod h1:jMx5INQFYFYB3lQD9W0D8Ohgq6Wnl7NYOJ2TQndbulI= github.com/aws/aws-sdk-go-v2/service/s3 v1.48.0 h1:PJTdBMsyvra6FtED7JZtDpQrIAflYDHFoZAu/sKYkwU= github.com/aws/aws-sdk-go-v2/service/s3 v1.48.0/go.mod h1:4qXHrG1Ne3VGIMZPCB8OjH/pLFO94sKABIusjh0KWPU= -github.com/aws/aws-sdk-go-v2/service/sso v1.18.6 h1:dGrs+Q/WzhsiUKh82SfTVN66QzyulXuMDTV/G8ZxOac= -github.com/aws/aws-sdk-go-v2/service/sso v1.18.6/go.mod h1:+mJNDdF+qiUlNKNC3fxn74WWNN+sOiGOEImje+3ScPM= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.6 h1:Yf2MIo9x+0tyv76GljxzqA3WtC5mw7NmazD2chwjxE4= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.6/go.mod h1:ykf3COxYI0UJmxcfcxcVuz7b6uADi1FkiUz6Eb7AgM8= +github.com/aws/aws-sdk-go-v2/service/sso v1.18.7 h1:eajuO3nykDPdYicLlP3AGgOyVN3MOlFmZv7WGTuJPow= +github.com/aws/aws-sdk-go-v2/service/sso v1.18.7/go.mod h1:+mJNDdF+qiUlNKNC3fxn74WWNN+sOiGOEImje+3ScPM= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.7 h1:QPMJf+Jw8E1l7zqhZmMlFw6w1NmfkfiSK8mS4zOx3BA= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.7/go.mod h1:ykf3COxYI0UJmxcfcxcVuz7b6uADi1FkiUz6Eb7AgM8= github.com/aws/aws-sdk-go-v2/service/sts v1.26.7 h1:NzO4Vrau795RkUdSHKEwiR01FaGzGOH1EETJ+5QHnm0= github.com/aws/aws-sdk-go-v2/service/sts v1.26.7/go.mod h1:6h2YuIoxaMSCFf5fi1EgZAwdfkGMgDY+DVfa61uLe4U= github.com/aws/smithy-go v1.19.0 h1:KWFKQV80DpP3vJrrA9sVAHQ5gc2z8i4EzrLhLlWXcBM= @@ -165,8 +165,8 @@ github.com/go-errors/errors v1.4.2 h1:J6MZopCL4uSllY1OfXM374weqZFFItUbrImctkmUxI github.com/go-errors/errors v1.4.2/go.mod h1:sIVyrIiJhuEF+Pj9Ebtd6P/rEYROXFi3BopGUQ5a5Og= github.com/go-faster/city v1.0.1 h1:4WAxSZ3V2Ws4QRDrscLEDcibJY8uf41H6AhXDrNDcGw= github.com/go-faster/city v1.0.1/go.mod h1:jKcUJId49qdW3L1qKHH/3wPeUstCVpVSXTM6vO3VcTw= -github.com/go-faster/errors v0.6.1 h1:nNIPOBkprlKzkThvS/0YaX8Zs9KewLCOSFQS5BU06FI= -github.com/go-faster/errors v0.6.1/go.mod h1:5MGV2/2T9yvlrbhe9pD9LO5Z/2zCSq2T8j+Jpi2LAyY= +github.com/go-faster/errors v0.7.1 h1:MkJTnDoEdi9pDabt1dpWf7AA8/BaSYZqibYyhZ20AYg= +github.com/go-faster/errors v0.7.1/go.mod h1:5ySTjWFiphBs07IKuiL69nxdfd5+fzh1u7FPGZP2quo= github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= @@ -274,8 +274,8 @@ github.com/jackc/pgservicefile v0.0.0-20200714003250-2b9c44734f2b/go.mod h1:vsD4 github.com/jackc/pgservicefile v0.0.0-20231201235250-de7065d80cb9 h1:L0QtFUgDarD7Fpv9jeVMgy/+Ec0mtnmYuImjTz6dtDA= github.com/jackc/pgservicefile v0.0.0-20231201235250-de7065d80cb9/go.mod h1:5TJZWKEWniPve33vlWYSoGYefn3gLQRzjfDlhSJ9ZKM= github.com/jackc/pgx/v5 v5.0.3/go.mod h1:JBbvW3Hdw77jKl9uJrEDATUZIFM2VFPzRq4RWIhkF4o= -github.com/jackc/pgx/v5 v5.5.1 h1:5I9etrGkLrN+2XPCsi6XLlV5DITbSL/xBZdmAxFcXPI= -github.com/jackc/pgx/v5 v5.5.1/go.mod h1:Ig06C2Vu0t5qXC60W8sqIthScaEnFvojjj9dSljmHRA= +github.com/jackc/pgx/v5 v5.5.2 h1:iLlpgp4Cp/gC9Xuscl7lFL1PhhW+ZLtXZcrfCt4C3tA= +github.com/jackc/pgx/v5 v5.5.2/go.mod h1:ez9gk+OAat140fv9ErkZDYFWmXLfV+++K0uAOiwgm1A= github.com/jackc/puddle/v2 v2.0.0/go.mod h1:itE7ZJY8xnoo0JqJEpSMprN0f+NQkMCuEV/N9j8h0oc= github.com/jackc/puddle/v2 v2.2.1 h1:RhxXJtFG022u4ibrCSMSiu5aOq1i77R3OHKNJj77OAk= github.com/jackc/puddle/v2 v2.2.1/go.mod h1:vriiEXHvEE654aYKXXjOvZM39qJ0q+azkZFrfEOc3H4= @@ -316,8 +316,6 @@ github.com/linkedin/goavro/v2 v2.12.0 h1:rIQQSj8jdAUlKQh6DttK8wCRv4t4QO09g1C4aBW github.com/linkedin/goavro/v2 v2.12.0/go.mod h1:KXx+erlq+RPlGSPmLF7xGo6SAbh8sCQ53x064+ioxhk= github.com/mattn/go-sqlite3 v1.14.6 h1:dNPt6NO46WmLVt2DLNpwczCmdV5boIZ6g/tlDrlRUbg= github.com/mattn/go-sqlite3 v1.14.6/go.mod h1:NyWgC/yNuGj7Q9rpYnZvas74GogHl5/Z4A/KQRfk6bU= -github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0 h1:jWpvCLoY8Z/e3VKvlsiIGKtc+UG6U5vzxaoagmhXfyg= -github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0/go.mod h1:QUyp042oQthUoa9bqDv0ER0wrtXnBruoNd7aNjkbP+k= github.com/microsoft/go-mssqldb v1.6.0 h1:mM3gYdVwEPFrlg/Dvr2DNVEgYFG7L42l+dGc67NNNpc= github.com/microsoft/go-mssqldb v1.6.0/go.mod h1:00mDtPbeQCRGC1HwOOR5K/gr30P1NcEG0vx6Kbv2aJU= github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8 h1:AMFGa4R4MiIpspGNG7Z948v4n35fFGB3RR3G/ry4FWs= @@ -331,8 +329,8 @@ github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLA github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/orcaman/concurrent-map/v2 v2.0.1 h1:jOJ5Pg2w1oeB6PeDurIYf6k9PQ+aTITr/6lP/L/zp6c= github.com/orcaman/concurrent-map/v2 v2.0.1/go.mod h1:9Eq3TG2oBe5FirmYWQfYO5iH1q0Jv47PLaNK++uCdOM= -github.com/paulmach/orb v0.10.0 h1:guVYVqzxHE/CQ1KpfGO077TR0ATHSNjp4s6XGLn3W9s= -github.com/paulmach/orb v0.10.0/go.mod h1:5mULz1xQfs3bmQm63QEJA6lNGujuRafwA5S/EnuLaLU= +github.com/paulmach/orb v0.11.0 h1:JfVXJUBeH9ifc/OrhBY0lL16QsmPgpCHMlqSSYhcgAA= +github.com/paulmach/orb v0.11.0/go.mod h1:5mULz1xQfs3bmQm63QEJA6lNGujuRafwA5S/EnuLaLU= github.com/paulmach/protoscan v0.2.1/go.mod h1:SpcSwydNLrxUGSDvXvO0P7g7AuhJ7lcKfDlhJCDw2gY= github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= @@ -355,8 +353,8 @@ github.com/prometheus/client_golang v1.18.0/go.mod h1:T+GXkCk5wSJyOqMIzVgvvjFDlk github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.5.0 h1:VQw1hfvPvk3Uv6Qf29VrPF32JB6rtbgI6cYPYQjL0Qw= github.com/prometheus/client_model v0.5.0/go.mod h1:dTiFglRmd66nLR9Pv9f0mZi7B7fk5Pm3gvsjB5tr+kI= -github.com/prometheus/common v0.45.0 h1:2BGz0eBc2hdMDLnO/8n0jeB3oPrt2D08CekT0lneoxM= -github.com/prometheus/common v0.45.0/go.mod h1:YJmSTw9BoKxJplESWWxlbyttQR4uaEcGyv9MZjVOJsY= +github.com/prometheus/common v0.46.0 h1:doXzt5ybi1HBKpsZOL0sSkaNHJJqkyfEWZGGqqScV0Y= +github.com/prometheus/common v0.46.0/go.mod h1:Tp0qkxpb9Jsg54QMe+EAmqXkSV7Evdy1BTn+g2pa/hQ= github.com/prometheus/procfs v0.12.0 h1:jluTpSng7V9hY0O2R9DzzJHYb2xULk9VTR1V1R/k6Bo= github.com/prometheus/procfs v0.12.0/go.mod h1:pcuDEFsWDnvcgNzo4EEweacyhjeA9Zk3cnaOZAZEfOo= github.com/robfig/cron v1.2.0 h1:ZjScXvvxeQ63Dbyxy76Fj3AT3Ut0aKsyd2/tl3DTMuQ= @@ -375,8 +373,8 @@ github.com/sirupsen/logrus v1.9.3 h1:dueUQJ1C2q9oE3F7wvmSGAaVtTmUizReu6fjN8uqzbQ github.com/sirupsen/logrus v1.9.3/go.mod h1:naHLuLoDiP4jHNo9R0sCBMtWGeIprob74mVsIT4qYEQ= github.com/slack-go/slack v0.12.3 h1:92/dfFU8Q5XP6Wp5rr5/T5JHLM5c5Smtn53fhToAP88= github.com/slack-go/slack v0.12.3/go.mod h1:hlGi5oXA+Gt+yWTPP0plCdRKmjsDxecdHxYQdlMQKOw= -github.com/snowflakedb/gosnowflake v1.7.1 h1:c9JjyjjDlvxex9ud71TwKL+Wu54Vfx+39h4DAwbIdqU= -github.com/snowflakedb/gosnowflake v1.7.1/go.mod h1:JI3eRZL8CpimPek6CJO0aTbDQjDGOt7Rxv9A/ti4f5c= +github.com/snowflakedb/gosnowflake v1.7.2 h1:HRSwva8YXC64WUppfmHcMNVVzSE1+EwXXaJxgS0EkTo= +github.com/snowflakedb/gosnowflake v1.7.2/go.mod h1:03tW856vc3ceM4rJuj7KO4dzqN7qoezTm+xw7aPIIFo= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= @@ -397,8 +395,8 @@ github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/tidwall/pretty v1.0.0/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhVysOjyk= -github.com/twpayne/go-geos v0.15.0 h1:L8RCcbaEDfhRz/HhzOvw8fU2s7SzxBLh1sID125EneY= -github.com/twpayne/go-geos v0.15.0/go.mod h1:zmBwZNTaMTB1usptcCl4n7FjIDoBi2IGtm6h6nq9G8c= +github.com/twpayne/go-geos v0.16.1 h1:2tqZyAtDFxBqKn7HR2/g2tQkC/GuQQ2D8dxgK9BjdV8= +github.com/twpayne/go-geos v0.16.1/go.mod h1:zmBwZNTaMTB1usptcCl4n7FjIDoBi2IGtm6h6nq9G8c= github.com/urfave/cli/v3 v3.0.0-alpha8 h1:H+qxFPoCkGzdF8KUMs2fEOZl5io/1QySgUiGfar8occ= github.com/urfave/cli/v3 v3.0.0-alpha8/go.mod h1:0kK/RUFHyh+yIKSfWxwheGndfnrvYSmYFVeKCh03ZUc= github.com/xdg-go/pbkdf2 v1.0.0/go.mod h1:jrpuAogTd400dnrH08LKmI/xc1MbPOebTwRqcT5RDeI= @@ -420,18 +418,18 @@ github.com/zeebo/xxh3 v1.0.2/go.mod h1:5NWz9Sef7zIDm2JHfFlcQvNekmcEl9ekUZQQKCYaD go.mongodb.org/mongo-driver v1.11.4/go.mod h1:PTSz5yu21bkT/wXpkS7WR5f0ddqw5quethTUn9WM+2g= go.opencensus.io v0.24.0 h1:y73uSU6J157QMP2kn2r30vwW1A2W2WFwSCGnAVxeaD0= go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo= -go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.46.1 h1:SpGay3w+nEwMpfVnbqOLH5gY52/foP8RE8UzTZ1pdSE= -go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.46.1/go.mod h1:4UoMYEZOC0yN/sPGH76KPkkU7zgiEWYWL9vwmbnTJPE= -go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.46.1 h1:aFJWCqJMNjENlcleuuOkGAPH82y0yULBScfXcIEdS24= -go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.46.1/go.mod h1:sEGXWArGqc3tVa+ekntsN65DmVbVeW+7lTKTjZF3/Fo= -go.opentelemetry.io/otel v1.21.0 h1:hzLeKBZEL7Okw2mGzZ0cc4k/A7Fta0uoPgaJCr8fsFc= -go.opentelemetry.io/otel v1.21.0/go.mod h1:QZzNPQPm1zLX4gZK4cMi+71eaorMSGT3A4znnUvNNEo= -go.opentelemetry.io/otel/metric v1.21.0 h1:tlYWfeo+Bocx5kLEloTjbcDwBuELRrIFxwdQ36PlJu4= -go.opentelemetry.io/otel/metric v1.21.0/go.mod h1:o1p3CA8nNHW8j5yuQLdc1eeqEaPfzug24uvsyIEJRWM= +go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.47.0 h1:UNQQKPfTDe1J81ViolILjTKPr9WetKW6uei2hFgJmFs= +go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.47.0/go.mod h1:r9vWsPS/3AQItv3OSlEJ/E4mbrhUbbw18meOjArPtKQ= +go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.47.0 h1:sv9kVfal0MK0wBMCOGr+HeJm9v803BkJxGrk2au7j08= +go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.47.0/go.mod h1:SK2UL73Zy1quvRPonmOmRDiWk1KBV3LyIeeIxcEApWw= +go.opentelemetry.io/otel v1.22.0 h1:xS7Ku+7yTFvDfDraDIJVpw7XPyuHlB9MCiqqX5mcJ6Y= +go.opentelemetry.io/otel v1.22.0/go.mod h1:eoV4iAi3Ea8LkAEI9+GFT44O6T/D0GWAVFyZVCC6pMI= +go.opentelemetry.io/otel/metric v1.22.0 h1:lypMQnGyJYeuYPhOM/bgjbFM6WE44W1/T45er4d8Hhg= +go.opentelemetry.io/otel/metric v1.22.0/go.mod h1:evJGjVpZv0mQ5QBRJoBF64yMuOf4xCWdXjK8pzFvliY= go.opentelemetry.io/otel/sdk v1.21.0 h1:FTt8qirL1EysG6sTQRZ5TokkU8d0ugCj8htOgThZXQ8= go.opentelemetry.io/otel/sdk v1.21.0/go.mod h1:Nna6Yv7PWTdgJHVRD9hIYywQBRx7pbox6nwBnZIxl/E= -go.opentelemetry.io/otel/trace v1.21.0 h1:WD9i5gzvoUPuXIXH24ZNBudiarZDKuekPqi/E8fpfLc= -go.opentelemetry.io/otel/trace v1.21.0/go.mod h1:LGbsEB0f9LGjN+OZaQQ26sohbOmiMR+BaslueVtS/qQ= +go.opentelemetry.io/otel/trace v1.22.0 h1:Hg6pPujv0XG9QaVbGOBVHunyuLcCC3jN7WEhPx83XD0= +go.opentelemetry.io/otel/trace v1.22.0/go.mod h1:RbbHXVqKES9QhzZq/fE5UnOSILqRt40a21sPw2He1xo= go.temporal.io/api v1.26.0 h1:N4V0Daqa0qqK5+9LELSZV7clBYrwB4l33iaFfKgycPk= go.temporal.io/api v1.26.0/go.mod h1:uVAcpQJ6bM4mxZ3m7vSHU65fHjrwy9ktGQMtsNfMZQQ= go.temporal.io/sdk v1.25.1 h1:jC9l9vHHz5OJ7PR6OjrpYSN4+uEG0bLe5rdF9nlMSGk= @@ -454,8 +452,8 @@ golang.org/x/crypto v0.0.0-20220829220503-c86fa9a7ed90/go.mod h1:IxCIyHEi3zRg3s0 golang.org/x/crypto v0.18.0 h1:PGVlW0xEltQnzFZ55hkuX5+KLyrMYhHld1YHO4AKcdc= golang.org/x/crypto v0.18.0/go.mod h1:R0j02AL6hcrfOiy9T4ZYp/rcWeMxM3L6QYxlOuEG1mg= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20240112132812-db7319d0e0e3 h1:hNQpMuAJe5CtcUqCXaWga3FHu+kQvCqcsoVaQgSV60o= -golang.org/x/exp v0.0.0-20240112132812-db7319d0e0e3/go.mod h1:idGWGoKP1toJGkd5/ig9ZLuPcZBC3ewk7SzmH0uou08= +golang.org/x/exp v0.0.0-20240119083558-1b970713d09a h1:Q8/wZp0KX97QFTc2ywcOE0YRjZPVIx+MXInMzdvQqcA= +golang.org/x/exp v0.0.0-20240119083558-1b970713d09a/go.mod h1:idGWGoKP1toJGkd5/ig9ZLuPcZBC3ewk7SzmH0uou08= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= @@ -548,10 +546,10 @@ golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8T golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 h1:+cNy6SZtPcJQH3LJVLOSmiC7MMxXNOb3PU/VUEz+EhU= golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028/go.mod h1:NDW/Ps6MPRej6fsCIbMTohpP40sJ/P/vI1MoTEGwX90= -gonum.org/v1/gonum v0.11.0 h1:f1IJhK4Km5tBJmaiJXtk/PkL4cdVX6J+tGiM187uT5E= -gonum.org/v1/gonum v0.11.0/go.mod h1:fSG4YDCxxUZQJ7rKsQrj0gMOg00Il0Z96/qMA4bVQhA= -google.golang.org/api v0.156.0 h1:yloYcGbBtVYjLKQe4enCunxvwn3s2w/XPrrhVf6MsvQ= -google.golang.org/api v0.156.0/go.mod h1:bUSmn4KFO0Q+69zo9CNIDp4Psi6BqM0np0CbzKRSiSY= +gonum.org/v1/gonum v0.12.0 h1:xKuo6hzt+gMav00meVPUlXwSdoEJP46BR+wdxQEFK2o= +gonum.org/v1/gonum v0.12.0/go.mod h1:73TDxJfAAHeA8Mk9mf8NlIppyhQNo5GLTcYeqgo2lvY= +google.golang.org/api v0.157.0 h1:ORAeqmbrrozeyw5NjnMxh7peHO0UzV4wWYSwZeCUb20= +google.golang.org/api v0.157.0/go.mod h1:+z4v4ufbZ1WEpld6yMGHyggs+PmAHiaLNj5ytP3N01g= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.6.8 h1:IhEN5q69dyKagZPYMSdIjS2HqprW324FRQZJcGqPAsM= @@ -562,12 +560,12 @@ google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98 google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200513103714-09dca8ec2884/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= -google.golang.org/genproto v0.0.0-20240108191215-35c7eff3a6b1 h1:/IWabOtPziuXTEtI1KYCpM6Ss7vaAkeMxk+uXV/xvZs= -google.golang.org/genproto v0.0.0-20240108191215-35c7eff3a6b1/go.mod h1:+Rvu7ElI+aLzyDQhpHMFMMltsD6m7nqpuWDd2CwJw3k= -google.golang.org/genproto/googleapis/api v0.0.0-20240108191215-35c7eff3a6b1 h1:OPXtXn7fNMaXwO3JvOmF1QyTc00jsSFFz1vXXBOdCDo= -google.golang.org/genproto/googleapis/api v0.0.0-20240108191215-35c7eff3a6b1/go.mod h1:B5xPO//w8qmBDjGReYLpR6UJPnkldGkCSMoH/2vxJeg= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240108191215-35c7eff3a6b1 h1:gphdwh0npgs8elJ4T6J+DQJHPVF7RsuJHCfwztUb4J4= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240108191215-35c7eff3a6b1/go.mod h1:daQN87bsDqDoe316QbbvX60nMoJQa4r6Ds0ZuoAe5yA= +google.golang.org/genproto v0.0.0-20240116215550-a9fa1716bcac h1:ZL/Teoy/ZGnzyrqK/Optxxp2pmVh+fmJ97slxSRyzUg= +google.golang.org/genproto v0.0.0-20240116215550-a9fa1716bcac/go.mod h1:+Rvu7ElI+aLzyDQhpHMFMMltsD6m7nqpuWDd2CwJw3k= +google.golang.org/genproto/googleapis/api v0.0.0-20240116215550-a9fa1716bcac h1:OZkkudMUu9LVQMCoRUbI/1p5VCo9BOrlvkqMvWtqa6s= +google.golang.org/genproto/googleapis/api v0.0.0-20240116215550-a9fa1716bcac/go.mod h1:B5xPO//w8qmBDjGReYLpR6UJPnkldGkCSMoH/2vxJeg= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240116215550-a9fa1716bcac h1:nUQEQmH/csSvFECKYRv6HWEyypysidKl2I6Qpsglq/0= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240116215550-a9fa1716bcac/go.mod h1:daQN87bsDqDoe316QbbvX60nMoJQa4r6Ds0ZuoAe5yA= google.golang.org/grpc v1.12.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= diff --git a/flow/hstore/hstore.go b/flow/hstore/hstore.go index 0253fef2b1..cbb7d60c8a 100644 --- a/flow/hstore/hstore.go +++ b/flow/hstore/hstore.go @@ -40,11 +40,11 @@ func (p *hstoreParser) atEnd() bool { } // consume returns the next byte of the string, or end if the string is done. -func (p *hstoreParser) consume() (b byte, end bool) { +func (p *hstoreParser) consume() (byte, bool) { if p.pos >= len(p.str) { return 0, true } - b = p.str[p.pos] + b := p.str[p.pos] p.pos++ return b, false } diff --git a/flow/model/conversion_avro.go b/flow/model/conversion_avro.go index 6399e54b5d..1bbbcc2cb2 100644 --- a/flow/model/conversion_avro.go +++ b/flow/model/conversion_avro.go @@ -8,14 +8,14 @@ import ( ) type QRecordAvroConverter struct { - QRecord QRecord + QRecord []qvalue.QValue TargetDWH qvalue.QDWHType NullableFields map[string]struct{} ColNames []string } func NewQRecordAvroConverter( - q QRecord, + q []qvalue.QValue, targetDWH qvalue.QDWHType, nullableFields map[string]struct{}, colNames []string, @@ -31,18 +31,18 @@ func NewQRecordAvroConverter( func (qac *QRecordAvroConverter) Convert() (map[string]interface{}, error) { m := map[string]interface{}{} - for idx := range qac.QRecord.Entries { + for idx, val := range qac.QRecord { key := qac.ColNames[idx] _, nullable := qac.NullableFields[key] avroConverter := qvalue.NewQValueAvroConverter( - qac.QRecord.Entries[idx], + val, qac.TargetDWH, nullable, ) avroVal, err := avroConverter.ToAvroValue() if err != nil { - return nil, fmt.Errorf("failed to convert QValue to Avro-compatible value: %v", err) + return nil, fmt.Errorf("failed to convert QValue to Avro-compatible value: %w", err) } m[key] = avroVal @@ -100,7 +100,7 @@ func GetAvroSchemaDefinition( avroSchemaJSON, err := json.Marshal(avroSchema) if err != nil { - return nil, fmt.Errorf("failed to marshal Avro schema to JSON: %v", err) + return nil, fmt.Errorf("failed to marshal Avro schema to JSON: %w", err) } return &QRecordAvroSchemaDefinition{ diff --git a/flow/model/model.go b/flow/model/model.go index 776157559d..dd875a2583 100644 --- a/flow/model/model.go +++ b/flow/model/model.go @@ -6,12 +6,12 @@ import ( "fmt" "math" "math/big" - "slices" "sync/atomic" "time" "github.com/PeerDB-io/peer-flow/generated/protos" hstore_util "github.com/PeerDB-io/peer-flow/hstore" + "github.com/PeerDB-io/peer-flow/model/numeric" "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/PeerDB-io/peer-flow/peerdbenv" ) @@ -57,8 +57,8 @@ type PullRecordsRequest struct { } type Record interface { - // GetCheckPointID returns the ID of the record. - GetCheckPointID() int64 + // GetCheckpointID returns the ID of the record. + GetCheckpointID() int64 // get table name GetDestinationTableName() string // get columns and values for the record @@ -215,7 +215,7 @@ func (r *RecordItems) toMap(hstoreAsJSON bool) (map[string]interface{}, error) { if !ok { return nil, errors.New("expected *big.Rat value") } - jsonStruct[col] = bigRat.FloatString(9) + jsonStruct[col] = bigRat.FloatString(numeric.PeerDBNumericScale) case qvalue.QValueKindFloat64: floatVal, ok := v.Value.(float64) if !ok { @@ -338,8 +338,8 @@ type InsertRecord struct { SourceTableName string // Name of the destination table DestinationTableName string - // CheckPointID is the ID of the record. - CheckPointID int64 + // CheckpointID is the ID of the record. + CheckpointID int64 // CommitID is the ID of the commit corresponding to this record. CommitID int64 // Items is a map of column name to value. @@ -347,8 +347,8 @@ type InsertRecord struct { } // Implement Record interface for InsertRecord. -func (r *InsertRecord) GetCheckPointID() int64 { - return r.CheckPointID +func (r *InsertRecord) GetCheckpointID() int64 { + return r.CheckpointID } func (r *InsertRecord) GetDestinationTableName() string { @@ -362,8 +362,8 @@ func (r *InsertRecord) GetItems() *RecordItems { type UpdateRecord struct { // Name of the source table SourceTableName string - // CheckPointID is the ID of the record. - CheckPointID int64 + // CheckpointID is the ID of the record. + CheckpointID int64 // Name of the destination table DestinationTableName string // OldItems is a map of column name to value. @@ -375,8 +375,8 @@ type UpdateRecord struct { } // Implement Record interface for UpdateRecord. -func (r *UpdateRecord) GetCheckPointID() int64 { - return r.CheckPointID +func (r *UpdateRecord) GetCheckpointID() int64 { + return r.CheckpointID } // Implement Record interface for UpdateRecord. @@ -393,8 +393,8 @@ type DeleteRecord struct { SourceTableName string // Name of the destination table DestinationTableName string - // CheckPointID is the ID of the record. - CheckPointID int64 + // CheckpointID is the ID of the record. + CheckpointID int64 // Items is a map of column name to value. Items *RecordItems // unchanged toast columns, filled from latest UpdateRecord @@ -402,8 +402,8 @@ type DeleteRecord struct { } // Implement Record interface for DeleteRecord. -func (r *DeleteRecord) GetCheckPointID() int64 { - return r.CheckPointID +func (r *DeleteRecord) GetCheckpointID() int64 { + return r.CheckpointID } func (r *DeleteRecord) GetDestinationTableName() string { @@ -424,13 +424,11 @@ type CDCRecordStream struct { // Records are a list of json objects. records chan Record // Schema changes from the slot - SchemaDeltas chan *protos.TableSchemaDelta - // Relation message mapping - RelationMessageMapping chan RelationMessageMapping + SchemaDeltas []*protos.TableSchemaDelta // Indicates if the last checkpoint has been set. lastCheckpointSet bool - // lastCheckPointID is the last ID of the commit that corresponds to this batch. - lastCheckPointID atomic.Int64 + // lastCheckpointID is the last ID of the commit that corresponds to this batch. + lastCheckpointID atomic.Int64 // empty signal to indicate if the records are going to be empty or not. emptySignal chan bool } @@ -438,22 +436,20 @@ type CDCRecordStream struct { func NewCDCRecordStream() *CDCRecordStream { channelBuffer := peerdbenv.PeerDBCDCChannelBufferSize() return &CDCRecordStream{ - records: make(chan Record, channelBuffer), - // TODO (kaushik): more than 1024 schema deltas can cause problems! - SchemaDeltas: make(chan *protos.TableSchemaDelta, 1<<10), - emptySignal: make(chan bool, 1), - RelationMessageMapping: make(chan RelationMessageMapping, 1), - lastCheckpointSet: false, - lastCheckPointID: atomic.Int64{}, + records: make(chan Record, channelBuffer), + SchemaDeltas: make([]*protos.TableSchemaDelta, 0), + emptySignal: make(chan bool, 1), + lastCheckpointSet: false, + lastCheckpointID: atomic.Int64{}, } } func (r *CDCRecordStream) UpdateLatestCheckpoint(val int64) { // TODO update with https://github.com/golang/go/issues/63999 once implemented - // r.lastCheckPointID.Max(val) - oldLast := r.lastCheckPointID.Load() - for oldLast < val && !r.lastCheckPointID.CompareAndSwap(oldLast, val) { - oldLast = r.lastCheckPointID.Load() + // r.lastCheckpointID.Max(val) + oldLast := r.lastCheckpointID.Load() + for oldLast < val && !r.lastCheckpointID.CompareAndSwap(oldLast, val) { + oldLast = r.lastCheckpointID.Load() } } @@ -461,7 +457,7 @@ func (r *CDCRecordStream) GetLastCheckpoint() (int64, error) { if !r.lastCheckpointSet { return 0, errors.New("last checkpoint not set, stream is still active") } - return r.lastCheckPointID.Load(), nil + return r.lastCheckpointID.Load(), nil } func (r *CDCRecordStream) AddRecord(record Record) { @@ -481,41 +477,29 @@ func (r *CDCRecordStream) WaitAndCheckEmpty() bool { return isEmpty } -func (r *CDCRecordStream) WaitForSchemaDeltas(tableMappings []*protos.TableMapping) []*protos.TableSchemaDelta { - schemaDeltas := make([]*protos.TableSchemaDelta, 0) -schemaLoop: - for delta := range r.SchemaDeltas { - for _, tm := range tableMappings { - if delta.SrcTableName == tm.SourceTableIdentifier && delta.DstTableName == tm.DestinationTableIdentifier { - if len(tm.Exclude) == 0 { - break - } - added := make([]*protos.DeltaAddedColumn, 0, len(delta.AddedColumns)) - for _, column := range delta.AddedColumns { - if !slices.Contains(tm.Exclude, column.ColumnName) { - added = append(added, column) - } - } - if len(added) != 0 { - schemaDeltas = append(schemaDeltas, &protos.TableSchemaDelta{ - SrcTableName: delta.SrcTableName, - DstTableName: delta.DstTableName, - AddedColumns: added, - }) - } - continue schemaLoop +func (r *CDCRecordStream) AddSchemaDelta(tableNameMapping map[string]NameAndExclude, delta *protos.TableSchemaDelta) { + if tm, ok := tableNameMapping[delta.SrcTableName]; ok && len(tm.Exclude) != 0 { + added := make([]*protos.DeltaAddedColumn, 0, len(delta.AddedColumns)) + for _, column := range delta.AddedColumns { + if _, has := tm.Exclude[column.ColumnName]; !has { + added = append(added, column) } } - schemaDeltas = append(schemaDeltas, delta) + if len(added) != 0 { + r.SchemaDeltas = append(r.SchemaDeltas, &protos.TableSchemaDelta{ + SrcTableName: delta.SrcTableName, + DstTableName: delta.DstTableName, + AddedColumns: added, + }) + } + } else { + r.SchemaDeltas = append(r.SchemaDeltas, delta) } - return schemaDeltas } func (r *CDCRecordStream) Close() { close(r.emptySignal) close(r.records) - close(r.SchemaDeltas) - close(r.RelationMessageMapping) r.lastCheckpointSet = true } @@ -524,7 +508,8 @@ func (r *CDCRecordStream) GetRecords() <-chan Record { } type SyncRecordsRequest struct { - Records *CDCRecordStream + SyncBatchID int64 + Records *CDCRecordStream // FlowJobName is the name of the flow job. FlowJobName string // SyncMode to use for pushing raw records @@ -545,8 +530,8 @@ type NormalizeRecordsRequest struct { } type SyncResponse struct { - // LastSyncedCheckPointID is the last ID that was synced. - LastSyncedCheckPointID int64 + // LastSyncedCheckpointID is the last ID that was synced. + LastSyncedCheckpointID int64 // NumRecordsSynced is the number of records that were synced. NumRecordsSynced int64 // CurrentSyncBatchID is the ID of the currently synced batch. @@ -579,13 +564,13 @@ type NormalizeResponse struct { // being clever and passing the delta back as a regular record instead of heavy CDC refactoring. type RelationRecord struct { - CheckPointID int64 - TableSchemaDelta *protos.TableSchemaDelta + CheckpointID int64 `json:"checkpointId"` + TableSchemaDelta *protos.TableSchemaDelta `json:"tableSchemaDelta"` } // Implement Record interface for RelationRecord. -func (r *RelationRecord) GetCheckPointID() int64 { - return r.CheckPointID +func (r *RelationRecord) GetCheckpointID() int64 { + return r.CheckpointID } func (r *RelationRecord) GetDestinationTableName() string { diff --git a/flow/model/numeric/scale.go b/flow/model/numeric/scale.go new file mode 100644 index 0000000000..5ce1ecd024 --- /dev/null +++ b/flow/model/numeric/scale.go @@ -0,0 +1,3 @@ +package numeric + +const PeerDBNumericScale = 9 diff --git a/flow/model/qrecord.go b/flow/model/qrecord.go deleted file mode 100644 index ab7fbca24e..0000000000 --- a/flow/model/qrecord.go +++ /dev/null @@ -1,23 +0,0 @@ -package model - -import ( - "github.com/PeerDB-io/peer-flow/model/qvalue" -) - -type QRecord struct { - NumEntries int - Entries []qvalue.QValue -} - -// create a new QRecord with n values -func NewQRecord(n int) QRecord { - return QRecord{ - NumEntries: n, - Entries: make([]qvalue.QValue, n), - } -} - -// Sets the value at the given index -func (q QRecord) Set(idx int, value qvalue.QValue) { - q.Entries[idx] = value -} diff --git a/flow/model/qrecord_batch.go b/flow/model/qrecord_batch.go index 4729e04baa..cf888e1418 100644 --- a/flow/model/qrecord_batch.go +++ b/flow/model/qrecord_batch.go @@ -6,23 +6,23 @@ import ( "math/big" "time" - "github.com/PeerDB-io/peer-flow/geo" - "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/google/uuid" "github.com/jackc/pgx/v5/pgtype" + + "github.com/PeerDB-io/peer-flow/geo" + "github.com/PeerDB-io/peer-flow/model/qvalue" ) -// QRecordBatch holds a batch of QRecord objects. +// QRecordBatch holds a batch of []QValue slices type QRecordBatch struct { - NumRecords uint32 // NumRecords represents the number of records in the batch. - Records []QRecord - Schema *QRecordSchema + Records [][]qvalue.QValue + Schema *QRecordSchema } func (q *QRecordBatch) ToQRecordStream(buffer int) (*QRecordStream, error) { stream := NewQRecordStream(buffer) - slog.Info(fmt.Sprintf("Converting %d records to QRecordStream", q.NumRecords)) + slog.Info(fmt.Sprintf("Converting %d records to QRecordStream", len(q.Records))) go func() { err := stream.SetSchema(q.Schema) @@ -89,10 +89,10 @@ func (src *QRecordBatchCopyFromSource) Values() ([]interface{}, error) { } record := src.currentRecord.Record - numEntries := len(record.Entries) + numEntries := len(record) values := make([]interface{}, numEntries) - for i, qValue := range record.Entries { + for i, qValue := range record { if qValue.Value == nil { values[i] = nil continue @@ -147,6 +147,15 @@ func (src *QRecordBatchCopyFromSource) Values() ([]interface{}, error) { } values[i] = v + case qvalue.QValueKindTime: + t, ok := qValue.Value.(time.Time) + if !ok { + src.err = fmt.Errorf("invalid Time value") + return nil, src.err + } + time := pgtype.Time{Microseconds: t.UnixMicro(), Valid: true} + values[i] = time + case qvalue.QValueKindTimestamp: t, ok := qValue.Value.(time.Time) if !ok { diff --git a/flow/model/qrecord_stream.go b/flow/model/qrecord_stream.go index a293e66ebb..83ac85032d 100644 --- a/flow/model/qrecord_stream.go +++ b/flow/model/qrecord_stream.go @@ -1,9 +1,13 @@ package model -import "fmt" +import ( + "fmt" + + "github.com/PeerDB-io/peer-flow/model/qvalue" +) type QRecordOrError struct { - Record QRecord + Record []qvalue.QValue Err error } diff --git a/flow/model/qrecord_test.go b/flow/model/qrecord_test.go index 0cd6022e52..6c685a8f41 100644 --- a/flow/model/qrecord_test.go +++ b/flow/model/qrecord_test.go @@ -4,11 +4,11 @@ import ( "math/big" "testing" - "github.com/PeerDB-io/peer-flow/e2eshared" - "github.com/PeerDB-io/peer-flow/model" - "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/google/uuid" "github.com/stretchr/testify/assert" + + "github.com/PeerDB-io/peer-flow/e2eshared" + "github.com/PeerDB-io/peer-flow/model/qvalue" ) func TestEquals(t *testing.T) { @@ -17,69 +17,39 @@ func TestEquals(t *testing.T) { tests := []struct { name string - q1 model.QRecord - q2 model.QRecord + q1 []qvalue.QValue + q2 []qvalue.QValue want bool }{ { name: "Equal - Same UUID", - q1: model.QRecord{ - NumEntries: 1, - Entries: []qvalue.QValue{{Kind: qvalue.QValueKindUUID, Value: uuidVal1}}, - }, - q2: model.QRecord{ - NumEntries: 1, - Entries: []qvalue.QValue{ - {Kind: qvalue.QValueKindString, Value: uuidVal1.String()}, - }, - }, + q1: []qvalue.QValue{{Kind: qvalue.QValueKindUUID, Value: uuidVal1}}, + q2: []qvalue.QValue{{Kind: qvalue.QValueKindString, Value: uuidVal1.String()}}, want: true, }, { name: "Not Equal - Different UUID", - q1: model.QRecord{ - NumEntries: 1, - Entries: []qvalue.QValue{{Kind: qvalue.QValueKindUUID, Value: uuidVal1}}, - }, - q2: model.QRecord{ - NumEntries: 1, - Entries: []qvalue.QValue{{Kind: qvalue.QValueKindUUID, Value: uuidVal2}}, - }, + q1: []qvalue.QValue{{Kind: qvalue.QValueKindUUID, Value: uuidVal1}}, + q2: []qvalue.QValue{{Kind: qvalue.QValueKindUUID, Value: uuidVal2}}, want: false, }, { name: "Equal - Same numeric", - q1: model.QRecord{ - NumEntries: 1, - Entries: []qvalue.QValue{ - {Kind: qvalue.QValueKindNumeric, Value: big.NewRat(10, 2)}, - }, - }, - q2: model.QRecord{ - NumEntries: 1, - Entries: []qvalue.QValue{{Kind: qvalue.QValueKindString, Value: "5"}}, - }, + q1: []qvalue.QValue{{Kind: qvalue.QValueKindNumeric, Value: big.NewRat(10, 2)}}, + q2: []qvalue.QValue{{Kind: qvalue.QValueKindString, Value: "5"}}, want: true, }, { name: "Not Equal - Different numeric", - q1: model.QRecord{ - NumEntries: 1, - Entries: []qvalue.QValue{ - {Kind: qvalue.QValueKindNumeric, Value: big.NewRat(10, 2)}, - }, - }, - q2: model.QRecord{ - NumEntries: 1, - Entries: []qvalue.QValue{{Kind: qvalue.QValueKindNumeric, Value: "4.99"}}, - }, + q1: []qvalue.QValue{{Kind: qvalue.QValueKindNumeric, Value: big.NewRat(10, 2)}}, + q2: []qvalue.QValue{{Kind: qvalue.QValueKindNumeric, Value: "4.99"}}, want: false, }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - assert.Equal(t, e2eshared.CheckQRecordEquality(t, tt.q1, tt.q2), tt.want) + assert.Equal(t, tt.want, e2eshared.CheckQRecordEquality(t, tt.q1, tt.q2)) }) } } diff --git a/flow/model/qvalue/avro_converter.go b/flow/model/qvalue/avro_converter.go index b682d277f5..cfb016994a 100644 --- a/flow/model/qvalue/avro_converter.go +++ b/flow/model/qvalue/avro_converter.go @@ -6,9 +6,11 @@ import ( "math/big" "time" - hstore_util "github.com/PeerDB-io/peer-flow/hstore" "github.com/google/uuid" "github.com/linkedin/goavro/v2" + + hstore_util "github.com/PeerDB-io/peer-flow/hstore" + "github.com/PeerDB-io/peer-flow/model/numeric" ) // https://avro.apache.org/docs/1.11.0/spec.html @@ -148,7 +150,7 @@ func (c *QValueAvroConverter) ToAvroValue() (interface{}, error) { case QValueKindInvalid: // we will attempt to convert invalid to a string return c.processNullableUnion("string", c.Value.Value) - case QValueKindTime, QValueKindTimeTZ, QValueKindTimestamp, QValueKindTimestampTZ: + case QValueKindTime: t, err := c.processGoTime() if err != nil || t == nil { return t, err @@ -161,6 +163,54 @@ func (c *QValueAvroConverter) ToAvroValue() (interface{}, error) { } } + if c.TargetDWH == QDWHTypeClickhouse { + if c.Nullable { + return c.processNullableUnion("long", t.(int64)) + } else { + return t.(int64), nil + } + } + if c.Nullable { + return goavro.Union("long.time-micros", t.(int64)), nil + } + return t.(int64), nil + case QValueKindTimeTZ: + t, err := c.processGoTimeTZ() + if err != nil || t == nil { + return t, err + } + if c.TargetDWH == QDWHTypeSnowflake { + if c.Nullable { + return c.processNullableUnion("string", t.(string)) + } else { + return t.(string), nil + } + } + + if c.TargetDWH == QDWHTypeClickhouse { + if c.Nullable { + return c.processNullableUnion("long", t.(int64)) + } else { + return t.(int64), nil + } + } + if c.Nullable { + return goavro.Union("long.time-micros", t.(int64)), nil + } + return t.(int64), nil + case QValueKindTimestamp: + t, err := c.processGoTimestamp() + if err != nil || t == nil { + return t, err + } + if c.TargetDWH == QDWHTypeSnowflake { + if c.Nullable { + return c.processNullableUnion("string", t.(string)) + } else { + return t.(string), nil + } + } + if c.TargetDWH == QDWHTypeClickhouse { if c.Nullable { return c.processNullableUnion("long", t.(int64)) @@ -172,7 +222,30 @@ func (c *QValueAvroConverter) ToAvroValue() (interface{}, error) { return goavro.Union("long.timestamp-micros", t.(int64)), nil } return t.(int64), nil + case QValueKindTimestampTZ: + t, err := c.processGoTimestampTZ() + if err != nil || t == nil { + return t, err + } + if c.TargetDWH == QDWHTypeSnowflake { + if c.Nullable { + return c.processNullableUnion("string", t.(string)) + } else { + return t.(string), nil + } + } + if c.TargetDWH == QDWHTypeClickhouse { + if c.Nullable { + return c.processNullableUnion("long", t.(int64)) + } else { + return t.(int64), nil + } + } + if c.Nullable { + return goavro.Union("long.timestamp-micros", t.(int64)), nil + } + return t.(int64), nil case QValueKindDate: t, err := c.processGoDate() if err != nil || t == nil { @@ -188,9 +261,8 @@ func (c *QValueAvroConverter) ToAvroValue() (interface{}, error) { if c.Nullable { return goavro.Union("int.date", t), nil - } else { - return t, nil } + return t, nil case QValueKindString, QValueKindCIDR, QValueKindINET, QValueKindMacaddr: if c.TargetDWH == QDWHTypeSnowflake && c.Value.Value != nil && @@ -260,6 +332,24 @@ func (c *QValueAvroConverter) ToAvroValue() (interface{}, error) { } } +func (c *QValueAvroConverter) processGoTimeTZ() (interface{}, error) { + if c.Value.Value == nil && c.Nullable { + return nil, nil + } + + t, ok := c.Value.Value.(time.Time) + if !ok { + return nil, fmt.Errorf("invalid TimeTZ value") + } + + // Snowflake has issues with avro timestamp types, returning as string form + // See: https://stackoverflow.com/questions/66104762/snowflake-date-column-have-incorrect-date-from-avro-file + if c.TargetDWH == QDWHTypeSnowflake { + return t.Format("15:04:05.999999-0700"), nil + } + return t.UnixMicro(), nil +} + func (c *QValueAvroConverter) processGoTime() (interface{}, error) { if c.Value.Value == nil && c.Nullable { return nil, nil @@ -270,13 +360,48 @@ func (c *QValueAvroConverter) processGoTime() (interface{}, error) { return nil, fmt.Errorf("invalid Time value") } - ret := t.UnixMicro() - // Snowflake has issues with avro timestamp types, returning as string form of the int64 + // Snowflake has issues with avro timestamp types, returning as string form + // See: https://stackoverflow.com/questions/66104762/snowflake-date-column-have-incorrect-date-from-avro-file + if c.TargetDWH == QDWHTypeSnowflake { + return t.Format("15:04:05.999999"), nil + } + return t.UnixMicro(), nil +} + +func (c *QValueAvroConverter) processGoTimestampTZ() (interface{}, error) { + if c.Value.Value == nil && c.Nullable { + return nil, nil + } + + t, ok := c.Value.Value.(time.Time) + if !ok { + return nil, fmt.Errorf("invalid TimestampTZ value") + } + + // Snowflake has issues with avro timestamp types, returning as string form + // See: https://stackoverflow.com/questions/66104762/snowflake-date-column-have-incorrect-date-from-avro-file + if c.TargetDWH == QDWHTypeSnowflake { + return t.Format("2006-01-02 15:04:05.999999-0700"), nil + } + return t.UnixMicro(), nil +} + +func (c *QValueAvroConverter) processGoTimestamp() (interface{}, error) { + if c.Value.Value == nil && c.Nullable { + return nil, nil + } + + t, ok := c.Value.Value.(time.Time) + if !ok { + return nil, fmt.Errorf("invalid Timestamp value") + } + + // Snowflake has issues with avro timestamp types, returning as string form // See: https://stackoverflow.com/questions/66104762/snowflake-date-column-have-incorrect-date-from-avro-file if c.TargetDWH == QDWHTypeSnowflake { - return fmt.Sprint(ret), nil + return t.Format("2006-01-02 15:04:05.999999"), nil } - return ret, nil + return t.UnixMicro(), nil } func (c *QValueAvroConverter) processGoDate() (interface{}, error) { @@ -289,11 +414,10 @@ func (c *QValueAvroConverter) processGoDate() (interface{}, error) { return nil, fmt.Errorf("invalid Time value for Date") } - // Snowflake has issues with avro timestamp types, returning as string form of the int64 + // Snowflake has issues with avro timestamp types, returning as string form // See: https://stackoverflow.com/questions/66104762/snowflake-date-column-have-incorrect-date-from-avro-file if c.TargetDWH == QDWHTypeSnowflake { - ret := t.UnixMicro() - return fmt.Sprint(ret), nil + return t.Format("2006-01-02"), nil } return t, nil } @@ -323,6 +447,9 @@ func (c *QValueAvroConverter) processNumeric() (interface{}, error) { return nil, fmt.Errorf("invalid Numeric value: expected *big.Rat, got %T", c.Value.Value) } + scale := numeric.PeerDBNumericScale + decimalValue := num.FloatString(scale) + num.SetString(decimalValue) if c.Nullable { return goavro.Union("bytes.decimal", num), nil } @@ -415,7 +542,7 @@ func (c *QValueAvroConverter) processArrayTime() (interface{}, error) { transformedTimeArr := make([]interface{}, 0, len(arrayTime)) for _, t := range arrayTime { - // Snowflake has issues with avro timestamp types, returning as string form of the int64 + // Snowflake has issues with avro timestamp types, returning as string form // See: https://stackoverflow.com/questions/66104762/snowflake-date-column-have-incorrect-date-from-avro-file if c.TargetDWH == QDWHTypeSnowflake { transformedTimeArr = append(transformedTimeArr, t.String()) @@ -505,7 +632,7 @@ func (c *QValueAvroConverter) processUUID() (interface{}, error) { u, err := uuid.FromBytes(byteData[:]) if err != nil { - return nil, fmt.Errorf("[conversion] conversion of invalid UUID value: %v", err) + return nil, fmt.Errorf("[conversion] conversion of invalid UUID value: %w", err) } uuidString := u.String() diff --git a/flow/model/qvalue/kind.go b/flow/model/qvalue/kind.go index bba156bb88..5c6b069639 100644 --- a/flow/model/qvalue/kind.go +++ b/flow/model/qvalue/kind.go @@ -68,12 +68,12 @@ var QValueKindToSnowflakeTypeMap = map[QValueKind]string{ QValueKindTimestamp: "TIMESTAMP_NTZ", QValueKindTimestampTZ: "TIMESTAMP_TZ", QValueKindTime: "TIME", + QValueKindTimeTZ: "TIME", QValueKindDate: "DATE", QValueKindBit: "BINARY", QValueKindBytes: "BINARY", QValueKindStruct: "STRING", QValueKindUUID: "STRING", - QValueKindTimeTZ: "STRING", QValueKindInvalid: "STRING", QValueKindHStore: "VARIANT", QValueKindGeography: "GEOGRAPHY", diff --git a/flow/model/qvalue/qvalue.go b/flow/model/qvalue/qvalue.go index 786065227b..d64059ca71 100644 --- a/flow/model/qvalue/qvalue.go +++ b/flow/model/qvalue/qvalue.go @@ -10,9 +10,10 @@ import ( "time" "cloud.google.com/go/civil" + "github.com/google/uuid" + "github.com/PeerDB-io/peer-flow/geo" hstore_util "github.com/PeerDB-io/peer-flow/hstore" - "github.com/google/uuid" ) // if new types are added, register them in gob - cdc_records_storage.go @@ -44,9 +45,11 @@ func (q QValue) Equals(other QValue) bool { case QValueKindString: return compareString(q.Value, other.Value) // all internally represented as a Golang time.Time - case QValueKindTime, QValueKindTimeTZ, QValueKindDate, + case QValueKindDate, QValueKindTimestamp, QValueKindTimestampTZ: return compareGoTime(q.Value, other.Value) + case QValueKindTime, QValueKindTimeTZ: + return compareGoCivilTime(q.Value, other.Value) case QValueKindNumeric: return compareNumeric(q.Value, other.Value) case QValueKindBytes: @@ -161,6 +164,29 @@ func compareGoTime(value1, value2 interface{}) bool { return t1 == t2 } +func compareGoCivilTime(value1, value2 interface{}) bool { + if value1 == nil && value2 == nil { + return true + } + + t1, ok1 := value1.(time.Time) + t2, ok2 := value2.(time.Time) + + if !ok1 || !ok2 { + if !ok2 { + // For BigQuery, we need to compare civil.Time with time.Time + ct2, ok3 := value2.(civil.Time) + if !ok3 { + return false + } + return t1.Hour() == ct2.Hour && t1.Minute() == ct2.Minute && t1.Second() == ct2.Second + } + return false + } + + return t1.Hour() == t2.Hour() && t1.Minute() == t2.Minute() && t1.Second() == t2.Second() +} + func compareUUID(value1, value2 interface{}) bool { if value1 == nil && value2 == nil { return true diff --git a/flow/shared/alerting/alerting.go b/flow/shared/alerting/alerting.go index 18c6cc2b9b..98922d22bc 100644 --- a/flow/shared/alerting/alerting.go +++ b/flow/shared/alerting/alerting.go @@ -7,9 +7,10 @@ import ( "log/slog" "time" - "github.com/PeerDB-io/peer-flow/dynamicconf" "github.com/jackc/pgx/v5" "github.com/jackc/pgx/v5/pgxpool" + + "github.com/PeerDB-io/peer-flow/dynamicconf" ) // alerting service, no cool name :( diff --git a/flow/shared/signals.go b/flow/shared/signals.go index 2097ba95c5..5e441d0110 100644 --- a/flow/shared/signals.go +++ b/flow/shared/signals.go @@ -7,16 +7,17 @@ import ( func FlowSignalHandler(activeSignal CDCFlowSignal, v CDCFlowSignal, logger log.Logger, ) CDCFlowSignal { - if v == ShutdownSignal { + switch v { + case ShutdownSignal: logger.Info("received shutdown signal") return v - } else if v == PauseSignal { + case PauseSignal: logger.Info("received pause signal") if activeSignal == NoopSignal { logger.Info("workflow was running, pausing it") return v } - } else if v == NoopSignal { + case NoopSignal: logger.Info("received resume signal") if activeSignal == PauseSignal { logger.Info("workflow was paused, resuming it") diff --git a/flow/workflows/cdc_flow.go b/flow/workflows/cdc_flow.go index 421081071b..f0b3eb385f 100644 --- a/flow/workflows/cdc_flow.go +++ b/flow/workflows/cdc_flow.go @@ -7,16 +7,17 @@ import ( "strings" "time" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/model" - "github.com/PeerDB-io/peer-flow/peerdbenv" - "github.com/PeerDB-io/peer-flow/shared" "github.com/google/uuid" "go.temporal.io/api/enums/v1" "go.temporal.io/sdk/log" "go.temporal.io/sdk/temporal" "go.temporal.io/sdk/workflow" "google.golang.org/protobuf/proto" + + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/model" + "github.com/PeerDB-io/peer-flow/peerdbenv" + "github.com/PeerDB-io/peer-flow/shared" ) const ( diff --git a/flow/workflows/drop_flow.go b/flow/workflows/drop_flow.go index b21d89f198..fd73a90285 100644 --- a/flow/workflows/drop_flow.go +++ b/flow/workflows/drop_flow.go @@ -1,43 +1,38 @@ package peerflow import ( + "errors" + "fmt" "time" + "go.temporal.io/sdk/workflow" + "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/shared" - "go.temporal.io/sdk/log" - "go.temporal.io/sdk/workflow" ) -// DropFlowWorkflowExecution represents the state for execution of a drop flow. -type DropFlowWorkflowExecution struct { - shutDownRequest *protos.ShutdownRequest - flowExecutionID string - logger log.Logger -} - -func newDropFlowWorkflowExecution(ctx workflow.Context, req *protos.ShutdownRequest) *DropFlowWorkflowExecution { - return &DropFlowWorkflowExecution{ - shutDownRequest: req, - flowExecutionID: workflow.GetInfo(ctx).WorkflowExecution.ID, - logger: workflow.GetLogger(ctx), - } -} - func DropFlowWorkflow(ctx workflow.Context, req *protos.ShutdownRequest) error { - execution := newDropFlowWorkflowExecution(ctx, req) - execution.logger.Info("performing cleanup for flow ", req.FlowJobName) + logger := workflow.GetLogger(ctx) + logger.Info("performing cleanup for flow ", req.FlowJobName) ctx = workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ StartToCloseTimeout: 1 * time.Minute, }) ctx = workflow.WithValue(ctx, shared.FlowNameKey, req.FlowJobName) + dropSourceFuture := workflow.ExecuteActivity(ctx, flowable.DropFlowSource, req) + dropDestinationFuture := workflow.ExecuteActivity(ctx, flowable.DropFlowDestination, req) - dropFlowFuture := workflow.ExecuteActivity(ctx, flowable.DropFlow, req) - if err := dropFlowFuture.Get(ctx, nil); err != nil { - return err - } + var sourceError, destinationError error + selector := workflow.NewNamedSelector(ctx, fmt.Sprintf("%s-drop", req.FlowJobName)) + selector.AddFuture(dropSourceFuture, func(f workflow.Future) { + sourceError = f.Get(ctx, nil) + }) + selector.AddFuture(dropDestinationFuture, func(f workflow.Future) { + destinationError = f.Get(ctx, nil) + }) + selector.Select(ctx) + selector.Select(ctx) - return nil + return errors.Join(sourceError, destinationError) } diff --git a/flow/workflows/normalize_flow.go b/flow/workflows/normalize_flow.go index ea8b574aa8..608623c75f 100644 --- a/flow/workflows/normalize_flow.go +++ b/flow/workflows/normalize_flow.go @@ -4,10 +4,11 @@ import ( "fmt" "time" + "go.temporal.io/sdk/workflow" + "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/peerdbenv" - "go.temporal.io/sdk/workflow" ) func NormalizeFlowWorkflow(ctx workflow.Context, diff --git a/flow/workflows/qrep_flow.go b/flow/workflows/qrep_flow.go index 47354a148b..e2ddadd2ae 100644 --- a/flow/workflows/qrep_flow.go +++ b/flow/workflows/qrep_flow.go @@ -6,13 +6,14 @@ import ( "strings" "time" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/shared" "github.com/google/uuid" "go.temporal.io/api/enums/v1" "go.temporal.io/sdk/log" "go.temporal.io/sdk/temporal" "go.temporal.io/sdk/workflow" + + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/shared" ) type QRepFlowExecution struct { @@ -108,10 +109,9 @@ func (q *QRepFlowExecution) SetupWatermarkTableOnDestination(ctx workflow.Contex }) tableSchemaInput := &protos.GetTableSchemaBatchInput{ - PeerConnectionConfig: q.config.SourcePeer, - TableIdentifiers: []string{q.config.WatermarkTable}, - FlowName: q.config.FlowJobName, - SkipPkeyAndReplicaCheck: true, + PeerConnectionConfig: q.config.SourcePeer, + TableIdentifiers: []string{q.config.WatermarkTable}, + FlowName: q.config.FlowJobName, } future := workflow.ExecuteActivity(ctx, flowable.GetTableSchema, tableSchemaInput) diff --git a/flow/workflows/setup_flow.go b/flow/workflows/setup_flow.go index 9b9b168a89..7bee648f4e 100644 --- a/flow/workflows/setup_flow.go +++ b/flow/workflows/setup_flow.go @@ -6,13 +6,13 @@ import ( "sort" "time" + "go.temporal.io/sdk/log" + "go.temporal.io/sdk/workflow" + "golang.org/x/exp/maps" + "github.com/PeerDB-io/peer-flow/activities" "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/generated/protos" - "golang.org/x/exp/maps" - - "go.temporal.io/sdk/log" - "go.temporal.io/sdk/workflow" ) // SetupFlow is the workflow that is responsible for ensuring all the @@ -182,10 +182,9 @@ func (s *SetupFlowExecution) fetchTableSchemaAndSetupNormalizedTables( sort.Strings(sourceTables) tableSchemaInput := &protos.GetTableSchemaBatchInput{ - PeerConnectionConfig: flowConnectionConfigs.Source, - TableIdentifiers: sourceTables, - FlowName: s.cdcFlowName, - SkipPkeyAndReplicaCheck: flowConnectionConfigs.InitialSnapshotOnly, + PeerConnectionConfig: flowConnectionConfigs.Source, + TableIdentifiers: sourceTables, + FlowName: s.cdcFlowName, } future := workflow.ExecuteActivity(ctx, flowable.GetTableSchema, tableSchemaInput) diff --git a/flow/workflows/snapshot_flow.go b/flow/workflows/snapshot_flow.go index 8830d0feec..d38801b599 100644 --- a/flow/workflows/snapshot_flow.go +++ b/flow/workflows/snapshot_flow.go @@ -7,15 +7,15 @@ import ( "strings" "time" - "github.com/PeerDB-io/peer-flow/concurrency" - "github.com/PeerDB-io/peer-flow/connectors/utils" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/shared" "github.com/google/uuid" - "go.temporal.io/sdk/log" "go.temporal.io/sdk/temporal" "go.temporal.io/sdk/workflow" + + "github.com/PeerDB-io/peer-flow/concurrency" + "github.com/PeerDB-io/peer-flow/connectors/utils" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/shared" ) type SnapshotFlowExecution struct { diff --git a/flow/workflows/sync_flow.go b/flow/workflows/sync_flow.go index d2b0e72136..4c930ebf66 100644 --- a/flow/workflows/sync_flow.go +++ b/flow/workflows/sync_flow.go @@ -4,10 +4,11 @@ import ( "fmt" "time" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/model" "go.temporal.io/sdk/log" "go.temporal.io/sdk/workflow" + + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/model" ) type SyncFlowState struct { diff --git a/flow/workflows/xmin_flow.go b/flow/workflows/xmin_flow.go index 32b276ee96..c6885253df 100644 --- a/flow/workflows/xmin_flow.go +++ b/flow/workflows/xmin_flow.go @@ -5,11 +5,12 @@ import ( "fmt" "time" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/shared" "github.com/google/uuid" "go.temporal.io/sdk/log" "go.temporal.io/sdk/workflow" + + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/shared" ) type XminFlowExecution struct { diff --git a/nexus/Cargo.lock b/nexus/Cargo.lock index 76629b157c..3685c4773e 100644 --- a/nexus/Cargo.lock +++ b/nexus/Cargo.lock @@ -79,9 +79,9 @@ dependencies = [ [[package]] name = "anstream" -version = "0.6.7" +version = "0.6.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4cd2405b3ac1faab2990b74d728624cd9fd115651fcecc7c2d8daf01376275ba" +checksum = "6e2e1ebcb11de5c03c67de28a7df593d32191b44939c482e97702baaaa6ab6a5" dependencies = [ "anstyle", "anstyle-parse", @@ -151,9 +151,9 @@ checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" [[package]] name = "async-compression" -version = "0.4.5" +version = "0.4.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bc2d0cfb2a7388d34f590e76686704c494ed7aaceed62ee1ba35cbf363abc2a5" +checksum = "a116f46a969224200a0a97f29cfd4c50e7534e4b4826bd23ea2c3c533039c82c" dependencies = [ "flate2", "futures-core", @@ -302,9 +302,9 @@ checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" [[package]] name = "bitflags" -version = "2.4.1" +version = "2.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "327762f6e5a765692301e5bb513e0d9fef63be86bbc14528052b1cd3e6f03e07" +checksum = "ed570934406eb16438a4e976b1b4500774099c13b8cb96eec99f620f05090ddf" [[package]] name = "bitvec" @@ -402,9 +402,9 @@ checksum = "a2bd12c1caf447e69cd4528f47f94d203fd2582878ecb9e9465484c4148a8223" [[package]] name = "cargo-deb" -version = "2.0.3" +version = "2.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "03ffcdb035e3a2d990cb4c8b37a3f845ba309b9c090dacd66dbc4aec7952a5be" +checksum = "d83090b6ad2812ddd045728406032cb6f6f07e766b66b4748a1becd0a7c07e4e" dependencies = [ "ar", "cargo_toml", @@ -430,9 +430,9 @@ dependencies = [ [[package]] name = "cargo_toml" -version = "0.17.2" +version = "0.18.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8a969e13a7589e9e3e4207e153bae624ade2b5622fb4684a4923b23ec3d57719" +checksum = "802b755090e39835a4b0440fb0bbee0df7495a8b337f63db21e616f7821c7e8c" dependencies = [ "serde", "toml 0.8.8", @@ -513,9 +513,9 @@ dependencies = [ [[package]] name = "clap" -version = "4.4.16" +version = "4.4.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "58e54881c004cec7895b0068a0a954cd5d62da01aef83fa35b1e594497bf5445" +checksum = "1e578d6ec4194633722ccf9544794b71b1385c3c027efe0c55db226fc880865c" dependencies = [ "clap_builder", "clap_derive", @@ -523,9 +523,9 @@ dependencies = [ [[package]] name = "clap_builder" -version = "4.4.16" +version = "4.4.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "59cb82d7f531603d2fd1f507441cdd35184fa81beff7bd489570de7f773460bb" +checksum = "4df4df40ec50c46000231c914968278b1eb05098cf8f1b3a518a95030e71d1c7" dependencies = [ "anstream", "anstyle", @@ -797,9 +797,9 @@ dependencies = [ [[package]] name = "env_logger" -version = "0.10.1" +version = "0.10.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "95b3f3e67048839cb0d0781f445682a35113da7121f7c949db0e2be96a4fbece" +checksum = "4cd405aab171cb85d6735e5c8d9db038c17d3ca007a4d2c25f337935c3d90580" dependencies = [ "humantime", "is-terminal", @@ -889,21 +889,6 @@ version = "1.0.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3f9eec918d3f24069decb9af1554cad7c880e2da24a9afd88aca000531ab82c1" -[[package]] -name = "foreign-types" -version = "0.3.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f6f339eb8adc052cd2ca78910fda869aefa38d22d5cb648e6485e4d3fc06f3b1" -dependencies = [ - "foreign-types-shared", -] - -[[package]] -name = "foreign-types-shared" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "00b0228411908ca8685dba7fc2cdd70ec9990a6e753e89b6ac91a84c40fbaf4b" - [[package]] name = "form_urlencoded" version = "1.2.1" @@ -1077,9 +1062,9 @@ checksum = "d2fabcfbdc87f4758337ca535fb41a6d701b65693ce38287d856d1674551ec9b" [[package]] name = "h2" -version = "0.3.23" +version = "0.3.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b553656127a00601c8ae5590fcfdc118e4083a7924b6cf4ffc1ea4b99dc429d7" +checksum = "bb2c4422095b67ee78da96fbb51a4cc413b3b25883c7717ff7ca1ab31022c9c9" dependencies = [ "bytes", "fnv", @@ -1130,9 +1115,9 @@ checksum = "95505c38b4572b2d910cecb0281560f54b440a19336cbbcb27bf6ce6adc6f5a8" [[package]] name = "hermit-abi" -version = "0.3.3" +version = "0.3.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d77f7ec81a6d05a3abb01ab6eb7590f6083d08449fe5a1c8b1e620283546ccb7" +checksum = "5d3d0e0f38255e7fa3cf31335b3a56f05febd18025f4db5ef7a0cfb4f8da651f" [[package]] name = "hex" @@ -1401,9 +1386,9 @@ checksum = "4ec2a862134d2a7d32d7983ddcdd1c4923530833c9f2ea1a44fc5fa473989058" [[package]] name = "linux-raw-sys" -version = "0.4.12" +version = "0.4.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c4cd1a83af159aa67994778be9070f0ae1bd732942279cabb14f86f986a21456" +checksum = "01cda141df6706de531b6c46c3a33ecca755538219bd484262fa09410c13539c" [[package]] name = "lock_api" @@ -1620,50 +1605,12 @@ version = "1.19.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3fdb12b2476b595f9358c5161aa467c2438859caa136dec86c26fdd2efe17b92" -[[package]] -name = "openssl" -version = "0.10.62" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8cde4d2d9200ad5909f8dac647e29482e07c3a35de8a13fce7c9c7747ad9f671" -dependencies = [ - "bitflags 2.4.1", - "cfg-if", - "foreign-types", - "libc", - "once_cell", - "openssl-macros", - "openssl-sys", -] - -[[package]] -name = "openssl-macros" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a948666b637a0f465e8564c73e89d4dde00d72d4d473cc972f390fc3dcee7d9c" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.48", -] - [[package]] name = "openssl-probe" version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ff011a302c396a5197692431fc1948019154afc178baf7d8e37367442a4601cf" -[[package]] -name = "openssl-sys" -version = "0.9.98" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c1665caf8ab2dc9aef43d1c0023bd904633a6a05cb30b0ad59bec2ae986e57a7" -dependencies = [ - "cc", - "libc", - "pkg-config", - "vcpkg", -] - [[package]] name = "overload" version = "0.1.1" @@ -2056,9 +2003,9 @@ dependencies = [ [[package]] name = "pkg-config" -version = "0.3.28" +version = "0.3.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "69d3587f8a9e599cc7ec2c00e331f71c4e69a5f9a4b8a6efd5b07466b9736f9a" +checksum = "2900ede94e305130c13ddd391e0ab7cbaeb783945ae07a279c268cb05109c6cb" [[package]] name = "postgres" @@ -2079,11 +2026,11 @@ name = "postgres-connection" version = "0.1.0" dependencies = [ "anyhow", - "openssl", - "postgres-openssl", "pt", + "rustls 0.22.2", "tokio", "tokio-postgres", + "tokio-postgres-rustls", "tracing", "urlencoding", ] @@ -2098,19 +2045,6 @@ dependencies = [ "postgres-types", ] -[[package]] -name = "postgres-openssl" -version = "0.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1de0ea6504e07ca78355a6fb88ad0f36cafe9e696cbc6717f16a207f3a60be72" -dependencies = [ - "futures", - "openssl", - "tokio", - "tokio-openssl", - "tokio-postgres", -] - [[package]] name = "postgres-protocol" version = "0.6.6" @@ -2169,9 +2103,9 @@ dependencies = [ [[package]] name = "proc-macro-crate" -version = "3.0.0" +version = "3.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6b2685dd208a3771337d8d386a89840f0f43cd68be8dae90a5f8c2384effc9cd" +checksum = "6d37c51ca738a55da99dc0c4a34860fd675453b8b36209178c2249bb13651284" dependencies = [ "toml_edit 0.21.0", ] @@ -2201,9 +2135,9 @@ dependencies = [ [[package]] name = "proc-macro2" -version = "1.0.76" +version = "1.0.78" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "95fc56cda0b5c3325f5fbbd7ff9fda9e02bb00bb3dac51252d2f1bfa1cb8cc8c" +checksum = "e2422ad645d89c99f8f3e6b88a9fdeca7fabeac836b1002371c4367c8f984aae" dependencies = [ "unicode-ident", ] @@ -2351,9 +2285,9 @@ dependencies = [ [[package]] name = "rayon" -version = "1.8.0" +version = "1.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c27db03db7734835b3f53954b534c91069375ce6ccaa2e065441e07d9b6cdb1" +checksum = "fa7237101a77a10773db45d62004a272517633fbcc3df19d96455ede1122e051" dependencies = [ "either", "rayon-core", @@ -2361,9 +2295,9 @@ dependencies = [ [[package]] name = "rayon-core" -version = "1.12.0" +version = "1.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5ce3fb6ad83f861aac485e76e1985cd109d9a3713802152be56c3b1f0e0658ed" +checksum = "1465873a3dfdaa8ae7cb14b4383657caab0b3e8a0aa9ae8e04b044854c8dfce2" dependencies = [ "crossbeam-deque", "crossbeam-utils", @@ -2425,13 +2359,13 @@ dependencies = [ [[package]] name = "regex" -version = "1.10.2" +version = "1.10.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "380b951a9c5e80ddfd6136919eef32310721aa4aacd4889a8d39124b026ab343" +checksum = "b62dbe01f0b06f9d8dc7d49e05a0785f153b00b2c227856282f671e0318c9b15" dependencies = [ "aho-corasick", "memchr", - "regex-automata 0.4.3", + "regex-automata 0.4.4", "regex-syntax 0.8.2", ] @@ -2446,9 +2380,9 @@ dependencies = [ [[package]] name = "regex-automata" -version = "0.4.3" +version = "0.4.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f804c7828047e88b2d32e2d7fe5a105da8ee3264f01902f796c8e067dc2483f" +checksum = "3b7fa1134405e2ec9353fd416b17f8dacd46c473d7d3fd1cf202706a14eb792a" dependencies = [ "aho-corasick", "memchr", @@ -2611,7 +2545,7 @@ version = "0.38.30" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "322394588aaf33c24007e8bb3238ee3e4c5c09c084ab32bc73890b99ff326bca" dependencies = [ - "bitflags 2.4.1", + "bitflags 2.4.2", "errno", "libc", "linux-raw-sys", @@ -2956,9 +2890,9 @@ dependencies = [ [[package]] name = "smallvec" -version = "1.11.2" +version = "1.13.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4dccd0940a2dcdf68d092b8cbab7dc0ad8fa938bf95787e1b916b0e3d0e8e970" +checksum = "e6ecd384b10a64542d77071bd64bd7b231f4ed5940fba55e98c3de13824cf3d7" [[package]] name = "socket2" @@ -3251,18 +3185,6 @@ dependencies = [ "syn 2.0.48", ] -[[package]] -name = "tokio-openssl" -version = "0.6.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6ffab79df67727f6acf57f1ff743091873c24c579b1e2ce4d8f53e47ded4d63d" -dependencies = [ - "futures-util", - "openssl", - "openssl-sys", - "tokio", -] - [[package]] name = "tokio-postgres" version = "0.7.10" @@ -3289,6 +3211,21 @@ dependencies = [ "whoami", ] +[[package]] +name = "tokio-postgres-rustls" +version = "0.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0ea13f22eda7127c827983bdaf0d7fff9df21c8817bab02815ac277a21143677" +dependencies = [ + "futures", + "ring", + "rustls 0.22.2", + "tokio", + "tokio-postgres", + "tokio-rustls 0.25.0", + "x509-certificate", +] + [[package]] name = "tokio-rustls" version = "0.24.1" @@ -3572,9 +3509,9 @@ checksum = "42ff0bf0c66b8238c6f3b578df37d0b7848e55df8577b3f74f92a69acceeb825" [[package]] name = "unicode-bidi" -version = "0.3.14" +version = "0.3.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6f2528f27a9eb2b21e69c95319b30bd0efd85d09c379741b0f78ea1d86be2416" +checksum = "08f95100a766bf4f8f28f90d77e0a5461bbdb219042e7679bebe79004fed8d75" [[package]] name = "unicode-ident" @@ -3647,9 +3584,9 @@ checksum = "711b9620af191e0cdc7468a8d14e709c3dcdb115b36f838e601583af800a370a" [[package]] name = "uuid" -version = "1.6.1" +version = "1.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5e395fcf16a7a3d8127ec99782007af141946b4795001f876d54fb0d55978560" +checksum = "f00cc9702ca12d3c81455259621e676d0f7251cec66a21e98fe2e9a37db93b2a" dependencies = [ "getrandom", "serde", @@ -3679,12 +3616,6 @@ dependencies = [ "uuid", ] -[[package]] -name = "vcpkg" -version = "0.2.15" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "accd4ea62f7bb7a82fe23066fb0957d48ef677f6eeb8215f372f52e48bb32426" - [[package]] name = "version_check" version = "0.9.4" @@ -4041,9 +3972,9 @@ dependencies = [ [[package]] name = "xattr" -version = "1.2.0" +version = "1.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "914566e6413e7fa959cc394fb30e563ba80f3541fbd40816d4c05a0fc3f2a0f1" +checksum = "8da84f1a25939b27f6820d92aed108f83ff920fdf11a7b19366c27c4cda81d4f" dependencies = [ "libc", "linux-raw-sys", diff --git a/nexus/analyzer/src/lib.rs b/nexus/analyzer/src/lib.rs index a3dbe1b37d..6602782e72 100644 --- a/nexus/analyzer/src/lib.rs +++ b/nexus/analyzer/src/lib.rs @@ -176,11 +176,11 @@ impl<'a> StatementAnalyzer for PeerDDLAnalyzer<'a> { partition_key: table_mapping .partition_key .as_ref() - .map(|s| s.to_string()), + .map(|s| s.value.clone()), exclude: table_mapping .exclude .as_ref() - .map(|ss| ss.iter().map(|s| s.to_string()).collect()) + .map(|ss| ss.iter().map(|s| s.value.clone()).collect()) .unwrap_or_default(), }); } diff --git a/nexus/postgres-connection/Cargo.toml b/nexus/postgres-connection/Cargo.toml index a7110f07be..56632c7d78 100644 --- a/nexus/postgres-connection/Cargo.toml +++ b/nexus/postgres-connection/Cargo.toml @@ -8,9 +8,9 @@ edition = "2021" [dependencies] anyhow = "1" pt = { path = "../pt" } -openssl = "0.10.55" -postgres-openssl = "0.5.0" +rustls = "0.22" urlencoding = "2" tokio-postgres = "0.7.2" +tokio-postgres-rustls = "0.11.1" tokio = { version = "1", features = ["full"] } tracing = "0.1" diff --git a/nexus/postgres-connection/src/lib.rs b/nexus/postgres-connection/src/lib.rs index dfa165b7dc..b4a6c82bdd 100644 --- a/nexus/postgres-connection/src/lib.rs +++ b/nexus/postgres-connection/src/lib.rs @@ -1,7 +1,59 @@ -use openssl::ssl::{SslConnector, SslMethod, SslVerifyMode}; -use postgres_openssl::MakeTlsConnector; use pt::peerdb_peers::PostgresConfig; +use rustls::pki_types::{CertificateDer, ServerName, UnixTime}; +use rustls::{ClientConfig, DigitallySignedStruct, RootCertStore, SignatureScheme}; use std::fmt::Write; +use std::sync::Arc; +use tokio_postgres_rustls::MakeRustlsConnect; + +#[derive(Copy, Clone, Debug)] +struct NoCertificateVerification; + +impl rustls::client::danger::ServerCertVerifier for NoCertificateVerification { + fn verify_server_cert( + &self, + _end_entity: &CertificateDer<'_>, + _intermediates: &[CertificateDer<'_>], + _server_name: &ServerName<'_>, + _ocsp_response: &[u8], + _now: UnixTime, + ) -> std::result::Result { + Ok(rustls::client::danger::ServerCertVerified::assertion()) + } + + fn verify_tls12_signature( + &self, + message: &[u8], + cert: &CertificateDer<'_>, + dss: &DigitallySignedStruct, + ) -> Result { + rustls::crypto::verify_tls12_signature( + message, + cert, + dss, + &rustls::crypto::ring::default_provider().signature_verification_algorithms, + ) + } + + fn verify_tls13_signature( + &self, + message: &[u8], + cert: &CertificateDer<'_>, + dss: &DigitallySignedStruct, + ) -> Result { + rustls::crypto::verify_tls13_signature( + message, + cert, + dss, + &rustls::crypto::ring::default_provider().signature_verification_algorithms, + ) + } + + fn supported_verify_schemes(&self) -> Vec { + rustls::crypto::ring::default_provider() + .signature_verification_algorithms + .supported_schemes() + } +} pub fn get_pg_connection_string(config: &PostgresConfig) -> String { let mut connection_string = String::from("postgres://"); @@ -26,12 +78,13 @@ pub fn get_pg_connection_string(config: &PostgresConfig) -> String { pub async fn connect_postgres(config: &PostgresConfig) -> anyhow::Result { let connection_string = get_pg_connection_string(config); - let mut builder = SslConnector::builder(SslMethod::tls())?; - // NONE seems roughly equivalent to the guarantees of sslmode=prefer or sslmode=require - // PEER seems to be equivalent to sslmode=verify-ca or sslmode=verify-full, requires presence of root certs. - builder.set_verify(SslVerifyMode::NONE); - - let tls_connector = MakeTlsConnector::new(builder.build()); + let mut config = ClientConfig::builder() + .with_root_certificates(RootCertStore::empty()) + .with_no_client_auth(); + config + .dangerous() + .set_certificate_verifier(Arc::new(NoCertificateVerification)); + let tls_connector = MakeRustlsConnect::new(config); let (client, connection) = tokio_postgres::connect(&connection_string, tls_connector) .await .map_err(|e| anyhow::anyhow!("error encountered while connecting to postgres {:?}", e))?; diff --git a/protos/flow.proto b/protos/flow.proto index 27859cbcdd..66ba78c8be 100644 --- a/protos/flow.proto +++ b/protos/flow.proto @@ -184,7 +184,6 @@ message GetTableSchemaBatchInput { peerdb_peers.Peer peer_connection_config = 1; repeated string table_identifiers = 2; string flow_name = 3; - bool skip_pkey_and_replica_check = 4; } message GetTableSchemaBatchOutput { diff --git a/protos/peers.proto b/protos/peers.proto index b16c35fccd..372c02936b 100644 --- a/protos/peers.proto +++ b/protos/peers.proto @@ -8,6 +8,7 @@ message SSHConfig { string user = 3; string password = 4; string private_key = 5; + string host_key = 6; } message SnowflakeConfig { @@ -115,7 +116,7 @@ enum DBType { S3 = 5; SQLSERVER = 6; EVENTHUB_GROUP = 7; - CLICKHOUSE = 8; + CLICKHOUSE = 8; } message Peer { diff --git a/stacks/peerdb-server.Dockerfile b/stacks/peerdb-server.Dockerfile index b83f846de6..789f1e8883 100644 --- a/stacks/peerdb-server.Dockerfile +++ b/stacks/peerdb-server.Dockerfile @@ -12,7 +12,7 @@ FROM chef as builder RUN apt-get update \ && DEBIAN_FRONTEND=noninteractive \ apt-get install --assume-yes --no-install-recommends \ - build-essential libssl-dev pkg-config curl unzip + build-essential pkg-config curl unzip WORKDIR /root/nexus COPY scripts /root/scripts RUN /root/scripts/install-protobuf.sh diff --git a/ui/app/peers/create/[peerType]/helpers/pg.ts b/ui/app/peers/create/[peerType]/helpers/pg.ts index 8e822fb5c4..122b92be2e 100644 --- a/ui/app/peers/create/[peerType]/helpers/pg.ts +++ b/ui/app/peers/create/[peerType]/helpers/pg.ts @@ -86,6 +86,13 @@ export const sshSetting = [ optional: true, tips: 'Private key as a BASE64 string for authentication in order to SSH into your machine.', }, + { + label: 'Host Key', + stateHandler: (value: string, setter: sshSetter) => + setter((curr) => ({ ...curr, hostKey: value })), + optional: true, + tips: 'Public key of host to mitigate MITM attacks when SSHing into your machine.', + }, ]; export const blankSSHConfig: SSHConfig = { @@ -94,6 +101,7 @@ export const blankSSHConfig: SSHConfig = { user: '', password: '', privateKey: '', + hostKey: '', }; export const blankPostgresSetting: PostgresConfig = { diff --git a/ui/app/peers/create/[peerType]/schema.ts b/ui/app/peers/create/[peerType]/schema.ts index 6a21892f39..0bf959b904 100644 --- a/ui/app/peers/create/[peerType]/schema.ts +++ b/ui/app/peers/create/[peerType]/schema.ts @@ -226,11 +226,7 @@ export const bqSchema = z.object({ required_error: 'Dataset ID is required', }) .min(1, { message: 'Dataset ID must be non-empty' }) - .max(1024, 'DatasetID must be less than 1025 characters') - .regex( - /^[\w]+$/, - 'Dataset ID must only contain numbers, letters, and underscores' - ), + .max(1024, 'DatasetID must be less than 1025 characters'), }); export const chSchema = z.object({ diff --git a/ui/components/PeerForms/ClickhouseConfig.tsx b/ui/components/PeerForms/ClickhouseConfig.tsx index 1819e626df..1d76286db0 100644 --- a/ui/components/PeerForms/ClickhouseConfig.tsx +++ b/ui/components/PeerForms/ClickhouseConfig.tsx @@ -137,12 +137,9 @@ export default function PostgresForm({ settings, setter }: ConfigProps) { (sshConfig as SSHConfig)[ sshParam.label === 'BASE64 Private Key' ? 'privateKey' - : (sshParam.label.toLowerCase() as - | 'host' - | 'port' - | 'user' - | 'password' - | 'privateKey') + : sshParam.label === 'Host Key' + ? 'hostKey' + : (sshParam.label.toLowerCase() as keyof SSHConfig) ] || '' } /> diff --git a/ui/components/PeerForms/PostgresForm.tsx b/ui/components/PeerForms/PostgresForm.tsx index 1819e626df..1d76286db0 100644 --- a/ui/components/PeerForms/PostgresForm.tsx +++ b/ui/components/PeerForms/PostgresForm.tsx @@ -137,12 +137,9 @@ export default function PostgresForm({ settings, setter }: ConfigProps) { (sshConfig as SSHConfig)[ sshParam.label === 'BASE64 Private Key' ? 'privateKey' - : (sshParam.label.toLowerCase() as - | 'host' - | 'port' - | 'user' - | 'password' - | 'privateKey') + : sshParam.label === 'Host Key' + ? 'hostKey' + : (sshParam.label.toLowerCase() as keyof SSHConfig) ] || '' } />