diff --git a/.github/workflows/codeql-analysis.yml b/.github/workflows/codeql-analysis.yml new file mode 100644 index 0000000000..08ae2e2f1b --- /dev/null +++ b/.github/workflows/codeql-analysis.yml @@ -0,0 +1,61 @@ +name: "CodeQL" + +on: + push: + branches: [ "main", "customer-*", "stable" ] + pull_request: + branches: [ "main", "customer-*", "stable" ] + schedule: + - cron: '15 17 * * 2' + +jobs: + analyze: + name: Analyze (${{ matrix.language }}) + runs-on: ${{ (matrix.language == 'swift' && 'macos-latest') || 'ubicloud-standard-16' }} + timeout-minutes: ${{ (matrix.language == 'swift' && 120) || 360 }} + permissions: + # required for all workflows + security-events: write + + # required to fetch internal or private CodeQL packs + packages: read + + # only required for workflows in private repositories + actions: read + contents: read + + strategy: + fail-fast: false + matrix: + include: + - language: go + build-mode: autobuild + - language: javascript-typescript + build-mode: none + steps: + - name: Checkout repository + uses: actions/checkout@v4 + + - uses: bufbuild/buf-setup-action@v1.29.0-1 + with: + github_token: ${{ secrets.GITHUB_TOKEN }} + + - run: buf generate protos + + - name: install lib-geos + run: | + sudo apt-get update + sudo apt-get install libgeos-dev + + + # Initializes the CodeQL tools for scanning. + - name: Initialize CodeQL + uses: github/codeql-action/init@v3 + with: + languages: ${{ matrix.language }} + build-mode: ${{ matrix.build-mode }} + + - name: Perform CodeQL Analysis + uses: github/codeql-action/analyze@v3 + with: + category: "/language:${{matrix.language}}" diff --git a/e2e_cleanup/go.mod b/e2e_cleanup/go.mod index 27625f98d9..6fc5bcc146 100644 --- a/e2e_cleanup/go.mod +++ b/e2e_cleanup/go.mod @@ -1,6 +1,6 @@ module github.com/PeerDB-io/peer-flow-cleanup -go 1.22.0 +go 1.22.2 require ( cloud.google.com/go/bigquery v1.59.1 diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index c8de2ba3e4..7c3a41ac58 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -36,7 +36,7 @@ type CheckConnectionResult struct { } type CdcCacheEntry struct { - connector connectors.CDCPullConnector + connector connectors.CDCPullConnectorCore done chan struct{} } @@ -133,9 +133,9 @@ func (a *FlowableActivity) GetTableSchema( config *protos.GetTableSchemaBatchInput, ) (*protos.GetTableSchemaBatchOutput, error) { ctx = context.WithValue(ctx, shared.FlowNameKey, config.FlowName) - srcConn, err := connectors.GetCDCPullConnector(ctx, config.PeerConnectionConfig) + srcConn, err := connectors.GetAs[connectors.GetTableSchemaConnector](ctx, config.PeerConnectionConfig) if err != nil { - return nil, fmt.Errorf("failed to get connector: %w", err) + return nil, fmt.Errorf("failed to get CDCPullPgConnector: %w", err) } defer connectors.CloseConnector(ctx, srcConn) @@ -175,7 +175,8 @@ func (a *FlowableActivity) CreateNormalizedTable( tableExistsMapping := make(map[string]bool) for tableIdentifier, tableSchema := range config.TableNameSchemaMapping { - existing, err := conn.SetupNormalizedTable( + var existing bool + existing, err = conn.SetupNormalizedTable( ctx, tx, tableIdentifier, @@ -197,8 +198,7 @@ func (a *FlowableActivity) CreateNormalizedTable( } } - err = conn.FinishSetupNormalizedTables(ctx, tx) - if err != nil { + if err := conn.FinishSetupNormalizedTables(ctx, tx); err != nil { return nil, fmt.Errorf("failed to commit normalized tables tx: %w", err) } @@ -264,226 +264,26 @@ func (a *FlowableActivity) UnmaintainPull(ctx context.Context, sessionID string) return nil } -func (a *FlowableActivity) waitForCdcCache(ctx context.Context, sessionID string) (connectors.CDCPullConnector, error) { - logger := activity.GetLogger(ctx) - attempt := 0 - for { - a.CdcCacheRw.RLock() - entry, ok := a.CdcCache[sessionID] - a.CdcCacheRw.RUnlock() - if ok { - return entry.connector, nil - } - activity.RecordHeartbeat(ctx, "wait another second for source connector") - attempt += 1 - if attempt > 2 { - logger.Info("waiting on source connector setup", slog.Int("attempt", attempt)) - } - if err := ctx.Err(); err != nil { - return nil, err - } - time.Sleep(time.Second) - } +func (a *FlowableActivity) SyncRecords( + ctx context.Context, + config *protos.FlowConnectionConfigs, + options *protos.SyncFlowOptions, + sessionID string, +) (*model.SyncResponse, error) { + return syncCore(ctx, a, config, options, sessionID, + connectors.CDCPullConnector.PullRecords, + connectors.CDCSyncConnector.SyncRecords) } -func (a *FlowableActivity) SyncFlow( +func (a *FlowableActivity) SyncPg( ctx context.Context, config *protos.FlowConnectionConfigs, options *protos.SyncFlowOptions, sessionID string, ) (*model.SyncResponse, error) { - flowName := config.FlowJobName - ctx = context.WithValue(ctx, shared.FlowNameKey, flowName) - logger := activity.GetLogger(ctx) - activity.RecordHeartbeat(ctx, "starting flow...") - dstConn, err := connectors.GetCDCSyncConnector(ctx, config.Destination) - if err != nil { - return nil, fmt.Errorf("failed to get destination connector: %w", err) - } - defer connectors.CloseConnector(ctx, dstConn) - - tblNameMapping := make(map[string]model.NameAndExclude, len(options.TableMappings)) - for _, v := range options.TableMappings { - tblNameMapping[v.SourceTableIdentifier] = model.NewNameAndExclude(v.DestinationTableIdentifier, v.Exclude) - } - - var srcConn connectors.CDCPullConnector - if sessionID == "" { - srcConn, err = connectors.GetCDCPullConnector(ctx, config.Source) - if err != nil { - return nil, err - } - defer connectors.CloseConnector(ctx, srcConn) - - if err := srcConn.SetupReplConn(ctx); err != nil { - return nil, err - } - } else { - srcConn, err = a.waitForCdcCache(ctx, sessionID) - if err != nil { - return nil, err - } - if err := srcConn.ConnectionActive(ctx); err != nil { - return nil, temporal.NewNonRetryableApplicationError("connection to source down", "disconnect", nil) - } - } - - shutdown := utils.HeartbeatRoutine(ctx, func() string { - return "transferring records for job" - }) - defer shutdown() - - batchSize := options.BatchSize - if batchSize == 0 { - batchSize = 1_000_000 - } - - lastOffset, err := dstConn.GetLastOffset(ctx, config.FlowJobName) - if err != nil { - return nil, err - } - logger.Info("pulling records...", slog.Int64("LastOffset", lastOffset)) - - // start a goroutine to pull records from the source - recordBatch := model.NewCDCRecordStream() - startTime := time.Now() - - errGroup, errCtx := errgroup.WithContext(ctx) - errGroup.Go(func() error { - return srcConn.PullRecords(errCtx, a.CatalogPool, &model.PullRecordsRequest{ - FlowJobName: flowName, - SrcTableIDNameMapping: options.SrcTableIdNameMapping, - TableNameMapping: tblNameMapping, - LastOffset: lastOffset, - MaxBatchSize: batchSize, - IdleTimeout: peerdbenv.PeerDBCDCIdleTimeoutSeconds( - int(options.IdleTimeoutSeconds), - ), - TableNameSchemaMapping: options.TableNameSchemaMapping, - OverridePublicationName: config.PublicationName, - OverrideReplicationSlotName: config.ReplicationSlotName, - RecordStream: recordBatch, - }) - }) - - hasRecords := !recordBatch.WaitAndCheckEmpty() - logger.Info("current sync flow has records?", slog.Bool("hasRecords", hasRecords)) - - if !hasRecords { - // wait for the pull goroutine to finish - err = errGroup.Wait() - if err != nil { - a.Alerter.LogFlowError(ctx, flowName, err) - if temporal.IsApplicationError(err) { - return nil, err - } else { - return nil, fmt.Errorf("failed in pull records when: %w", err) - } - } - logger.Info("no records to push") - - err := dstConn.ReplayTableSchemaDeltas(ctx, flowName, recordBatch.SchemaDeltas) - if err != nil { - return nil, fmt.Errorf("failed to sync schema: %w", err) - } - - return &model.SyncResponse{ - CurrentSyncBatchID: -1, - TableSchemaDeltas: recordBatch.SchemaDeltas, - }, nil - } - - var syncStartTime time.Time - var res *model.SyncResponse - errGroup.Go(func() error { - syncBatchID, err := dstConn.GetLastSyncBatchID(errCtx, flowName) - if err != nil && config.Destination.Type != protos.DBType_EVENTHUBS { - return err - } - syncBatchID += 1 - - err = monitoring.AddCDCBatchForFlow(errCtx, a.CatalogPool, flowName, - monitoring.CDCBatchInfo{ - BatchID: syncBatchID, - RowsInBatch: 0, - BatchEndlSN: 0, - StartTime: startTime, - }) - if err != nil { - a.Alerter.LogFlowError(ctx, flowName, err) - return err - } - - syncStartTime = time.Now() - res, err = dstConn.SyncRecords(errCtx, &model.SyncRecordsRequest{ - SyncBatchID: syncBatchID, - Records: recordBatch, - FlowJobName: flowName, - TableMappings: options.TableMappings, - StagingPath: config.CdcStagingPath, - Script: config.Script, - }) - if err != nil { - a.Alerter.LogFlowError(ctx, flowName, err) - return fmt.Errorf("failed to push records: %w", err) - } - - return nil - }) - - err = errGroup.Wait() - if err != nil { - a.Alerter.LogFlowError(ctx, flowName, err) - if temporal.IsApplicationError(err) { - return nil, err - } else { - return nil, fmt.Errorf("failed to pull records: %w", err) - } - } - - numRecords := res.NumRecordsSynced - syncDuration := time.Since(syncStartTime) - - logger.Info(fmt.Sprintf("pushed %d records in %d seconds", numRecords, int(syncDuration.Seconds()))) - - lastCheckpoint := recordBatch.GetLastCheckpoint() - srcConn.UpdateReplStateLastOffset(lastCheckpoint) - - err = monitoring.UpdateNumRowsAndEndLSNForCDCBatch( - ctx, - a.CatalogPool, - flowName, - res.CurrentSyncBatchID, - uint32(numRecords), - lastCheckpoint, - ) - if err != nil { - a.Alerter.LogFlowError(ctx, flowName, err) - return nil, err - } - - err = monitoring.UpdateLatestLSNAtTargetForCDCFlow(ctx, a.CatalogPool, flowName, lastCheckpoint) - if err != nil { - a.Alerter.LogFlowError(ctx, flowName, err) - return nil, err - } - if res.TableNameRowsMapping != nil { - err = monitoring.AddCDCBatchTablesForFlow(ctx, a.CatalogPool, flowName, - res.CurrentSyncBatchID, res.TableNameRowsMapping) - if err != nil { - return nil, err - } - } - if err != nil { - a.Alerter.LogFlowError(ctx, flowName, err) - return nil, err - } - - pushedRecordsWithCount := fmt.Sprintf("pushed %d records", numRecords) - activity.RecordHeartbeat(ctx, pushedRecordsWithCount) - a.Alerter.LogFlowInfo(ctx, flowName, pushedRecordsWithCount) - - return res, nil + return syncCore(ctx, a, config, options, sessionID, + connectors.CDCPullPgConnector.PullPg, + connectors.CDCSyncPgConnector.SyncPg) } func (a *FlowableActivity) StartNormalize( @@ -633,104 +433,6 @@ func (a *FlowableActivity) ReplicateQRepPartitions(ctx context.Context, return nil } -// ReplicateQRepPartition replicates a QRepPartition from the source to the destination. -func (a *FlowableActivity) replicateQRepPartition(ctx context.Context, - config *protos.QRepConfig, - idx int, - total int, - partition *protos.QRepPartition, - runUUID string, -) error { - msg := fmt.Sprintf("replicating partition - %s: %d of %d total.", partition.PartitionId, idx, total) - activity.RecordHeartbeat(ctx, msg) - - ctx = context.WithValue(ctx, shared.FlowNameKey, config.FlowJobName) - logger := log.With(activity.GetLogger(ctx), slog.String(string(shared.FlowNameKey), config.FlowJobName)) - - srcConn, err := connectors.GetQRepPullConnector(ctx, config.SourcePeer) - if err != nil { - a.Alerter.LogFlowError(ctx, config.FlowJobName, err) - return fmt.Errorf("failed to get qrep source connector: %w", err) - } - defer connectors.CloseConnector(ctx, srcConn) - - dstConn, err := connectors.GetQRepSyncConnector(ctx, config.DestinationPeer) - if err != nil { - a.Alerter.LogFlowError(ctx, config.FlowJobName, err) - return fmt.Errorf("failed to get qrep destination connector: %w", err) - } - defer connectors.CloseConnector(ctx, dstConn) - - done, err := dstConn.IsQRepPartitionSynced(ctx, &protos.IsQRepPartitionSyncedInput{ - FlowJobName: config.FlowJobName, - PartitionId: partition.PartitionId, - }) - if err != nil { - a.Alerter.LogFlowError(ctx, config.FlowJobName, err) - return fmt.Errorf("failed to get fetch status of partition: %w", err) - } - if done { - logger.Info("no records to push for partition " + partition.PartitionId) - activity.RecordHeartbeat(ctx, "no records to push for partition "+partition.PartitionId) - return nil - } - - err = monitoring.UpdateStartTimeForPartition(ctx, a.CatalogPool, runUUID, partition, time.Now()) - if err != nil { - a.Alerter.LogFlowError(ctx, config.FlowJobName, err) - return fmt.Errorf("failed to update start time for partition: %w", err) - } - - logger.Info("replicating partition " + partition.PartitionId) - shutdown := utils.HeartbeatRoutine(ctx, func() string { - return fmt.Sprintf("syncing partition - %s: %d of %d total.", partition.PartitionId, idx, total) - }) - defer shutdown() - - var rowsSynced int - bufferSize := shared.FetchAndChannelSize - errGroup, errCtx := errgroup.WithContext(ctx) - stream := model.NewQRecordStream(bufferSize) - errGroup.Go(func() error { - tmp, err := srcConn.PullQRepRecords(errCtx, config, partition, stream) - if err != nil { - a.Alerter.LogFlowError(ctx, config.FlowJobName, err) - return fmt.Errorf("failed to pull records: %w", err) - } - numRecords := int64(tmp) - err = monitoring.UpdatePullEndTimeAndRowsForPartition(errCtx, - a.CatalogPool, runUUID, partition, numRecords) - if err != nil { - logger.Error(err.Error()) - } - return nil - }) - - errGroup.Go(func() error { - rowsSynced, err = dstConn.SyncQRepRecords(errCtx, config, partition, stream) - if err != nil { - a.Alerter.LogFlowError(ctx, config.FlowJobName, err) - return fmt.Errorf("failed to sync records: %w", err) - } - return context.Canceled - }) - - if err := errGroup.Wait(); err != nil && err != context.Canceled { - a.Alerter.LogFlowError(ctx, config.FlowJobName, err) - return err - } - - if rowsSynced > 0 { - logger.Info(fmt.Sprintf("pushed %d records", rowsSynced)) - err := monitoring.UpdateRowsSyncedForPartition(ctx, a.CatalogPool, rowsSynced, runUUID, partition) - if err != nil { - return err - } - } - - return monitoring.UpdateEndTimeForPartition(ctx, a.CatalogPool, runUUID, partition) -} - func (a *FlowableActivity) ConsolidateQRepPartitions(ctx context.Context, config *protos.QRepConfig, runUUID string, ) error { @@ -792,36 +494,6 @@ func (a *FlowableActivity) DropFlowDestination(ctx context.Context, config *prot return dstConn.SyncFlowCleanup(ctx, config.FlowJobName) } -func (a *FlowableActivity) getPostgresPeerConfigs(ctx context.Context) ([]*protos.Peer, error) { - optionRows, err := a.CatalogPool.Query(ctx, ` - SELECT DISTINCT p.name, p.options - FROM peers p - JOIN flows f ON p.id = f.source_peer - WHERE p.type = $1`, protos.DBType_POSTGRES) - if err != nil { - return nil, err - } - - return pgx.CollectRows(optionRows, func(row pgx.CollectableRow) (*protos.Peer, error) { - var peerName string - var peerOptions []byte - err := optionRows.Scan(&peerName, &peerOptions) - if err != nil { - return nil, err - } - var pgPeerConfig protos.PostgresConfig - unmarshalErr := proto.Unmarshal(peerOptions, &pgPeerConfig) - if unmarshalErr != nil { - return nil, unmarshalErr - } - return &protos.Peer{ - Name: peerName, - Type: protos.DBType_POSTGRES, - Config: &protos.Peer_PostgresConfig{PostgresConfig: &pgPeerConfig}, - }, nil - }) -} - func (a *FlowableActivity) SendWALHeartbeat(ctx context.Context) error { logger := activity.GetLogger(ctx) if !peerdbenv.PeerDBEnableWALHeartbeat() { diff --git a/flow/activities/flowable_core.go b/flow/activities/flowable_core.go new file mode 100644 index 0000000000..e9687a2240 --- /dev/null +++ b/flow/activities/flowable_core.go @@ -0,0 +1,387 @@ +// internal methods for flowable.go +package activities + +import ( + "context" + "fmt" + "log/slog" + "reflect" + "sync/atomic" + "time" + + "github.com/jackc/pgx/v5" + "github.com/jackc/pgx/v5/pgxpool" + "go.temporal.io/sdk/activity" + "go.temporal.io/sdk/log" + "go.temporal.io/sdk/temporal" + "golang.org/x/sync/errgroup" + "google.golang.org/protobuf/proto" + + "github.com/PeerDB-io/peer-flow/connectors" + "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/peerdbenv" + "github.com/PeerDB-io/peer-flow/shared" +) + +func waitForCdcCache[TPull connectors.CDCPullConnectorCore](ctx context.Context, a *FlowableActivity, sessionID string) (TPull, error) { + var none TPull + logger := activity.GetLogger(ctx) + attempt := 0 + for { + a.CdcCacheRw.RLock() + entry, ok := a.CdcCache[sessionID] + a.CdcCacheRw.RUnlock() + if ok { + if conn, ok := entry.connector.(TPull); ok { + return conn, nil + } + return none, fmt.Errorf("expected %s, cache held %T", reflect.TypeFor[TPull]().Name(), entry.connector) + } + activity.RecordHeartbeat(ctx, "wait another second for source connector") + attempt += 1 + if attempt > 2 { + logger.Info("waiting on source connector setup", slog.Int("attempt", attempt)) + } + if err := ctx.Err(); err != nil { + return none, err + } + time.Sleep(time.Second) + } +} + +func syncCore[TPull connectors.CDCPullConnectorCore, TSync connectors.CDCSyncConnectorCore, Items model.Items]( + ctx context.Context, + a *FlowableActivity, + config *protos.FlowConnectionConfigs, + options *protos.SyncFlowOptions, + sessionID string, + pull func(TPull, context.Context, *pgxpool.Pool, *model.PullRecordsRequest[Items]) error, + sync func(TSync, context.Context, *model.SyncRecordsRequest[Items]) (*model.SyncResponse, error), +) (*model.SyncResponse, error) { + flowName := config.FlowJobName + ctx = context.WithValue(ctx, shared.FlowNameKey, flowName) + logger := activity.GetLogger(ctx) + activity.RecordHeartbeat(ctx, "starting flow...") + dstConn, err := connectors.GetAs[TSync](ctx, config.Destination) + if err != nil { + return nil, fmt.Errorf("failed to get destination connector: %w", err) + } + defer connectors.CloseConnector(ctx, dstConn) + + tblNameMapping := make(map[string]model.NameAndExclude, len(options.TableMappings)) + for _, v := range options.TableMappings { + tblNameMapping[v.SourceTableIdentifier] = model.NewNameAndExclude(v.DestinationTableIdentifier, v.Exclude) + } + + var srcConn TPull + if sessionID == "" { + srcConn, err = connectors.GetAs[TPull](ctx, config.Source) + if err != nil { + return nil, err + } + defer connectors.CloseConnector(ctx, srcConn) + + if err := srcConn.SetupReplConn(ctx); err != nil { + return nil, err + } + } else { + srcConn, err = waitForCdcCache[TPull](ctx, a, sessionID) + if err != nil { + return nil, err + } + if err := srcConn.ConnectionActive(ctx); err != nil { + return nil, temporal.NewNonRetryableApplicationError("connection to source down", "disconnect", nil) + } + } + + shutdown := utils.HeartbeatRoutine(ctx, func() string { + return "transferring records for job" + }) + defer shutdown() + + batchSize := options.BatchSize + if batchSize == 0 { + batchSize = 1_000_000 + } + + lastOffset, err := dstConn.GetLastOffset(ctx, config.FlowJobName) + if err != nil { + return nil, err + } + logger.Info("pulling records...", slog.Int64("LastOffset", lastOffset)) + consumedOffset := atomic.Int64{} + consumedOffset.Store(lastOffset) + + recordBatch := model.NewCDCStream[Items]() + startTime := time.Now() + + errGroup, errCtx := errgroup.WithContext(ctx) + errGroup.Go(func() error { + return pull(srcConn, errCtx, a.CatalogPool, &model.PullRecordsRequest[Items]{ + FlowJobName: flowName, + SrcTableIDNameMapping: options.SrcTableIdNameMapping, + TableNameMapping: tblNameMapping, + LastOffset: lastOffset, + ConsumedOffset: &consumedOffset, + MaxBatchSize: batchSize, + IdleTimeout: peerdbenv.PeerDBCDCIdleTimeoutSeconds( + int(options.IdleTimeoutSeconds), + ), + TableNameSchemaMapping: options.TableNameSchemaMapping, + OverridePublicationName: config.PublicationName, + OverrideReplicationSlotName: config.ReplicationSlotName, + RecordStream: recordBatch, + }) + }) + + hasRecords := !recordBatch.WaitAndCheckEmpty() + logger.Info("current sync flow has records?", slog.Bool("hasRecords", hasRecords)) + + if !hasRecords { + // wait for the pull goroutine to finish + err = errGroup.Wait() + if err != nil { + a.Alerter.LogFlowError(ctx, flowName, err) + if temporal.IsApplicationError(err) { + return nil, err + } else { + return nil, fmt.Errorf("failed in pull records when: %w", err) + } + } + logger.Info("no records to push") + + err := dstConn.ReplayTableSchemaDeltas(ctx, flowName, recordBatch.SchemaDeltas) + if err != nil { + return nil, fmt.Errorf("failed to sync schema: %w", err) + } + + return &model.SyncResponse{ + CurrentSyncBatchID: -1, + TableSchemaDeltas: recordBatch.SchemaDeltas, + }, nil + } + + var syncStartTime time.Time + var res *model.SyncResponse + errGroup.Go(func() error { + syncBatchID, err := dstConn.GetLastSyncBatchID(errCtx, flowName) + if err != nil && config.Destination.Type != protos.DBType_EVENTHUBS { + return err + } + syncBatchID += 1 + + err = monitoring.AddCDCBatchForFlow(errCtx, a.CatalogPool, flowName, + monitoring.CDCBatchInfo{ + BatchID: syncBatchID, + RowsInBatch: 0, + BatchEndlSN: 0, + StartTime: startTime, + }) + if err != nil { + a.Alerter.LogFlowError(ctx, flowName, err) + return err + } + + syncStartTime = time.Now() + res, err = sync(dstConn, errCtx, &model.SyncRecordsRequest[Items]{ + SyncBatchID: syncBatchID, + Records: recordBatch, + ConsumedOffset: &consumedOffset, + FlowJobName: flowName, + TableMappings: options.TableMappings, + StagingPath: config.CdcStagingPath, + Script: config.Script, + }) + if err != nil { + a.Alerter.LogFlowError(ctx, flowName, err) + return fmt.Errorf("failed to push records: %w", err) + } + + return nil + }) + + err = errGroup.Wait() + if err != nil { + a.Alerter.LogFlowError(ctx, flowName, err) + if temporal.IsApplicationError(err) { + return nil, err + } else { + return nil, fmt.Errorf("failed to pull records: %w", err) + } + } + + numRecords := res.NumRecordsSynced + syncDuration := time.Since(syncStartTime) + + logger.Info(fmt.Sprintf("pushed %d records in %d seconds", numRecords, int(syncDuration.Seconds()))) + + lastCheckpoint := recordBatch.GetLastCheckpoint() + srcConn.UpdateReplStateLastOffset(lastCheckpoint) + + err = monitoring.UpdateNumRowsAndEndLSNForCDCBatch( + ctx, + a.CatalogPool, + flowName, + res.CurrentSyncBatchID, + uint32(numRecords), + lastCheckpoint, + ) + if err != nil { + a.Alerter.LogFlowError(ctx, flowName, err) + return nil, err + } + + err = monitoring.UpdateLatestLSNAtTargetForCDCFlow(ctx, a.CatalogPool, flowName, lastCheckpoint) + if err != nil { + a.Alerter.LogFlowError(ctx, flowName, err) + return nil, err + } + if res.TableNameRowsMapping != nil { + err = monitoring.AddCDCBatchTablesForFlow(ctx, a.CatalogPool, flowName, + res.CurrentSyncBatchID, res.TableNameRowsMapping) + if err != nil { + return nil, err + } + } + if err != nil { + a.Alerter.LogFlowError(ctx, flowName, err) + return nil, err + } + + pushedRecordsWithCount := fmt.Sprintf("pushed %d records", numRecords) + activity.RecordHeartbeat(ctx, pushedRecordsWithCount) + a.Alerter.LogFlowInfo(ctx, flowName, pushedRecordsWithCount) + + return res, nil +} + +func (a *FlowableActivity) getPostgresPeerConfigs(ctx context.Context) ([]*protos.Peer, error) { + optionRows, err := a.CatalogPool.Query(ctx, ` + SELECT DISTINCT p.name, p.options + FROM peers p + JOIN flows f ON p.id = f.source_peer + WHERE p.type = $1`, protos.DBType_POSTGRES) + if err != nil { + return nil, err + } + + return pgx.CollectRows(optionRows, func(row pgx.CollectableRow) (*protos.Peer, error) { + var peerName string + var peerOptions []byte + err := optionRows.Scan(&peerName, &peerOptions) + if err != nil { + return nil, err + } + var pgPeerConfig protos.PostgresConfig + unmarshalErr := proto.Unmarshal(peerOptions, &pgPeerConfig) + if unmarshalErr != nil { + return nil, unmarshalErr + } + return &protos.Peer{ + Name: peerName, + Type: protos.DBType_POSTGRES, + Config: &protos.Peer_PostgresConfig{PostgresConfig: &pgPeerConfig}, + }, nil + }) +} + +// replicateQRepPartition replicates a QRepPartition from the source to the destination. +func (a *FlowableActivity) replicateQRepPartition(ctx context.Context, + config *protos.QRepConfig, + idx int, + total int, + partition *protos.QRepPartition, + runUUID string, +) error { + msg := fmt.Sprintf("replicating partition - %s: %d of %d total.", partition.PartitionId, idx, total) + activity.RecordHeartbeat(ctx, msg) + + ctx = context.WithValue(ctx, shared.FlowNameKey, config.FlowJobName) + logger := log.With(activity.GetLogger(ctx), slog.String(string(shared.FlowNameKey), config.FlowJobName)) + + srcConn, err := connectors.GetQRepPullConnector(ctx, config.SourcePeer) + if err != nil { + a.Alerter.LogFlowError(ctx, config.FlowJobName, err) + return fmt.Errorf("failed to get qrep source connector: %w", err) + } + defer connectors.CloseConnector(ctx, srcConn) + + dstConn, err := connectors.GetQRepSyncConnector(ctx, config.DestinationPeer) + if err != nil { + a.Alerter.LogFlowError(ctx, config.FlowJobName, err) + return fmt.Errorf("failed to get qrep destination connector: %w", err) + } + defer connectors.CloseConnector(ctx, dstConn) + + done, err := dstConn.IsQRepPartitionSynced(ctx, &protos.IsQRepPartitionSyncedInput{ + FlowJobName: config.FlowJobName, + PartitionId: partition.PartitionId, + }) + if err != nil { + a.Alerter.LogFlowError(ctx, config.FlowJobName, err) + return fmt.Errorf("failed to get fetch status of partition: %w", err) + } + if done { + logger.Info("no records to push for partition " + partition.PartitionId) + activity.RecordHeartbeat(ctx, "no records to push for partition "+partition.PartitionId) + return nil + } + + err = monitoring.UpdateStartTimeForPartition(ctx, a.CatalogPool, runUUID, partition, time.Now()) + if err != nil { + a.Alerter.LogFlowError(ctx, config.FlowJobName, err) + return fmt.Errorf("failed to update start time for partition: %w", err) + } + + logger.Info("replicating partition " + partition.PartitionId) + shutdown := utils.HeartbeatRoutine(ctx, func() string { + return fmt.Sprintf("syncing partition - %s: %d of %d total.", partition.PartitionId, idx, total) + }) + defer shutdown() + + var rowsSynced int + bufferSize := shared.FetchAndChannelSize + errGroup, errCtx := errgroup.WithContext(ctx) + stream := model.NewQRecordStream(bufferSize) + errGroup.Go(func() error { + tmp, err := srcConn.PullQRepRecords(errCtx, config, partition, stream) + if err != nil { + a.Alerter.LogFlowError(ctx, config.FlowJobName, err) + return fmt.Errorf("failed to pull records: %w", err) + } + numRecords := int64(tmp) + err = monitoring.UpdatePullEndTimeAndRowsForPartition(errCtx, + a.CatalogPool, runUUID, partition, numRecords) + if err != nil { + logger.Error(err.Error()) + } + return nil + }) + + errGroup.Go(func() error { + rowsSynced, err = dstConn.SyncQRepRecords(errCtx, config, partition, stream) + if err != nil { + a.Alerter.LogFlowError(ctx, config.FlowJobName, err) + return fmt.Errorf("failed to sync records: %w", err) + } + return context.Canceled + }) + + if err := errGroup.Wait(); err != nil && err != context.Canceled { + a.Alerter.LogFlowError(ctx, config.FlowJobName, err) + return err + } + + if rowsSynced > 0 { + logger.Info(fmt.Sprintf("pushed %d records", rowsSynced)) + err := monitoring.UpdateRowsSyncedForPartition(ctx, a.CatalogPool, rowsSynced, runUUID, partition) + if err != nil { + return err + } + } + + return monitoring.UpdateEndTimeForPartition(ctx, a.CatalogPool, runUUID, partition) +} diff --git a/flow/alerting/alerting.go b/flow/alerting/alerting.go index 3d5f7bcbaf..ccfcdd571f 100644 --- a/flow/alerting/alerting.go +++ b/flow/alerting/alerting.go @@ -256,7 +256,7 @@ func (a *Alerter) sendTelemetryMessage(ctx context.Context, flowName string, mor _, err := a.telemetrySender.SendMessage(ctx, details, details, telemetry.Attributes{ Level: level, DeploymentUID: peerdbenv.PeerDBDeploymentUID(), - Tags: []string{flowName}, + Tags: []string{flowName, peerdbenv.PeerDBDeploymentUID()}, Type: flowName, }) if err != nil { diff --git a/flow/cmd/handler.go b/flow/cmd/handler.go index fb001f1f0f..3adb3b988d 100644 --- a/flow/cmd/handler.go +++ b/flow/cmd/handler.go @@ -319,56 +319,62 @@ func (h *FlowRequestHandler) ShutdownFlow( }, fmt.Errorf("unable to wait for PeerFlow workflow to close: %w", err) } - workflowID := fmt.Sprintf("%s-dropflow-%s", req.FlowJobName, uuid.New()) - workflowOptions := client.StartWorkflowOptions{ - ID: workflowID, - TaskQueue: h.peerflowTaskQueueID, - SearchAttributes: map[string]interface{}{ - shared.MirrorNameSearchAttribute: req.FlowJobName, - }, - } - dropFlowHandle, err := h.temporalClient.ExecuteWorkflow(ctx, workflowOptions, peerflow.DropFlowWorkflow, req) + isCdc, err := h.isCDCFlow(ctx, req.FlowJobName) if err != nil { - slog.Error("unable to start DropFlow workflow", - logs, - slog.Any("error", err)) + slog.Error("unable to check if workflow is cdc", logs, slog.Any("error", err)) return &protos.ShutdownResponse{ Ok: false, - ErrorMessage: fmt.Sprintf("unable to start DropFlow workflow: %v", err), - }, fmt.Errorf("unable to start DropFlow workflow: %w", err) - } - - cancelCtx, cancel := context.WithTimeout(ctx, 2*time.Minute) - defer cancel() - - errChan := make(chan error, 1) - go func() { - errChan <- dropFlowHandle.Get(cancelCtx, nil) - }() - - select { - case err := <-errChan: - if err != nil { - slog.Error("DropFlow workflow did not execute successfully", - logs, - slog.Any("error", err), - ) - return &protos.ShutdownResponse{ - Ok: false, - ErrorMessage: fmt.Sprintf("DropFlow workflow did not execute successfully: %v", err), - }, fmt.Errorf("DropFlow workflow did not execute successfully: %w", err) + ErrorMessage: fmt.Sprintf("unable to determine if workflow is cdc: %v", err), + }, fmt.Errorf("unable to determine if workflow is cdc: %w", err) + } else if isCdc { + workflowID := fmt.Sprintf("%s-dropflow-%s", req.FlowJobName, uuid.New()) + workflowOptions := client.StartWorkflowOptions{ + ID: workflowID, + TaskQueue: h.peerflowTaskQueueID, + SearchAttributes: map[string]interface{}{ + shared.MirrorNameSearchAttribute: req.FlowJobName, + }, } - case <-time.After(5 * time.Minute): - err := h.handleCancelWorkflow(ctx, workflowID, "") + dropFlowHandle, err := h.temporalClient.ExecuteWorkflow(ctx, workflowOptions, peerflow.DropFlowWorkflow, req) if err != nil { - slog.Error("unable to wait for DropFlow workflow to close", + slog.Error("unable to start DropFlow workflow", logs, - slog.Any("error", err), - ) + slog.Any("error", err)) return &protos.ShutdownResponse{ Ok: false, - ErrorMessage: fmt.Sprintf("unable to wait for DropFlow workflow to close: %v", err), - }, fmt.Errorf("unable to wait for DropFlow workflow to close: %w", err) + ErrorMessage: fmt.Sprintf("unable to start DropFlow workflow: %v", err), + }, fmt.Errorf("unable to start DropFlow workflow: %w", err) + } + + cancelCtx, cancel := context.WithTimeout(ctx, 2*time.Minute) + defer cancel() + + errChan := make(chan error, 1) + go func() { + errChan <- dropFlowHandle.Get(cancelCtx, nil) + }() + + select { + case err := <-errChan: + if err != nil { + slog.Error("DropFlow workflow did not execute successfully", + logs, + slog.Any("error", err), + ) + return &protos.ShutdownResponse{ + Ok: false, + ErrorMessage: fmt.Sprintf("DropFlow workflow did not execute successfully: %v", err), + }, fmt.Errorf("DropFlow workflow did not execute successfully: %w", err) + } + case <-time.After(5 * time.Minute): + err := h.handleCancelWorkflow(ctx, workflowID, "") + if err != nil { + slog.Error("unable to wait for DropFlow workflow to close", logs, slog.Any("error", err)) + return &protos.ShutdownResponse{ + Ok: false, + ErrorMessage: fmt.Sprintf("unable to wait for DropFlow workflow to close: %v", err), + }, fmt.Errorf("unable to wait for DropFlow workflow to close: %w", err) + } } } @@ -590,6 +596,13 @@ func (h *FlowRequestHandler) CreatePeer( } ehConfig := ehConfigObject.EventhubGroupConfig encodedConfig, encodingErr = proto.Marshal(ehConfig) + case protos.DBType_ELASTICSEARCH: + esConfigObject, ok := config.(*protos.Peer_ElasticsearchConfig) + if !ok { + return wrongConfigResponse, nil + } + esConfig := esConfigObject.ElasticsearchConfig + encodedConfig, encodingErr = proto.Marshal(esConfig) default: return wrongConfigResponse, nil } diff --git a/flow/cmd/mirror_status.go b/flow/cmd/mirror_status.go index 172a5e9833..06f64d34a9 100644 --- a/flow/cmd/mirror_status.go +++ b/flow/cmd/mirror_status.go @@ -347,18 +347,14 @@ func (h *FlowRequestHandler) getQRepConfigFromCatalog(ctx context.Context, flowJ } func (h *FlowRequestHandler) isCDCFlow(ctx context.Context, flowJobName string) (bool, error) { - var query pgtype.Text - err := h.pool.QueryRow(ctx, "SELECT query_string FROM flows WHERE name = $1", flowJobName).Scan(&query) + var isCdc bool + err := h.pool.QueryRow(ctx, "SELECT exists(SELECT * FROM flows WHERE name=$1 and coalesce(query_string, '')='')", + flowJobName).Scan(&isCdc) if err != nil { slog.Error("unable to query flow", slog.Any("error", err)) return false, fmt.Errorf("unable to query flow: %w", err) } - - if !query.Valid || query.String == "" { - return true, nil - } - - return false, nil + return isCdc, nil } func (h *FlowRequestHandler) getWorkflowStatus(ctx context.Context, workflowID string) (protos.FlowStatus, error) { diff --git a/flow/connectors/bigquery/bigquery.go b/flow/connectors/bigquery/bigquery.go index 81fb8266a6..7c9d66ab93 100644 --- a/flow/connectors/bigquery/bigquery.go +++ b/flow/connectors/bigquery/bigquery.go @@ -6,7 +6,6 @@ import ( "fmt" "log/slog" "reflect" - "regexp" "strings" "time" @@ -218,19 +217,19 @@ func (c *BigQueryConnector) ReplayTableSchemaDeltas( for _, addedColumn := range schemaDelta.AddedColumns { dstDatasetTable, _ := c.convertToDatasetTable(schemaDelta.DstTableName) - addedColumnBigQueryType := qValueKindToBigQueryTypeString(addedColumn.ColumnType) + addedColumnBigQueryType := qValueKindToBigQueryTypeString(addedColumn.Type) query := c.client.Query(fmt.Sprintf( "ALTER TABLE %s ADD COLUMN IF NOT EXISTS `%s` %s", - dstDatasetTable.table, addedColumn.ColumnName, addedColumnBigQueryType)) + dstDatasetTable.table, addedColumn.Name, addedColumnBigQueryType)) query.DefaultProjectID = c.projectID query.DefaultDatasetID = dstDatasetTable.dataset _, err := query.Read(ctx) if err != nil { - return fmt.Errorf("failed to add column %s for table %s: %w", addedColumn.ColumnName, + return fmt.Errorf("failed to add column %s for table %s: %w", addedColumn.Name, schemaDelta.DstTableName, err) } c.logger.Info(fmt.Sprintf("[schema delta replay] added column %s with data type %s to table %s", - addedColumn.ColumnName, addedColumn.ColumnType, schemaDelta.DstTableName)) + addedColumn.Name, addedColumn.Type, schemaDelta.DstTableName)) } } @@ -327,7 +326,7 @@ func (c *BigQueryConnector) getTableNametoUnchangedCols( // SyncRecords pushes records to the destination. // Currently only supports inserts, updates, and deletes. // More record types will be added in the future. -func (c *BigQueryConnector) SyncRecords(ctx context.Context, req *model.SyncRecordsRequest) (*model.SyncResponse, error) { +func (c *BigQueryConnector) SyncRecords(ctx context.Context, req *model.SyncRecordsRequest[model.RecordItems]) (*model.SyncResponse, error) { rawTableName := c.getRawTableName(req.FlowJobName) c.logger.Info(fmt.Sprintf("pushing records to %s.%s...", c.datasetID, rawTableName)) @@ -343,7 +342,7 @@ func (c *BigQueryConnector) SyncRecords(ctx context.Context, req *model.SyncReco func (c *BigQueryConnector) syncRecordsViaAvro( ctx context.Context, - req *model.SyncRecordsRequest, + req *model.SyncRecordsRequest[model.RecordItems], rawTableName string, syncBatchID int64, ) (*model.SyncResponse, error) { @@ -713,9 +712,7 @@ func (c *BigQueryConnector) SyncFlowCleanup(ctx context.Context, jobName string) // getRawTableName returns the raw table name for the given table identifier. func (c *BigQueryConnector) getRawTableName(flowJobName string) string { - // replace all non-alphanumeric characters with _ - flowJobName = regexp.MustCompile("[^a-zA-Z0-9_]+").ReplaceAllString(flowJobName, "_") - return "_peerdb_raw_" + flowJobName + return "_peerdb_raw_" + shared.ReplaceIllegalCharactersWithUnderscores(flowJobName) } func (c *BigQueryConnector) RenameTables(ctx context.Context, req *protos.RenameTablesInput) (*protos.RenameTablesOutput, error) { diff --git a/flow/connectors/bigquery/qrep.go b/flow/connectors/bigquery/qrep.go index 1f11b97d11..58d8eac28f 100644 --- a/flow/connectors/bigquery/qrep.go +++ b/flow/connectors/bigquery/qrep.go @@ -8,6 +8,7 @@ import ( "cloud.google.com/go/bigquery" + "github.com/PeerDB-io/peer-flow/datatypes" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" @@ -22,10 +23,8 @@ func (c *BigQueryConnector) SyncQRepRecords( ) (int, error) { // Ensure the destination table is available. destTable := config.DestinationTableIdentifier - srcSchema, err := stream.Schema() - if err != nil { - return 0, fmt.Errorf("failed to get schema of source table %s: %w", config.WatermarkTable, err) - } + srcSchema := stream.Schema() + tblMetadata, err := c.replayTableSchemaDeltasQRep(ctx, config, partition, srcSchema) if err != nil { return 0, err @@ -44,7 +43,7 @@ func (c *BigQueryConnector) replayTableSchemaDeltasQRep( ctx context.Context, config *protos.QRepConfig, partition *protos.QRepPartition, - srcSchema *qvalue.QRecordSchema, + srcSchema qvalue.QRecordSchema, ) (*bigquery.TableMetadata, error) { destDatasetTable, _ := c.convertToDatasetTable(config.DestinationTableIdentifier) bqTable := c.client.DatasetInProject(c.projectID, destDatasetTable.dataset).Table(destDatasetTable.table) @@ -72,10 +71,12 @@ func (c *BigQueryConnector) replayTableSchemaDeltasQRep( c.logger.Info(fmt.Sprintf("adding column %s to destination table %s", col.Name, config.DestinationTableIdentifier), slog.String(string(shared.PartitionIDKey), partition.PartitionId)) - tableSchemaDelta.AddedColumns = append(tableSchemaDelta.AddedColumns, &protos.DeltaAddedColumn{ - ColumnName: col.Name, - ColumnType: string(col.Type), - }) + tableSchemaDelta.AddedColumns = append(tableSchemaDelta.AddedColumns, &protos.FieldDescription{ + Name: col.Name, + Type: string(col.Type), + TypeModifier: datatypes.MakeNumericTypmod(int32(col.Precision), int32(col.Scale)), + }, + ) } } diff --git a/flow/connectors/bigquery/qrep_avro_sync.go b/flow/connectors/bigquery/qrep_avro_sync.go index 5048777f71..d022045b70 100644 --- a/flow/connectors/bigquery/qrep_avro_sync.go +++ b/flow/connectors/bigquery/qrep_avro_sync.go @@ -39,7 +39,7 @@ func NewQRepAvroSyncMethod(connector *BigQueryConnector, gcsBucket string, func (s *QRepAvroSyncMethod) SyncRecords( ctx context.Context, - req *model.SyncRecordsRequest, + req *model.SyncRecordsRequest[model.RecordItems], rawTableName string, dstTableMetadata *bigquery.TableMetadata, syncBatchID int64, @@ -196,18 +196,23 @@ func (s *QRepAvroSyncMethod) SyncQRepRecords( transformedColumns := getTransformedColumns(&dstTableMetadata.Schema, syncedAtCol, softDeleteCol) selector := strings.Join(transformedColumns, ", ") - // Insert the records from the staging table into the destination table - insertStmt := fmt.Sprintf("INSERT INTO `%s`(%s) SELECT %s FROM `%s`;", - dstTableName, insertColumnSQL, selector, stagingDatasetTable.string()) + // The staging table may not exist if there are no rows (not created by the bq loader) + if numRecords > 0 { + // Insert the records from the staging table into the destination table + insertStmt := fmt.Sprintf("INSERT INTO `%s`(%s) SELECT %s FROM `%s`;", + dstTableName, insertColumnSQL, selector, stagingDatasetTable.string()) - s.connector.logger.Info("Performing transaction inside QRep sync function", flowLog) + s.connector.logger.Info("Performing transaction inside QRep sync function", flowLog) - query := bqClient.Query(insertStmt) - query.DefaultDatasetID = s.connector.datasetID - query.DefaultProjectID = s.connector.projectID - _, err = query.Read(ctx) - if err != nil { - return -1, fmt.Errorf("failed to execute statements in a transaction: %w", err) + query := bqClient.Query(insertStmt) + query.DefaultDatasetID = s.connector.datasetID + query.DefaultProjectID = s.connector.projectID + _, err = query.Read(ctx) + if err != nil { + return -1, fmt.Errorf("SyncQRepRecords: failed to execute statements in a transaction: %w", err) + } + } else { + s.connector.logger.Info("SyncQRepRecords: no rows to sync, hence skipping transaction", flowLog) } err = s.connector.FinishQRepPartition(ctx, partition, flowJobName, startTime) diff --git a/flow/connectors/clickhouse/cdc.go b/flow/connectors/clickhouse/cdc.go index b53fe6611c..735e0f1cd0 100644 --- a/flow/connectors/clickhouse/cdc.go +++ b/flow/connectors/clickhouse/cdc.go @@ -6,7 +6,6 @@ import ( "errors" "fmt" "log/slog" - "regexp" "strings" _ "github.com/ClickHouse/clickhouse-go/v2" @@ -16,6 +15,7 @@ import ( "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" ) const ( @@ -25,9 +25,7 @@ const ( // getRawTableName returns the raw table name for the given table identifier. func (c *ClickhouseConnector) getRawTableName(flowJobName string) string { - // replace all non-alphanumeric characters with _ - flowJobName = regexp.MustCompile("[^a-zA-Z0-9_]+").ReplaceAllString(flowJobName, "_") - return "_peerdb_raw_" + flowJobName + return "_peerdb_raw_" + shared.ReplaceIllegalCharactersWithUnderscores(flowJobName) } func (c *ClickhouseConnector) checkIfTableExists(ctx context.Context, databaseName string, tableIdentifier string) (bool, error) { @@ -70,7 +68,7 @@ func (c *ClickhouseConnector) CreateRawTable(ctx context.Context, req *protos.Cr func (c *ClickhouseConnector) syncRecordsViaAvro( ctx context.Context, - req *model.SyncRecordsRequest, + req *model.SyncRecordsRequest[model.RecordItems], rawTableIdentifier string, syncBatchID int64, ) (*model.SyncResponse, error) { @@ -111,7 +109,7 @@ func (c *ClickhouseConnector) syncRecordsViaAvro( }, nil } -func (c *ClickhouseConnector) SyncRecords(ctx context.Context, req *model.SyncRecordsRequest) (*model.SyncResponse, error) { +func (c *ClickhouseConnector) SyncRecords(ctx context.Context, req *model.SyncRecordsRequest[model.RecordItems]) (*model.SyncResponse, error) { rawTableName := c.getRawTableName(req.FlowJobName) c.logger.Info("pushing records to Clickhouse table " + rawTableName) @@ -154,20 +152,20 @@ func (c *ClickhouseConnector) ReplayTableSchemaDeltas(ctx context.Context, flowJ } for _, addedColumn := range schemaDelta.AddedColumns { - clickhouseColType, err := qvalue.QValueKind(addedColumn.ColumnType).ToDWHColumnType(protos.DBType_CLICKHOUSE) + clickhouseColType, err := qvalue.QValueKind(addedColumn.Type).ToDWHColumnType(protos.DBType_CLICKHOUSE) if err != nil { return fmt.Errorf("failed to convert column type %s to clickhouse type: %w", - addedColumn.ColumnType, err) + addedColumn.Type, err) } _, err = tableSchemaModifyTx.ExecContext(ctx, fmt.Sprintf("ALTER TABLE %s ADD COLUMN IF NOT EXISTS \"%s\" %s", - schemaDelta.DstTableName, addedColumn.ColumnName, clickhouseColType)) + schemaDelta.DstTableName, addedColumn.Name, clickhouseColType)) if err != nil { - return fmt.Errorf("failed to add column %s for table %s: %w", addedColumn.ColumnName, + return fmt.Errorf("failed to add column %s for table %s: %w", addedColumn.Name, schemaDelta.DstTableName, err) } - c.logger.Info(fmt.Sprintf("[schema delta replay] added column %s with data type %s", addedColumn.ColumnName, - addedColumn.ColumnType), + c.logger.Info(fmt.Sprintf("[schema delta replay] added column %s with data type %s", addedColumn.Name, + addedColumn.Type), "destination table name", schemaDelta.DstTableName, "source table name", schemaDelta.SrcTableName) } diff --git a/flow/connectors/clickhouse/qrep_avro_sync.go b/flow/connectors/clickhouse/qrep_avro_sync.go index 78b076c9f2..e9602468d8 100644 --- a/flow/connectors/clickhouse/qrep_avro_sync.go +++ b/flow/connectors/clickhouse/qrep_avro_sync.go @@ -66,11 +66,7 @@ func (s *ClickhouseAvroSyncMethod) SyncRecords( tableLog := slog.String("destinationTable", s.config.DestinationTableIdentifier) dstTableName := s.config.DestinationTableIdentifier - schema, err := stream.Schema() - if err != nil { - return -1, fmt.Errorf("failed to get schema from stream: %w", err) - } - + schema := stream.Schema() s.connector.logger.Info("sync function called and schema acquired", tableLog) avroSchema, err := s.getAvroSchema(dstTableName, schema) @@ -103,11 +99,8 @@ func (s *ClickhouseAvroSyncMethod) SyncQRepRecords( startTime := time.Now() dstTableName := config.DestinationTableIdentifier stagingPath := s.connector.credsProvider.BucketPath - schema, err := stream.Schema() - if err != nil { - return -1, fmt.Errorf("failed to get schema from stream: %w", err) - } - avroSchema, err := s.getAvroSchema(dstTableName, schema) + + avroSchema, err := s.getAvroSchema(dstTableName, stream.Schema()) if err != nil { return 0, err } @@ -164,7 +157,7 @@ func (s *ClickhouseAvroSyncMethod) SyncQRepRecords( func (s *ClickhouseAvroSyncMethod) getAvroSchema( dstTableName string, - schema *qvalue.QRecordSchema, + schema qvalue.QRecordSchema, ) (*model.QRecordAvroSchemaDefinition, error) { avroSchema, err := model.GetAvroSchemaDefinition(dstTableName, schema, protos.DBType_CLICKHOUSE) if err != nil { diff --git a/flow/connectors/connelasticsearch/elasticsearch.go b/flow/connectors/connelasticsearch/elasticsearch.go new file mode 100644 index 0000000000..01fa27d7e7 --- /dev/null +++ b/flow/connectors/connelasticsearch/elasticsearch.go @@ -0,0 +1,210 @@ +package connelasticsearch + +import ( + "bytes" + "context" + "crypto/tls" + "encoding/json" + "fmt" + "log/slog" + "net/http" + "sync" + "time" + + "github.com/elastic/go-elasticsearch/v8" + "github.com/elastic/go-elasticsearch/v8/esutil" + "github.com/google/uuid" + "go.temporal.io/sdk/log" + + metadataStore "github.com/PeerDB-io/peer-flow/connectors/external_metadata" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/logger" + "github.com/PeerDB-io/peer-flow/model" + "github.com/PeerDB-io/peer-flow/model/qvalue" + "github.com/PeerDB-io/peer-flow/shared" +) + +type ElasticsearchConnector struct { + *metadataStore.PostgresMetadata + client *elasticsearch.Client + logger log.Logger +} + +func NewElasticsearchConnector(ctx context.Context, + config *protos.ElasticsearchConfig, +) (*ElasticsearchConnector, error) { + esCfg := &elasticsearch.Config{ + Addresses: config.Addresses, + Transport: &http.Transport{ + MaxIdleConnsPerHost: 4, + TLSClientConfig: &tls.Config{ + MinVersion: tls.VersionTLS13, + }, + }, + } + if config.AuthType == protos.ElasticsearchAuthType_BASIC { + esCfg.Username = *config.Username + esCfg.Password = *config.Password + } else if config.AuthType == protos.ElasticsearchAuthType_APIKEY { + esCfg.APIKey = *config.ApiKey + } + + esClient, err := elasticsearch.NewClient(*esCfg) + if err != nil { + return nil, fmt.Errorf("error creating elasticsearch connector: %w", err) + } + pgMetadata, err := metadataStore.NewPostgresMetadata(ctx) + if err != nil { + return nil, err + } + + return &ElasticsearchConnector{ + PostgresMetadata: pgMetadata, + client: esClient, + logger: logger.LoggerFromCtx(ctx), + }, nil +} + +func (esc *ElasticsearchConnector) ConnectionActive(ctx context.Context) error { + err := esc.client.DiscoverNodes() + if err != nil { + return fmt.Errorf("failed to check if elasticsearch peer is active: %w", err) + } + return nil +} + +func (esc *ElasticsearchConnector) Close() error { + // stateless connector + return nil +} + +func (esc *ElasticsearchConnector) SetupQRepMetadataTables(ctx context.Context, + config *protos.QRepConfig, +) error { + return nil +} + +func (esc *ElasticsearchConnector) SyncQRepRecords(ctx context.Context, config *protos.QRepConfig, + partition *protos.QRepPartition, stream *model.QRecordStream, +) (int, error) { + startTime := time.Now() + + schema := stream.Schema() + + var bulkIndexErrors []error + var bulkIndexMutex sync.Mutex + var docId string + numRecords := 0 + bulkIndexerHasShutdown := false + + // -1 means use UUID, >=0 means column in the record + upsertColIndex := -1 + // only support single upsert column for now + if config.WriteMode.WriteType == protos.QRepWriteType_QREP_WRITE_MODE_UPSERT && + len(config.WriteMode.UpsertKeyColumns) == 1 { + for i, field := range schema.Fields { + if config.WriteMode.UpsertKeyColumns[0] == field.Name { + upsertColIndex = i + } + } + } + + esBulkIndexer, err := esutil.NewBulkIndexer(esutil.BulkIndexerConfig{ + Index: config.DestinationTableIdentifier, + Client: esc.client, + // parallelism comes from the workflow design itself, no need for this + NumWorkers: 1, + FlushInterval: 10 * time.Second, + }) + if err != nil { + esc.logger.Error("[es] failed to initialize bulk indexer", slog.Any("error", err)) + return 0, fmt.Errorf("[es] failed to initialize bulk indexer: %w", err) + } + defer func() { + if !bulkIndexerHasShutdown { + err := esBulkIndexer.Close(context.Background()) + if err != nil { + esc.logger.Error("[es] failed to close bulk indexer", slog.Any("error", err)) + } + } + }() + + for qRecord := range stream.Records { + qRecordJsonMap := make(map[string]any) + + if upsertColIndex >= 0 { + docId = fmt.Sprintf("%v", qRecord[upsertColIndex].Value()) + } else { + docId = uuid.New().String() + } + for i, field := range schema.Fields { + switch r := qRecord[i].(type) { + // JSON is stored as a string, fix that + case qvalue.QValueJSON: + qRecordJsonMap[field.Name] = json.RawMessage(shared. + UnsafeFastStringToReadOnlyBytes(r.Val)) + default: + qRecordJsonMap[field.Name] = r.Value() + } + } + qRecordJsonBytes, err := json.Marshal(qRecordJsonMap) + if err != nil { + esc.logger.Error("[es] failed to json.Marshal record", slog.Any("error", err)) + return 0, fmt.Errorf("[es] failed to json.Marshal record: %w", err) + } + + err = esBulkIndexer.Add(ctx, esutil.BulkIndexerItem{ + Action: "index", + DocumentID: docId, + Body: bytes.NewReader(qRecordJsonBytes), + + // OnFailure is called for each failed operation, log and let parent handle + OnFailure: func(ctx context.Context, item esutil.BulkIndexerItem, + res esutil.BulkIndexerResponseItem, err error, + ) { + bulkIndexMutex.Lock() + defer bulkIndexMutex.Unlock() + if err != nil { + bulkIndexErrors = append(bulkIndexErrors, err) + } else { + causeString := "" + if res.Error.Cause.Type != "" || res.Error.Cause.Reason != "" { + causeString = fmt.Sprintf("(caused by type:%s reason:%s)", res.Error.Cause.Type, res.Error.Cause.Reason) + } + bulkIndexErrors = append(bulkIndexErrors, + fmt.Errorf("id:%s type:%s reason:%s %s", item.DocumentID, res.Error.Type, + res.Error.Reason, causeString)) + } + }, + }) + if err != nil { + esc.logger.Error("[es] failed to add record to bulk indexer", slog.Any("error", err)) + return 0, fmt.Errorf("[es] failed to add record to bulk indexer: %w", err) + } + + // update here instead of OnSuccess, if we close successfully it should match + numRecords++ + } + + if err := stream.Err(); err != nil { + esc.logger.Error("[es] failed to get record from stream", slog.Any("error", err)) + return 0, fmt.Errorf("[es] failed to get record from stream: %w", err) + } + if err := esBulkIndexer.Close(ctx); err != nil { + esc.logger.Error("[es] failed to close bulk indexer", slog.Any("error", err)) + return 0, fmt.Errorf("[es] failed to close bulk indexer: %w", err) + } + bulkIndexerHasShutdown = true + if len(bulkIndexErrors) > 0 { + for _, err := range bulkIndexErrors { + esc.logger.Error("[es] failed to index record", slog.Any("err", err)) + } + } + + err = esc.FinishQRepPartition(ctx, partition, config.FlowJobName, startTime) + if err != nil { + esc.logger.Error("[es] failed to log partition info", slog.Any("error", err)) + return 0, fmt.Errorf("[es] failed to log partition info: %w", err) + } + return numRecords, nil +} diff --git a/flow/connectors/core.go b/flow/connectors/core.go index 12d87d0c4e..1e3321fa72 100644 --- a/flow/connectors/core.go +++ b/flow/connectors/core.go @@ -10,6 +10,7 @@ import ( "github.com/PeerDB-io/peer-flow/alerting" connbigquery "github.com/PeerDB-io/peer-flow/connectors/bigquery" connclickhouse "github.com/PeerDB-io/peer-flow/connectors/clickhouse" + connelasticsearch "github.com/PeerDB-io/peer-flow/connectors/connelasticsearch" conneventhub "github.com/PeerDB-io/peer-flow/connectors/eventhub" connkafka "github.com/PeerDB-io/peer-flow/connectors/kafka" connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" @@ -40,12 +41,11 @@ type ValidationConnector interface { type GetTableSchemaConnector interface { Connector - // GetTableSchema returns the schema of a table. + // GetTableSchema returns the schema of a table in terms of QValueKind. GetTableSchema(ctx context.Context, req *protos.GetTableSchemaBatchInput) (*protos.GetTableSchemaBatchOutput, error) } -type CDCPullConnector interface { - Connector +type CDCPullConnectorCore interface { GetTableSchemaConnector // EnsurePullability ensures that the connector is pullable. @@ -62,13 +62,9 @@ type CDCPullConnector interface { // Methods related to retrieving and pushing records for this connector as a source and destination. SetupReplConn(context.Context) error - // Ping source to keep connection alive. Can be called concurrently with PullRecords; skips ping in that case. + // Ping source to keep connection alive. Can be called concurrently with pulling records; skips ping in that case. ReplPing(context.Context) error - // PullRecords pulls records from the source, and returns a RecordBatch. - // This method should be idempotent, and should be able to be called multiple times with the same request. - PullRecords(ctx context.Context, catalogPool *pgxpool.Pool, req *model.PullRecordsRequest) error - // Called when offset has been confirmed to destination UpdateReplStateLastOffset(lastOffset int64) @@ -85,12 +81,34 @@ type CDCPullConnector interface { AddTablesToPublication(ctx context.Context, req *protos.AddTablesToPublicationInput) error } +type CDCPullConnector interface { + CDCPullConnectorCore + + // This method should be idempotent, and should be able to be called multiple times with the same request. + PullRecords(ctx context.Context, catalogPool *pgxpool.Pool, req *model.PullRecordsRequest[model.RecordItems]) error +} + +type CDCPullPgConnector interface { + CDCPullConnectorCore + + // This method should be idempotent, and should be able to be called multiple times with the same request. + // It's signature, aside from type parameter, should match CDCPullConnector.PullRecords. + PullPg(ctx context.Context, catalogPool *pgxpool.Pool, req *model.PullRecordsRequest[model.PgItems]) error +} + type NormalizedTablesConnector interface { Connector // StartSetupNormalizedTables may be used to have SetupNormalizedTable calls run in a transaction. StartSetupNormalizedTables(ctx context.Context) (any, error) + // CleanupSetupNormalizedTables may be used to rollback transaction started by StartSetupNormalizedTables. + // Calling CleanupSetupNormalizedTables after FinishSetupNormalizedTables must be a nop. + CleanupSetupNormalizedTables(ctx context.Context, tx any) + + // FinishSetupNormalizedTables may be used to finish transaction started by StartSetupNormalizedTables. + FinishSetupNormalizedTables(ctx context.Context, tx any) error + // SetupNormalizedTable sets up the normalized table on the connector. SetupNormalizedTable( ctx context.Context, @@ -100,16 +118,9 @@ type NormalizedTablesConnector interface { softDeleteColName string, syncedAtColName string, ) (bool, error) - - // CleanupSetupNormalizedTables may be used to rollback transaction started by StartSetupNormalizedTables. - // Calling CleanupSetupNormalizedTables after FinishSetupNormalizedTables must be a nop. - CleanupSetupNormalizedTables(ctx context.Context, tx any) - - // FinishSetupNormalizedTables may be used to finish transaction started by StartSetupNormalizedTables. - FinishSetupNormalizedTables(ctx context.Context, tx any) error } -type CDCSyncConnector interface { +type CDCSyncConnectorCore interface { Connector // NeedsSetupMetadataTables checks if the metadata table [PEERDB_MIRROR_JOBS] needs to be created. @@ -130,17 +141,30 @@ type CDCSyncConnector interface { // CreateRawTable creates a raw table for the connector with a given name and a fixed schema. CreateRawTable(ctx context.Context, req *protos.CreateRawTableInput) (*protos.CreateRawTableOutput, error) + // SyncFlowCleanup drops metadata tables on the destination, as a part of DROP MIRROR. + SyncFlowCleanup(ctx context.Context, jobName string) error + // ReplayTableSchemaDelta changes a destination table to match the schema at source // This could involve adding or dropping multiple columns. // Connectors which are non-normalizing should implement this as a nop. ReplayTableSchemaDeltas(ctx context.Context, flowJobName string, schemaDeltas []*protos.TableSchemaDelta) error +} + +type CDCSyncConnector interface { + CDCSyncConnectorCore - // SyncRecords pushes records to the destination peer and stores it in PeerDB specific tables. + // SyncRecords pushes RecordItems to the destination peer and stores it in PeerDB specific tables. // This method should be idempotent, and should be able to be called multiple times with the same request. - SyncRecords(ctx context.Context, req *model.SyncRecordsRequest) (*model.SyncResponse, error) + SyncRecords(ctx context.Context, req *model.SyncRecordsRequest[model.RecordItems]) (*model.SyncResponse, error) +} - // SyncFlowCleanup drops metadata tables on the destination, as a part of DROP MIRROR. - SyncFlowCleanup(ctx context.Context, jobName string) error +type CDCSyncPgConnector interface { + CDCSyncConnectorCore + + // SyncPg pushes PgItems to the destination peer and stores it in PeerDB specific tables. + // This method should be idempotent, and should be able to be called multiple times with the same request. + // It's signature, aside from type parameter, should match CDCSyncConnector.SyncRecords. + SyncPg(ctx context.Context, req *model.SyncRecordsRequest[model.PgItems]) (*model.SyncResponse, error) } type CDCNormalizeConnector interface { @@ -214,6 +238,8 @@ func GetConnector(ctx context.Context, config *protos.Peer) (Connector, error) { return connkafka.NewKafkaConnector(ctx, inner.KafkaConfig) case *protos.Peer_PubsubConfig: return connpubsub.NewPubSubConnector(ctx, inner.PubsubConfig) + case *protos.Peer_ElasticsearchConfig: + return connelasticsearch.NewElasticsearchConnector(ctx, inner.ElasticsearchConfig) default: return nil, ErrUnsupportedFunctionality } @@ -268,6 +294,8 @@ func CloseConnector(ctx context.Context, conn Connector) { var ( _ CDCPullConnector = &connpostgres.PostgresConnector{} + _ CDCPullPgConnector = &connpostgres.PostgresConnector{} + _ CDCSyncConnector = &connpostgres.PostgresConnector{} _ CDCSyncConnector = &connbigquery.BigQueryConnector{} _ CDCSyncConnector = &connsnowflake.SnowflakeConnector{} @@ -277,6 +305,8 @@ var ( _ CDCSyncConnector = &conns3.S3Connector{} _ CDCSyncConnector = &connclickhouse.ClickhouseConnector{} + _ CDCSyncPgConnector = &connpostgres.PostgresConnector{} + _ CDCNormalizeConnector = &connpostgres.PostgresConnector{} _ CDCNormalizeConnector = &connbigquery.BigQueryConnector{} _ CDCNormalizeConnector = &connsnowflake.SnowflakeConnector{} @@ -298,6 +328,7 @@ var ( _ QRepSyncConnector = &connsnowflake.SnowflakeConnector{} _ QRepSyncConnector = &connclickhouse.ClickhouseConnector{} _ QRepSyncConnector = &conns3.S3Connector{} + _ QRepSyncConnector = &connelasticsearch.ElasticsearchConnector{} _ QRepConsolidateConnector = &connsnowflake.SnowflakeConnector{} _ QRepConsolidateConnector = &connclickhouse.ClickhouseConnector{} diff --git a/flow/connectors/eventhub/eventhub.go b/flow/connectors/eventhub/eventhub.go index fe7c4de295..4f95d1007e 100644 --- a/flow/connectors/eventhub/eventhub.go +++ b/flow/connectors/eventhub/eventhub.go @@ -180,9 +180,7 @@ type ScopedEventhubData struct { // returns the number of records synced func (c *EventHubConnector) processBatch( ctx context.Context, - flowJobName string, - batch *model.CDCRecordStream, - script string, + req *model.SyncRecordsRequest[model.RecordItems], ) (uint32, error) { batchPerTopic := NewHubBatches(c.hubManager) toJSONOpts := model.NewToJSONOptions(c.config.UnnestColumns, false) @@ -191,21 +189,20 @@ func (c *EventHubConnector) processBatch( defer ticker.Stop() lastSeenLSN := int64(0) - lastUpdatedOffset := int64(0) numRecords := atomic.Uint32{} var ls *lua.LState var fn *lua.LFunction - if script != "" { + if req.Script != "" { var err error - ls, err = utils.LoadScript(ctx, script, func(ls *lua.LState) int { + ls, err = utils.LoadScript(ctx, req.Script, func(ls *lua.LState) int { top := ls.GetTop() ss := make([]string, top) for i := range top { ss[i] = ls.ToStringMeta(ls.Get(i + 1)).String() } - _ = c.LogFlowInfo(ctx, flowJobName, strings.Join(ss, "\t")) + _ = c.LogFlowInfo(ctx, req.FlowJobName, strings.Join(ss, "\t")) return 0 }) if err != nil { @@ -223,10 +220,10 @@ func (c *EventHubConnector) processBatch( for { select { - case record, ok := <-batch.GetRecords(): + case record, ok := <-req.Records.GetRecords(): if !ok { c.logger.Info("flushing batches because no more records") - err := batchPerTopic.flushAllBatches(ctx, flowJobName) + err := batchPerTopic.flushAllBatches(ctx, req.FlowJobName) if err != nil { return 0, err } @@ -322,25 +319,23 @@ func (c *EventHubConnector) processBatch( return 0, fmt.Errorf("[eventhub] context cancelled %w", ctx.Err()) case <-ticker.C: - err := batchPerTopic.flushAllBatches(ctx, flowJobName) + err := batchPerTopic.flushAllBatches(ctx, req.FlowJobName) if err != nil { return 0, err - } - - if lastSeenLSN > lastUpdatedOffset { - err = c.SetLastOffset(ctx, flowJobName, lastSeenLSN) - lastUpdatedOffset = lastSeenLSN - c.logger.Info("processBatch", slog.Int64("updated last offset", lastSeenLSN)) - if err != nil { - return 0, fmt.Errorf("failed to update last offset: %w", err) + } else if lastSeenLSN > req.ConsumedOffset.Load() { + if err := c.SetLastOffset(ctx, req.FlowJobName, lastSeenLSN); err != nil { + c.logger.Warn("[eventhubs] SetLastOffset error", slog.Any("error", err)) + } else { + shared.AtomicInt64Max(req.ConsumedOffset, lastSeenLSN) + c.logger.Info("processBatch", slog.Int64("updated last offset", lastSeenLSN)) } } } } } -func (c *EventHubConnector) SyncRecords(ctx context.Context, req *model.SyncRecordsRequest) (*model.SyncResponse, error) { - numRecords, err := c.processBatch(ctx, req.FlowJobName, req.Records, req.Script) +func (c *EventHubConnector) SyncRecords(ctx context.Context, req *model.SyncRecordsRequest[model.RecordItems]) (*model.SyncResponse, error) { + numRecords, err := c.processBatch(ctx, req) if err != nil { c.logger.Error("failed to process batch", slog.Any("error", err)) return nil, err diff --git a/flow/connectors/kafka/kafka.go b/flow/connectors/kafka/kafka.go index ced57f5bf6..d02bb94ab7 100644 --- a/flow/connectors/kafka/kafka.go +++ b/flow/connectors/kafka/kafka.go @@ -166,41 +166,46 @@ func lvalueToKafkaRecord(ls *lua.LState, value lua.LValue) (*kgo.Record, error) return kr, nil } -func (c *KafkaConnector) SyncRecords(ctx context.Context, req *model.SyncRecordsRequest) (*model.SyncResponse, error) { +func (c *KafkaConnector) SyncRecords(ctx context.Context, req *model.SyncRecordsRequest[model.RecordItems]) (*model.SyncResponse, error) { var wg sync.WaitGroup wgCtx, wgErr := context.WithCancelCause(ctx) - produceCb := func(r *kgo.Record, err error) { + produceCb := func(_ *kgo.Record, err error) { if err != nil { wgErr(err) } wg.Done() } - numRecords := int64(0) + numRecords := atomic.Int64{} tableNameRowsMapping := utils.InitialiseTableRowsMap(req.TableMappings) - ls, err := utils.LoadScript(wgCtx, req.Script, func(ls *lua.LState) int { - top := ls.GetTop() - ss := make([]string, top) - for i := range top { - ss[i] = ls.ToStringMeta(ls.Get(i + 1)).String() + pool, err := utils.LuaPool(func() (*lua.LState, error) { + ls, err := utils.LoadScript(wgCtx, req.Script, func(ls *lua.LState) int { + top := ls.GetTop() + ss := make([]string, top) + for i := range top { + ss[i] = ls.ToStringMeta(ls.Get(i + 1)).String() + } + _ = c.LogFlowInfo(ctx, req.FlowJobName, strings.Join(ss, "\t")) + return 0 + }) + if err != nil { + return nil, err + } + if req.Script == "" { + ls.Env.RawSetString("onRecord", ls.NewFunction(utils.DefaultOnRecord)) + } + return ls, nil + }, func(krs []*kgo.Record) { + wg.Add(len(krs)) + for _, kr := range krs { + c.client.Produce(wgCtx, kr, produceCb) } - _ = c.LogFlowInfo(ctx, req.FlowJobName, strings.Join(ss, "\t")) - return 0 }) if err != nil { return nil, err } - defer ls.Close() - if req.Script == "" { - ls.Env.RawSetString("onRecord", ls.NewFunction(utils.DefaultOnRecord)) - } - - lfn := ls.Env.RawGetString("onRecord") - fn, ok := lfn.(*lua.LFunction) - if !ok { - return nil, fmt.Errorf("script should define `onRecord` as function, not %s", lfn) - } + defer pool.Close() lastSeenLSN := atomic.Int64{} flushLoopDone := make(chan struct{}) @@ -208,7 +213,6 @@ func (c *KafkaConnector) SyncRecords(ctx context.Context, req *model.SyncRecords ticker := time.NewTicker(peerdbenv.PeerDBQueueFlushTimeoutSeconds()) defer ticker.Stop() - lastUpdatedOffset := int64(0) for { select { case <-ctx.Done(): @@ -221,11 +225,11 @@ func (c *KafkaConnector) SyncRecords(ctx context.Context, req *model.SyncRecords if err := c.client.Flush(ctx); err != nil { c.logger.Warn("[kafka] flush error", slog.Any("error", err)) continue - } else if lastSeen > lastUpdatedOffset { + } else if lastSeen > req.ConsumedOffset.Load() { if err := c.SetLastOffset(ctx, req.FlowJobName, lastSeen); err != nil { c.logger.Warn("[kafka] SetLastOffset error", slog.Any("error", err)) } else { - lastUpdatedOffset = lastSeen + shared.AtomicInt64Max(req.ConsumedOffset, lastSeen) c.logger.Info("processBatch", slog.Int64("updated last offset", lastSeen)) } } @@ -242,39 +246,54 @@ Loop: break Loop } - ls.Push(fn) - ls.Push(pua.LuaRecord.New(ls, record)) - err := ls.PCall(1, -1, nil) - if err != nil { - return nil, fmt.Errorf("script failed: %w", err) - } - args := ls.GetTop() - for i := range args { - kr, err := lvalueToKafkaRecord(ls, ls.Get(i-args)) + pool.Run(func(ls *lua.LState) []*kgo.Record { + lfn := ls.Env.RawGetString("onRecord") + fn, ok := lfn.(*lua.LFunction) + if !ok { + wgErr(fmt.Errorf("script should define `onRecord` as function, not %s", lfn)) + return nil + } + + ls.Push(fn) + ls.Push(pua.LuaRecord.New(ls, record)) + err := ls.PCall(1, -1, nil) if err != nil { - return nil, err + wgErr(fmt.Errorf("script failed: %w", err)) + return nil } - if kr != nil { - if kr.Topic == "" { - kr.Topic = record.GetDestinationTableName() - } - wg.Add(1) - c.client.Produce(wgCtx, kr, produceCb) - record.PopulateCountMap(tableNameRowsMapping) + args := ls.GetTop() + results := make([]*kgo.Record, 0, args) + for i := range args { + kr, err := lvalueToKafkaRecord(ls, ls.Get(i-args)) + if err != nil { + wgErr(err) + return nil + } + if kr != nil { + if kr.Topic == "" { + kr.Topic = record.GetDestinationTableName() + } + results = append(results, kr) + record.PopulateCountMap(tableNameRowsMapping) + } } - } - ls.SetTop(0) - numRecords += 1 - shared.AtomicInt64Max(&lastSeenLSN, record.GetCheckpointID()) + ls.SetTop(0) + numRecords.Add(1) + shared.AtomicInt64Max(&lastSeenLSN, record.GetCheckpointID()) + return results + }) case <-wgCtx.Done(): - return nil, wgCtx.Err() + break Loop } } close(flushLoopDone) - if err := c.client.Flush(ctx); err != nil { + if err := pool.Wait(wgCtx); err != nil { + return nil, err + } + if err := c.client.Flush(wgCtx); err != nil { return nil, fmt.Errorf("[kafka] final flush error: %w", err) } waitChan := make(chan struct{}) @@ -289,15 +308,14 @@ Loop: } lastCheckpoint := req.Records.GetLastCheckpoint() - err = c.FinishBatch(ctx, req.FlowJobName, req.SyncBatchID, lastCheckpoint) - if err != nil { + if err := c.FinishBatch(ctx, req.FlowJobName, req.SyncBatchID, lastCheckpoint); err != nil { return nil, err } return &model.SyncResponse{ CurrentSyncBatchID: req.SyncBatchID, LastSyncedCheckpointID: lastCheckpoint, - NumRecordsSynced: numRecords, + NumRecordsSynced: numRecords.Load(), TableNameRowsMapping: tableNameRowsMapping, TableSchemaDeltas: req.Records.SchemaDeltas, }, nil diff --git a/flow/connectors/postgres/cdc.go b/flow/connectors/postgres/cdc.go index 11eb66139c..4adcc1d0d4 100644 --- a/flow/connectors/postgres/cdc.go +++ b/flow/connectors/postgres/cdc.go @@ -18,7 +18,6 @@ import ( "go.temporal.io/sdk/activity" "github.com/PeerDB-io/peer-flow/connectors/utils" - "github.com/PeerDB-io/peer-flow/connectors/utils/cdc_records" geo "github.com/PeerDB-io/peer-flow/datatypes" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/logger" @@ -110,28 +109,209 @@ func GetChildToParentRelIDMap(ctx context.Context, conn *pgx.Conn) (map[uint32]u return childToParentRelIDMap, nil } -// PullRecords pulls records from the cdc stream -func (p *PostgresCDCSource) PullRecords(ctx context.Context, req *model.PullRecordsRequest) error { +// replProcessor implements ingesting PostgreSQL logical replication tuples into items. +type replProcessor[Items model.Items] interface { + NewItems(int) Items + + Process( + items Items, + p *PostgresCDCSource, + tuple *pglogrepl.TupleDataColumn, + col *pglogrepl.RelationMessageColumn, + ) error +} + +type pgProcessor struct{} + +func (pgProcessor) NewItems(size int) model.PgItems { + return model.NewPgItems(size) +} + +func (pgProcessor) Process( + items model.PgItems, + p *PostgresCDCSource, + tuple *pglogrepl.TupleDataColumn, + col *pglogrepl.RelationMessageColumn, +) error { + switch tuple.DataType { + case 'n': // null + items.AddColumn(col.Name, nil) + case 't': // text + // bytea also appears here as a hex + items.AddColumn(col.Name, tuple.Data) + case 'b': // binary + return fmt.Errorf( + "binary encoding not supported, received for %s type %d", + col.Name, + col.DataType, + ) + default: + return fmt.Errorf("unknown column data type: %s", string(tuple.DataType)) + } + return nil +} + +type qProcessor struct{} + +func (qProcessor) NewItems(size int) model.RecordItems { + return model.NewRecordItems(size) +} + +func (qProcessor) Process( + items model.RecordItems, + p *PostgresCDCSource, + tuple *pglogrepl.TupleDataColumn, + col *pglogrepl.RelationMessageColumn, +) error { + switch tuple.DataType { + case 'n': // null + items.AddColumn(col.Name, qvalue.QValueNull(qvalue.QValueKindInvalid)) + case 't': // text + // bytea also appears here as a hex + data, err := p.decodeColumnData(tuple.Data, col.DataType, pgtype.TextFormatCode) + if err != nil { + return fmt.Errorf("error decoding text column data: %w", err) + } + items.AddColumn(col.Name, data) + case 'b': // binary + data, err := p.decodeColumnData(tuple.Data, col.DataType, pgtype.BinaryFormatCode) + if err != nil { + return fmt.Errorf("error decoding binary column data: %w", err) + } + items.AddColumn(col.Name, data) + default: + return fmt.Errorf("unknown column data type: %s", string(tuple.DataType)) + } + return nil +} + +func processTuple[Items model.Items]( + processor replProcessor[Items], + p *PostgresCDCSource, + tuple *pglogrepl.TupleData, + rel *pglogrepl.RelationMessage, + exclude map[string]struct{}, +) (Items, map[string]struct{}, error) { + // if the tuple is nil, return an empty map + if tuple == nil { + return processor.NewItems(0), nil, nil + } + + items := processor.NewItems(len(tuple.Columns)) + var unchangedToastColumns map[string]struct{} + + for idx, tcol := range tuple.Columns { + rcol := rel.Columns[idx] + if _, ok := exclude[rcol.Name]; ok { + continue + } + if tcol.DataType == 'u' { + if unchangedToastColumns == nil { + unchangedToastColumns = make(map[string]struct{}) + } + unchangedToastColumns[rcol.Name] = struct{}{} + } else if err := processor.Process(items, p, tcol, rcol); err != nil { + var none Items + return none, nil, err + } + } + return items, unchangedToastColumns, nil +} + +func (p *PostgresCDCSource) decodeColumnData(data []byte, dataType uint32, formatCode int16) (qvalue.QValue, error) { + var parsedData any + var err error + if dt, ok := p.typeMap.TypeForOID(dataType); ok { + if dt.Name == "uuid" || dt.Name == "cidr" || dt.Name == "inet" || dt.Name == "macaddr" { + // below is required to decode above types to string + parsedData, err = dt.Codec.DecodeDatabaseSQLValue(p.typeMap, dataType, pgtype.TextFormatCode, data) + } else { + parsedData, err = dt.Codec.DecodeValue(p.typeMap, dataType, formatCode, data) + } + if err != nil { + if dt.Name == "time" || dt.Name == "timetz" || + dt.Name == "timestamp" || dt.Name == "timestamptz" { + // indicates year is more than 4 digits or something similar, + // which you can insert into postgres, + // but not representable by time.Time + p.logger.Warn(fmt.Sprintf("Invalidated and hence nulled %s data: %s", + dt.Name, string(data))) + switch dt.Name { + case "time": + return qvalue.QValueNull(qvalue.QValueKindTime), nil + case "timetz": + return qvalue.QValueNull(qvalue.QValueKindTimeTZ), nil + case "timestamp": + return qvalue.QValueNull(qvalue.QValueKindTimestamp), nil + case "timestamptz": + return qvalue.QValueNull(qvalue.QValueKindTimestampTZ), nil + } + } + return nil, err + } + retVal, err := p.parseFieldFromPostgresOID(dataType, parsedData) + if err != nil { + return nil, err + } + return retVal, nil + } else if dataType == uint32(oid.T_timetz) { // ugly TIMETZ workaround for CDC decoding. + retVal, err := p.parseFieldFromPostgresOID(dataType, string(data)) + if err != nil { + return nil, err + } + return retVal, nil + } + + typeName, ok := p.customTypesMapping[dataType] + if ok { + customQKind := customTypeToQKind(typeName) + switch customQKind { + case qvalue.QValueKindGeography, qvalue.QValueKindGeometry: + wkt, err := geo.GeoValidate(string(data)) + if err != nil { + return qvalue.QValueNull(customQKind), nil + } else if customQKind == qvalue.QValueKindGeography { + return qvalue.QValueGeography{Val: wkt}, nil + } else { + return qvalue.QValueGeometry{Val: wkt}, nil + } + case qvalue.QValueKindHStore: + return qvalue.QValueHStore{Val: string(data)}, nil + case qvalue.QValueKindString: + return qvalue.QValueString{Val: string(data)}, nil + default: + return nil, fmt.Errorf("unknown custom qkind: %s", customQKind) + } + } + + return qvalue.QValueString{Val: string(data)}, nil +} + +// PullCdcRecords pulls records from req's cdc stream +func PullCdcRecords[Items model.Items]( + ctx context.Context, + p *PostgresCDCSource, + req *model.PullRecordsRequest[Items], + processor replProcessor[Items], +) error { logger := logger.LoggerFromCtx(ctx) conn := p.replConn.PgConn() records := req.RecordStream // clientXLogPos is the last checkpoint id, we need to ack that we have processed // until clientXLogPos each time we send a standby status update. - // consumedXLogPos is the lsn that has been committed on the destination. - var clientXLogPos, consumedXLogPos pglogrepl.LSN + var clientXLogPos pglogrepl.LSN if req.LastOffset > 0 { clientXLogPos = pglogrepl.LSN(req.LastOffset) - consumedXLogPos = clientXLogPos err := pglogrepl.SendStandbyStatusUpdate(ctx, conn, - pglogrepl.StandbyStatusUpdate{WALWritePosition: consumedXLogPos}) + pglogrepl.StandbyStatusUpdate{WALWritePosition: pglogrepl.LSN(req.ConsumedOffset.Load())}) if err != nil { return fmt.Errorf("[initial-flush] SendStandbyStatusUpdate failed: %w", err) } } var standByLastLogged time.Time - cdcRecordsStorage := cdc_records.NewCDCRecordsStore(p.flowJobName) + cdcRecordsStorage := utils.NewCDCStore[Items](p.flowJobName) defer func() { if cdcRecordsStorage.IsEmpty() { records.SignalAsEmpty() @@ -154,7 +334,7 @@ func (p *PostgresCDCSource) PullRecords(ctx context.Context, req *model.PullReco standbyMessageTimeout := req.IdleTimeout nextStandbyMessageDeadline := time.Now().Add(standbyMessageTimeout) - addRecordWithKey := func(key model.TableWithPkey, rec model.Record) error { + addRecordWithKey := func(key model.TableWithPkey, rec model.Record[Items]) error { err := cdcRecordsStorage.Set(logger, key, rec) if err != nil { return err @@ -175,7 +355,7 @@ func (p *PostgresCDCSource) PullRecords(ctx context.Context, req *model.PullReco for { if pkmRequiresResponse { err := pglogrepl.SendStandbyStatusUpdate(ctx, conn, - pglogrepl.StandbyStatusUpdate{WALWritePosition: consumedXLogPos}) + pglogrepl.StandbyStatusUpdate{WALWritePosition: pglogrepl.LSN(req.ConsumedOffset.Load())}) if err != nil { return fmt.Errorf("SendStandbyStatusUpdate failed: %w", err) } @@ -289,7 +469,7 @@ func (p *PostgresCDCSource) PullRecords(ctx context.Context, req *model.PullReco logger.Debug(fmt.Sprintf("XLogData => WALStart %s ServerWALEnd %s ServerTime %s\n", xld.WALStart, xld.ServerWALEnd, xld.ServerTime)) - rec, err := p.processMessage(ctx, records, xld, clientXLogPos) + rec, err := processMessage(ctx, p, records, xld, clientXLogPos, processor) if err != nil { return fmt.Errorf("error processing message: %w", err) } @@ -297,7 +477,7 @@ func (p *PostgresCDCSource) PullRecords(ctx context.Context, req *model.PullReco if rec != nil { tableName := rec.GetDestinationTableName() switch r := rec.(type) { - case *model.UpdateRecord: + case *model.UpdateRecord[Items]: // tableName here is destination tableName. // should be ideally sourceTableName as we are in PullRecords. // will change in future @@ -308,7 +488,7 @@ func (p *PostgresCDCSource) PullRecords(ctx context.Context, req *model.PullReco return err } } else { - tablePkeyVal, err := p.recToTablePKey(req, rec) + tablePkeyVal, err := recToTablePKey(req, rec) if err != nil { return err } @@ -332,7 +512,7 @@ func (p *PostgresCDCSource) PullRecords(ctx context.Context, req *model.PullReco } } - case *model.InsertRecord: + case *model.InsertRecord[Items]: isFullReplica := req.TableNameSchemaMapping[tableName].IsReplicaIdentityFull if isFullReplica { err := addRecordWithKey(model.TableWithPkey{}, rec) @@ -340,7 +520,7 @@ func (p *PostgresCDCSource) PullRecords(ctx context.Context, req *model.PullReco return err } } else { - tablePkeyVal, err := p.recToTablePKey(req, rec) + tablePkeyVal, err := recToTablePKey(req, rec) if err != nil { return err } @@ -350,7 +530,7 @@ func (p *PostgresCDCSource) PullRecords(ctx context.Context, req *model.PullReco return err } } - case *model.DeleteRecord: + case *model.DeleteRecord[Items]: isFullReplica := req.TableNameSchemaMapping[tableName].IsReplicaIdentityFull if isFullReplica { err := addRecordWithKey(model.TableWithPkey{}, rec) @@ -358,7 +538,7 @@ func (p *PostgresCDCSource) PullRecords(ctx context.Context, req *model.PullReco return err } } else { - tablePkeyVal, err := p.recToTablePKey(req, rec) + tablePkeyVal, err := recToTablePKey(req, rec) if err != nil { return err } @@ -368,18 +548,15 @@ func (p *PostgresCDCSource) PullRecords(ctx context.Context, req *model.PullReco return err } if ok { - deleteRecord := rec.(*model.DeleteRecord) - deleteRecord.Items = latestRecord.GetItems() - updateRecord, ok := latestRecord.(*model.UpdateRecord) - if ok { - deleteRecord.UnchangedToastColumns = updateRecord.UnchangedToastColumns + r.Items = latestRecord.GetItems() + if updateRecord, ok := latestRecord.(*model.UpdateRecord[Items]); ok { + r.UnchangedToastColumns = updateRecord.UnchangedToastColumns } } else { - deleteRecord := rec.(*model.DeleteRecord) // there is nothing to backfill the items in the delete record with, // so don't update the row with this record // add sentinel value to prevent update statements from selecting - deleteRecord.UnchangedToastColumns = map[string]struct{}{ + r.UnchangedToastColumns = map[string]struct{}{ "_peerdb_not_backfilled_delete": {}, } } @@ -392,7 +569,7 @@ func (p *PostgresCDCSource) PullRecords(ctx context.Context, req *model.PullReco } } - case *model.RelationRecord: + case *model.RelationRecord[Items]: tableSchemaDelta := r.TableSchemaDelta if len(tableSchemaDelta.AddedColumns) > 0 { logger.Info(fmt.Sprintf("Detected schema change for table %s, addedColumns: %v", @@ -420,12 +597,14 @@ func (p *PostgresCDCSource) baseRecord(lsn pglogrepl.LSN) model.BaseRecord { } } -func (p *PostgresCDCSource) processMessage( +func processMessage[Items model.Items]( ctx context.Context, - batch *model.CDCRecordStream, + p *PostgresCDCSource, + batch *model.CDCStream[Items], xld pglogrepl.XLogData, currentClientXlogPos pglogrepl.LSN, -) (model.Record, error) { + processor replProcessor[Items], +) (model.Record[Items], error) { logger := logger.LoggerFromCtx(ctx) logicalMsg, err := pglogrepl.Parse(xld.WALData) if err != nil { @@ -438,11 +617,11 @@ func (p *PostgresCDCSource) processMessage( logger.Debug("Locking PullRecords at BeginMessage, awaiting CommitMessage") p.commitLock = msg case *pglogrepl.InsertMessage: - return p.processInsertMessage(xld.WALStart, msg) + return processInsertMessage(p, xld.WALStart, msg, processor) case *pglogrepl.UpdateMessage: - return p.processUpdateMessage(xld.WALStart, msg) + return processUpdateMessage(p, xld.WALStart, msg, processor) case *pglogrepl.DeleteMessage: - return p.processDeleteMessage(xld.WALStart, msg) + return processDeleteMessage(p, xld.WALStart, msg, processor) case *pglogrepl.CommitMessage: // for a commit message, update the last checkpoint id for the record batch. logger.Debug(fmt.Sprintf("CommitMessage => CommitLSN: %v, TransactionEndLSN: %v", @@ -460,7 +639,7 @@ func (p *PostgresCDCSource) processMessage( logger.Debug(fmt.Sprintf("RelationMessage => RelationID: %d, Namespace: %s, RelationName: %s, Columns: %v", msg.RelationID, msg.Namespace, msg.RelationName, msg.Columns)) - return p.processRelationMessage(ctx, currentClientXlogPos, msg) + return processRelationMessage[Items](ctx, p, currentClientXlogPos, msg) case *pglogrepl.TruncateMessage: logger.Warn("TruncateMessage not supported") @@ -469,10 +648,12 @@ func (p *PostgresCDCSource) processMessage( return nil, nil } -func (p *PostgresCDCSource) processInsertMessage( +func processInsertMessage[Items model.Items]( + p *PostgresCDCSource, lsn pglogrepl.LSN, msg *pglogrepl.InsertMessage, -) (model.Record, error) { + processor replProcessor[Items], +) (model.Record[Items], error) { relID := p.getParentRelIDIfPartitioned(msg.RelationID) tableName, exists := p.srcTableIDNameMapping[relID] @@ -489,13 +670,12 @@ func (p *PostgresCDCSource) processInsertMessage( return nil, fmt.Errorf("unknown relation id: %d", relID) } - // create empty map of string to interface{} - items, _, err := p.convertTupleToMap(msg.Tuple, rel, p.tableNameMapping[tableName].Exclude) + items, _, err := processTuple(processor, p, msg.Tuple, rel, p.tableNameMapping[tableName].Exclude) if err != nil { return nil, fmt.Errorf("error converting tuple to map: %w", err) } - return &model.InsertRecord{ + return &model.InsertRecord[Items]{ BaseRecord: p.baseRecord(lsn), Items: items, DestinationTableName: p.tableNameMapping[tableName].Name, @@ -504,10 +684,12 @@ func (p *PostgresCDCSource) processInsertMessage( } // processUpdateMessage processes an update message and returns an UpdateRecord -func (p *PostgresCDCSource) processUpdateMessage( +func processUpdateMessage[Items model.Items]( + p *PostgresCDCSource, lsn pglogrepl.LSN, msg *pglogrepl.UpdateMessage, -) (model.Record, error) { + processor replProcessor[Items], +) (model.Record[Items], error) { relID := p.getParentRelIDIfPartitioned(msg.RelationID) tableName, exists := p.srcTableIDNameMapping[relID] @@ -524,19 +706,18 @@ func (p *PostgresCDCSource) processUpdateMessage( return nil, fmt.Errorf("unknown relation id: %d", relID) } - // create empty map of string to interface{} - oldItems, _, err := p.convertTupleToMap(msg.OldTuple, rel, p.tableNameMapping[tableName].Exclude) + oldItems, _, err := processTuple(processor, p, msg.OldTuple, rel, p.tableNameMapping[tableName].Exclude) if err != nil { return nil, fmt.Errorf("error converting old tuple to map: %w", err) } - newItems, unchangedToastColumns, err := p.convertTupleToMap(msg.NewTuple, - rel, p.tableNameMapping[tableName].Exclude) + newItems, unchangedToastColumns, err := processTuple( + processor, p, msg.NewTuple, rel, p.tableNameMapping[tableName].Exclude) if err != nil { return nil, fmt.Errorf("error converting new tuple to map: %w", err) } - return &model.UpdateRecord{ + return &model.UpdateRecord[Items]{ BaseRecord: p.baseRecord(lsn), OldItems: oldItems, NewItems: newItems, @@ -547,10 +728,12 @@ func (p *PostgresCDCSource) processUpdateMessage( } // processDeleteMessage processes a delete message and returns a DeleteRecord -func (p *PostgresCDCSource) processDeleteMessage( +func processDeleteMessage[Items model.Items]( + p *PostgresCDCSource, lsn pglogrepl.LSN, msg *pglogrepl.DeleteMessage, -) (model.Record, error) { + processor replProcessor[Items], +) (model.Record[Items], error) { relID := p.getParentRelIDIfPartitioned(msg.RelationID) tableName, exists := p.srcTableIDNameMapping[relID] @@ -567,13 +750,12 @@ func (p *PostgresCDCSource) processDeleteMessage( return nil, fmt.Errorf("unknown relation id: %d", relID) } - // create empty map of string to interface{} - items, _, err := p.convertTupleToMap(msg.OldTuple, rel, p.tableNameMapping[tableName].Exclude) + items, _, err := processTuple(processor, p, msg.OldTuple, rel, p.tableNameMapping[tableName].Exclude) if err != nil { return nil, fmt.Errorf("error converting tuple to map: %w", err) } - return &model.DeleteRecord{ + return &model.DeleteRecord[Items]{ BaseRecord: p.baseRecord(lsn), Items: items, DestinationTableName: p.tableNameMapping[tableName].Name, @@ -581,128 +763,7 @@ func (p *PostgresCDCSource) processDeleteMessage( }, nil } -/* -convertTupleToMap converts a PostgreSQL logical replication -tuple to a map representation. -It takes a tuple and a relation message as input and returns -1. a map of column names to values and -2. a string slice of unchanged TOAST column names -*/ -func (p *PostgresCDCSource) convertTupleToMap( - tuple *pglogrepl.TupleData, - rel *pglogrepl.RelationMessage, - exclude map[string]struct{}, -) (model.RecordItems, map[string]struct{}, error) { - // if the tuple is nil, return an empty map - if tuple == nil { - return model.NewRecordItems(0), make(map[string]struct{}), nil - } - - // create empty map of string to interface{} - items := model.NewRecordItems(len(tuple.Columns)) - unchangedToastColumns := make(map[string]struct{}) - - for idx, col := range tuple.Columns { - colName := rel.Columns[idx].Name - if _, ok := exclude[colName]; ok { - continue - } - switch col.DataType { - case 'n': // null - val := qvalue.QValueNull(qvalue.QValueKindInvalid) - items.AddColumn(colName, val) - case 't': // text - /* bytea also appears here as a hex */ - data, err := p.decodeColumnData(col.Data, rel.Columns[idx].DataType, pgtype.TextFormatCode) - if err != nil { - return model.RecordItems{}, nil, fmt.Errorf("error decoding text column data: %w", err) - } - items.AddColumn(colName, data) - case 'b': // binary - data, err := p.decodeColumnData(col.Data, rel.Columns[idx].DataType, pgtype.BinaryFormatCode) - if err != nil { - return model.RecordItems{}, nil, fmt.Errorf("error decoding binary column data: %w", err) - } - items.AddColumn(colName, data) - case 'u': // unchanged toast - unchangedToastColumns[colName] = struct{}{} - default: - return model.RecordItems{}, nil, fmt.Errorf("unknown column data type: %s", string(col.DataType)) - } - } - return items, unchangedToastColumns, nil -} - -func (p *PostgresCDCSource) decodeColumnData(data []byte, dataType uint32, formatCode int16) (qvalue.QValue, error) { - var parsedData any - var err error - if dt, ok := p.typeMap.TypeForOID(dataType); ok { - if dt.Name == "uuid" || dt.Name == "cidr" || dt.Name == "inet" || dt.Name == "macaddr" { - // below is required to decode above types to string - parsedData, err = dt.Codec.DecodeDatabaseSQLValue(p.typeMap, dataType, pgtype.TextFormatCode, data) - } else { - parsedData, err = dt.Codec.DecodeValue(p.typeMap, dataType, formatCode, data) - } - if err != nil { - if dt.Name == "time" || dt.Name == "timetz" || - dt.Name == "timestamp" || dt.Name == "timestamptz" { - // indicates year is more than 4 digits or something similar, - // which you can insert into postgres, - // but not representable by time.Time - p.logger.Warn(fmt.Sprintf("Invalidated and hence nulled %s data: %s", - dt.Name, string(data))) - switch dt.Name { - case "time": - return qvalue.QValueNull(qvalue.QValueKindTime), nil - case "timetz": - return qvalue.QValueNull(qvalue.QValueKindTimeTZ), nil - case "timestamp": - return qvalue.QValueNull(qvalue.QValueKindTimestamp), nil - case "timestamptz": - return qvalue.QValueNull(qvalue.QValueKindTimestampTZ), nil - } - } - return nil, err - } - retVal, err := p.parseFieldFromPostgresOID(dataType, parsedData) - if err != nil { - return nil, err - } - return retVal, nil - } else if dataType == uint32(oid.T_timetz) { // ugly TIMETZ workaround for CDC decoding. - retVal, err := p.parseFieldFromPostgresOID(dataType, string(data)) - if err != nil { - return nil, err - } - return retVal, nil - } - - typeName, ok := p.customTypesMapping[dataType] - if ok { - customQKind := customTypeToQKind(typeName) - switch customQKind { - case qvalue.QValueKindGeography, qvalue.QValueKindGeometry: - wkt, err := geo.GeoValidate(string(data)) - if err != nil { - return qvalue.QValueNull(customQKind), nil - } else if customQKind == qvalue.QValueKindGeography { - return qvalue.QValueGeography{Val: wkt}, nil - } else { - return qvalue.QValueGeometry{Val: wkt}, nil - } - case qvalue.QValueKindHStore: - return qvalue.QValueHStore{Val: string(data)}, nil - case qvalue.QValueKindString: - return qvalue.QValueString{Val: string(data)}, nil - default: - return nil, fmt.Errorf("unknown custom qkind: %s", customQKind) - } - } - - return qvalue.QValueString{Val: string(data)}, nil -} - -func (p *PostgresCDCSource) auditSchemaDelta(ctx context.Context, flowJobName string, rec *model.RelationRecord) error { +func auditSchemaDelta[Items model.Items](ctx context.Context, p *PostgresCDCSource, rec *model.RelationRecord[Items]) error { activityInfo := activity.GetInfo(ctx) workflowID := activityInfo.WorkflowExecution.ID runID := activityInfo.WorkflowExecution.RunID @@ -711,7 +772,7 @@ func (p *PostgresCDCSource) auditSchemaDelta(ctx context.Context, flowJobName st `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, rec) + p.flowJobName, workflowID, runID, rec) if err != nil { return fmt.Errorf("failed to insert row into table: %w", err) } @@ -719,11 +780,12 @@ func (p *PostgresCDCSource) auditSchemaDelta(ctx context.Context, flowJobName st } // processRelationMessage processes a RelationMessage and returns a TableSchemaDelta -func (p *PostgresCDCSource) processRelationMessage( +func processRelationMessage[Items model.Items]( ctx context.Context, + p *PostgresCDCSource, lsn pglogrepl.LSN, currRel *pglogrepl.RelationMessage, -) (model.Record, error) { +) (model.Record[Items], error) { // not present in tables to sync, return immediately if _, ok := p.srcTableIDNameMapping[currRel.RelationID]; !ok { p.logger.Info("relid not present in srcTableIDNameMapping, skipping relation message", @@ -735,36 +797,46 @@ func (p *PostgresCDCSource) processRelationMessage( // tableNameSchemaMapping uses dst table name as the key, so annoying lookup prevSchema := p.tableNameSchemaMapping[p.tableNameMapping[p.srcTableIDNameMapping[currRel.RelationID]].Name] // creating maps for lookup later - prevRelMap := make(map[string]qvalue.QValueKind) - currRelMap := make(map[string]qvalue.QValueKind) + prevRelMap := make(map[string]string) + currRelMap := make(map[string]string) for _, column := range prevSchema.Columns { - prevRelMap[column.Name] = qvalue.QValueKind(column.Type) + prevRelMap[column.Name] = column.Type } for _, column := range currRel.Columns { - qKind := p.postgresOIDToQValueKind(column.DataType) - if qKind == qvalue.QValueKindInvalid { - typeName, ok := p.customTypesMapping[column.DataType] - if ok { - qKind = customTypeToQKind(typeName) + switch prevSchema.System { + case protos.TypeSystem_Q: + qKind := p.postgresOIDToQValueKind(column.DataType) + if qKind == qvalue.QValueKindInvalid { + typeName, ok := p.customTypesMapping[column.DataType] + if ok { + qKind = customTypeToQKind(typeName) + } } + currRelMap[column.Name] = string(qKind) + case protos.TypeSystem_PG: + currRelMap[column.Name] = p.postgresOIDToName(column.DataType) + default: + panic(fmt.Sprintf("cannot process schema changes for unknown type system %s", prevSchema.System)) } - currRelMap[column.Name] = qKind } schemaDelta := &protos.TableSchemaDelta{ SrcTableName: p.srcTableIDNameMapping[currRel.RelationID], DstTableName: p.tableNameMapping[p.srcTableIDNameMapping[currRel.RelationID]].Name, - AddedColumns: make([]*protos.DeltaAddedColumn, 0), + AddedColumns: make([]*protos.FieldDescription, 0), + System: prevSchema.System, } for _, column := range currRel.Columns { // not present in previous relation message, but in current one, so added. if _, ok := prevRelMap[column.Name]; !ok { // only add to delta if not excluded if _, ok := p.tableNameMapping[p.srcTableIDNameMapping[currRel.RelationID]].Exclude[column.Name]; !ok { - schemaDelta.AddedColumns = append(schemaDelta.AddedColumns, &protos.DeltaAddedColumn{ - ColumnName: column.Name, - ColumnType: string(currRelMap[column.Name]), - }) + schemaDelta.AddedColumns = append(schemaDelta.AddedColumns, &protos.FieldDescription{ + Name: column.Name, + Type: currRelMap[column.Name], + TypeModifier: column.TypeModifier, + }, + ) } // present in previous and current relation messages, but data types have changed. // so we add it to AddedColumns and DroppedColumns, knowing that we process DroppedColumns first. @@ -784,27 +856,28 @@ func (p *PostgresCDCSource) processRelationMessage( p.relationMessageMapping[currRel.RelationID] = currRel // only log audit if there is actionable delta if len(schemaDelta.AddedColumns) > 0 { - rec := &model.RelationRecord{ + rec := &model.RelationRecord[Items]{ BaseRecord: p.baseRecord(lsn), TableSchemaDelta: schemaDelta, } - return rec, p.auditSchemaDelta(ctx, p.flowJobName, rec) + return rec, auditSchemaDelta(ctx, p, rec) } return nil, nil } -func (p *PostgresCDCSource) recToTablePKey(req *model.PullRecordsRequest, - rec model.Record, +func recToTablePKey[Items model.Items]( + req *model.PullRecordsRequest[Items], + rec model.Record[Items], ) (model.TableWithPkey, error) { tableName := rec.GetDestinationTableName() pkeyColsMerged := make([][]byte, 0, len(req.TableNameSchemaMapping[tableName].PrimaryKeyColumns)) for _, pkeyCol := range req.TableNameSchemaMapping[tableName].PrimaryKeyColumns { - pkeyColVal, err := rec.GetItems().GetValueByColName(pkeyCol) + pkeyColBytes, err := rec.GetItems().GetBytesByColName(pkeyCol) if err != nil { return model.TableWithPkey{}, fmt.Errorf("error getting pkey column value: %w", err) } - pkeyColsMerged = append(pkeyColsMerged, []byte(fmt.Sprint(pkeyColVal.Value()))) + pkeyColsMerged = append(pkeyColsMerged, pkeyColBytes) } return model.TableWithPkey{ diff --git a/flow/connectors/postgres/client.go b/flow/connectors/postgres/client.go index 8bb172c746..c12ca20559 100644 --- a/flow/connectors/postgres/client.go +++ b/flow/connectors/postgres/client.go @@ -4,7 +4,6 @@ import ( "context" "errors" "fmt" - "regexp" "strings" "github.com/jackc/pglogrepl" @@ -425,8 +424,7 @@ func (c *PostgresConnector) createMetadataSchema(ctx context.Context) error { } func getRawTableIdentifier(jobName string) string { - jobName = regexp.MustCompile("[^a-zA-Z0-9_]+").ReplaceAllString(jobName, "_") - return fmt.Sprintf("%s_%s", rawTablePrefix, strings.ToLower(jobName)) + return rawTablePrefix + "_" + strings.ToLower(shared.ReplaceIllegalCharactersWithUnderscores(jobName)) } func generateCreateTableSQLForNormalizedTable( @@ -437,12 +435,13 @@ func generateCreateTableSQLForNormalizedTable( ) string { createTableSQLArray := make([]string, 0, len(sourceTableSchema.Columns)+2) for _, column := range sourceTableSchema.Columns { - pgColumnType := qValueKindToPostgresType(column.Type) - if column.Type == "numeric" { + pgColumnType := column.Type + if sourceTableSchema.System == protos.TypeSystem_Q { + pgColumnType = qValueKindToPostgresType(pgColumnType) + } + if column.Type == "numeric" && column.TypeModifier != -1 { precision, scale := numeric.ParseNumericTypmod(column.TypeModifier) - if column.TypeModifier != -1 { - pgColumnType = fmt.Sprintf("numeric(%d,%d)", precision, scale) - } + pgColumnType = fmt.Sprintf("numeric(%d,%d)", precision, scale) } createTableSQLArray = append(createTableSQLArray, fmt.Sprintf("%s %s", QuoteIdentifier(column.Name), pgColumnType)) diff --git a/flow/connectors/postgres/normalize_stmt_generator.go b/flow/connectors/postgres/normalize_stmt_generator.go index b6212c81b4..6d6dea8469 100644 --- a/flow/connectors/postgres/normalize_stmt_generator.go +++ b/flow/connectors/postgres/normalize_stmt_generator.go @@ -31,6 +31,17 @@ type normalizeStmtGenerator struct { supportsMerge bool } +func (n *normalizeStmtGenerator) columnTypeToPg(schema *protos.TableSchema, columnType string) string { + switch schema.System { + case protos.TypeSystem_Q: + return qValueKindToPostgresType(columnType) + case protos.TypeSystem_PG: + return columnType + default: + panic(fmt.Sprintf("unsupported system %s", schema.System)) + } +} + func (n *normalizeStmtGenerator) generateNormalizeStatements(dstTable string) []string { normalizedTableSchema := n.tableSchemaMapping[dstTable] if n.supportsMerge { @@ -58,17 +69,17 @@ func (n *normalizeStmtGenerator) generateFallbackStatements( quotedCol := QuoteIdentifier(column.Name) stringCol := QuoteLiteral(column.Name) columnNames = append(columnNames, quotedCol) - pgType := qValueKindToPostgresType(genericColumnType) - if qvalue.QValueKind(genericColumnType).IsArray() { - flattenedCastsSQLArray = append(flattenedCastsSQLArray, - fmt.Sprintf("ARRAY(SELECT * FROM JSON_ARRAY_ELEMENTS_TEXT((_peerdb_data->>%s)::JSON))::%s AS %s", - stringCol, pgType, quotedCol)) + pgType := n.columnTypeToPg(normalizedTableSchema, genericColumnType) + var expr string + if normalizedTableSchema.System == protos.TypeSystem_Q && qvalue.QValueKind(genericColumnType).IsArray() { + expr = fmt.Sprintf("ARRAY(SELECT JSON_ARRAY_ELEMENTS_TEXT((_peerdb_data->>%s)::JSON))::%s", stringCol, pgType) } else { - flattenedCastsSQLArray = append(flattenedCastsSQLArray, fmt.Sprintf("(_peerdb_data->>%s)::%s AS %s", - stringCol, pgType, quotedCol)) + expr = fmt.Sprintf("(_peerdb_data->>%s)::%s", stringCol, pgType) } + + flattenedCastsSQLArray = append(flattenedCastsSQLArray, fmt.Sprintf("%s AS %s", expr, quotedCol)) if slices.Contains(normalizedTableSchema.PrimaryKeyColumns, column.Name) { - primaryKeyColumnCasts[column.Name] = fmt.Sprintf("(_peerdb_data->>%s)::%s", stringCol, pgType) + primaryKeyColumnCasts[column.Name] = expr } } flattenedCastsSQL := strings.Join(flattenedCastsSQLArray, ",") @@ -128,19 +139,17 @@ func (n *normalizeStmtGenerator) generateMergeStatement( stringCol := QuoteLiteral(column.Name) quotedColumnNames[i] = quotedCol - pgType := qValueKindToPostgresType(genericColumnType) - if qvalue.QValueKind(genericColumnType).IsArray() { - flattenedCastsSQLArray = append(flattenedCastsSQLArray, - fmt.Sprintf("ARRAY(SELECT * FROM JSON_ARRAY_ELEMENTS_TEXT((_peerdb_data->>%s)::JSON))::%s AS %s", - stringCol, pgType, quotedCol)) + pgType := n.columnTypeToPg(normalizedTableSchema, genericColumnType) + var expr string + if normalizedTableSchema.System == protos.TypeSystem_Q && qvalue.QValueKind(genericColumnType).IsArray() { + expr = fmt.Sprintf("ARRAY(SELECT JSON_ARRAY_ELEMENTS_TEXT((_peerdb_data->>%s)::JSON))::%s", stringCol, pgType) } else { - flattenedCastsSQLArray = append(flattenedCastsSQLArray, fmt.Sprintf("(_peerdb_data->>%s)::%s AS %s", - stringCol, pgType, quotedCol)) + expr = fmt.Sprintf("(_peerdb_data->>%s)::%s", stringCol, pgType) } + flattenedCastsSQLArray = append(flattenedCastsSQLArray, fmt.Sprintf("%s AS %s", expr, quotedCol)) if slices.Contains(normalizedTableSchema.PrimaryKeyColumns, column.Name) { primaryKeyColumnCasts[column.Name] = fmt.Sprintf("(_peerdb_data->>%s)::%s", stringCol, pgType) - primaryKeySelectSQLArray = append(primaryKeySelectSQLArray, fmt.Sprintf("src.%s=dst.%s", - quotedCol, quotedCol)) + primaryKeySelectSQLArray = append(primaryKeySelectSQLArray, fmt.Sprintf("src.%s=dst.%s", quotedCol, quotedCol)) } } flattenedCastsSQL := strings.Join(flattenedCastsSQLArray, ",") diff --git a/flow/connectors/postgres/normalize_stmt_generator_test.go b/flow/connectors/postgres/normalize_stmt_generator_test.go index 160f193932..1bd355265b 100644 --- a/flow/connectors/postgres/normalize_stmt_generator_test.go +++ b/flow/connectors/postgres/normalize_stmt_generator_test.go @@ -17,7 +17,7 @@ func TestGenerateMergeUpdateStatement(t *testing.T) { THEN UPDATE SET "col1"=src."col1","col2"=src."col2","col3"=src."col3", "_peerdb_synced_at"=CURRENT_TIMESTAMP`, } - normalizeGen := &normalizeStmtGenerator{ + normalizeGen := normalizeStmtGenerator{ peerdbCols: &protos.PeerDBColumns{ SoftDelete: false, SyncedAtColName: "_peerdb_synced_at", @@ -48,7 +48,7 @@ func TestGenerateMergeUpdateStatement_WithSoftDelete(t *testing.T) { THEN UPDATE SET "col1"=src."col1","col2"=src."col2","col3"=src."col3", "_peerdb_synced_at"=CURRENT_TIMESTAMP,"_peerdb_soft_delete"=TRUE`, } - normalizeGen := &normalizeStmtGenerator{ + normalizeGen := normalizeStmtGenerator{ peerdbCols: &protos.PeerDBColumns{ SoftDelete: true, SyncedAtColName: "_peerdb_synced_at", @@ -81,7 +81,7 @@ func TestGenerateMergeUpdateStatement_WithUnchangedToastCols(t *testing.T) { `WHEN MATCHED AND src._peerdb_record_type!=2 AND _peerdb_unchanged_toast_columns='col3' THEN UPDATE SET "col1"=src."col1","col2"=src."col2","_peerdb_synced_at"=CURRENT_TIMESTAMP`, } - normalizeGen := &normalizeStmtGenerator{ + normalizeGen := normalizeStmtGenerator{ peerdbCols: &protos.PeerDBColumns{ SoftDelete: false, SyncedAtColName: "_peerdb_synced_at", @@ -124,7 +124,7 @@ func TestGenerateMergeUpdateStatement_WithUnchangedToastColsAndSoftDelete(t *tes `WHEN MATCHED AND src._peerdb_record_type=2 AND _peerdb_unchanged_toast_columns='col3' THEN UPDATE SET "col1"=src."col1","col2"=src."col2","_peerdb_synced_at"=CURRENT_TIMESTAMP,"_peerdb_soft_delete"=TRUE`, } - normalizeGen := &normalizeStmtGenerator{ + normalizeGen := normalizeStmtGenerator{ peerdbCols: &protos.PeerDBColumns{ SoftDelete: true, SyncedAtColName: "_peerdb_synced_at", diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index b438aec4ed..e88a736428 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -5,7 +5,6 @@ import ( "errors" "fmt" "log/slog" - "regexp" "strings" "sync" "sync/atomic" @@ -53,6 +52,7 @@ type ReplState struct { } func NewPostgresConnector(ctx context.Context, pgConfig *protos.PostgresConfig) (*PostgresConnector, error) { + logger := logger.LoggerFromCtx(ctx) connectionString := shared.GetPGConnectionString(pgConfig) // create a separate connection pool for non-replication queries as replication connections cannot @@ -69,11 +69,13 @@ func NewPostgresConnector(ctx context.Context, pgConfig *protos.PostgresConfig) tunnel, err := NewSSHTunnel(ctx, pgConfig.SshConfig) if err != nil { + logger.Error("failed to create ssh tunnel", slog.Any("error", err)) return nil, fmt.Errorf("failed to create ssh tunnel: %w", err) } conn, err := tunnel.NewPostgresConnFromConfig(ctx, connConfig) if err != nil { + logger.Error("failed to create connection", slog.Any("error", err)) return nil, fmt.Errorf("failed to create connection: %w", err) } @@ -83,6 +85,7 @@ func NewPostgresConnector(ctx context.Context, pgConfig *protos.PostgresConfig) customTypeMap, err := shared.GetCustomDataTypes(ctx, conn) if err != nil { + logger.Error("failed to get custom type map", slog.Any("error", err)) return nil, fmt.Errorf("failed to get custom type map: %w", err) } @@ -102,7 +105,7 @@ func NewPostgresConnector(ctx context.Context, pgConfig *protos.PostgresConfig) customTypesMapping: customTypeMap, metadataSchema: metadataSchema, hushWarnOID: make(map[uint32]struct{}), - logger: logger.LoggerFromCtx(ctx), + logger: logger, relationMessageMapping: make(model.RelationMessageMapping), }, nil } @@ -145,13 +148,13 @@ func (c *PostgresConnector) MaybeStartReplication( ctx context.Context, slotName string, publicationName string, - req *model.PullRecordsRequest, + lastOffset int64, ) error { - if c.replState != nil && (c.replState.Offset != req.LastOffset || + if c.replState != nil && (c.replState.Offset != lastOffset || c.replState.Slot != slotName || c.replState.Publication != publicationName) { msg := fmt.Sprintf("replState changed, reset connector. slot name: old=%s new=%s, publication: old=%s new=%s, offset: old=%d new=%d", - c.replState.Slot, slotName, c.replState.Publication, publicationName, c.replState.Offset, req.LastOffset, + c.replState.Slot, slotName, c.replState.Publication, publicationName, c.replState.Offset, lastOffset, ) c.logger.Info(msg) return temporal.NewNonRetryableApplicationError(msg, "desync", nil) @@ -164,9 +167,9 @@ func (c *PostgresConnector) MaybeStartReplication( } var startLSN pglogrepl.LSN - if req.LastOffset > 0 { - c.logger.Info("starting replication from last sync state", slog.Int64("last checkpoint", req.LastOffset)) - startLSN = pglogrepl.LSN(req.LastOffset + 1) + if lastOffset > 0 { + c.logger.Info("starting replication from last sync state", slog.Int64("last checkpoint", lastOffset)) + startLSN = pglogrepl.LSN(lastOffset + 1) } opts := startReplicationOpts{ @@ -184,10 +187,10 @@ func (c *PostgresConnector) MaybeStartReplication( c.replState = &ReplState{ Slot: slotName, Publication: publicationName, - Offset: req.LastOffset, + Offset: lastOffset, LastOffset: atomic.Int64{}, } - c.replState.LastOffset.Store(req.LastOffset) + c.replState.LastOffset.Store(lastOffset) } return nil } @@ -307,8 +310,30 @@ func (c *PostgresConnector) SetLastOffset(ctx context.Context, jobName string, l return nil } +func (c *PostgresConnector) PullRecords( + ctx context.Context, + catalogPool *pgxpool.Pool, + req *model.PullRecordsRequest[model.RecordItems], +) error { + return pullCore(ctx, c, catalogPool, req, qProcessor{}) +} + +func (c *PostgresConnector) PullPg( + ctx context.Context, + catalogPool *pgxpool.Pool, + req *model.PullRecordsRequest[model.PgItems], +) error { + return pullCore(ctx, c, catalogPool, req, pgProcessor{}) +} + // PullRecords pulls records from the source. -func (c *PostgresConnector) PullRecords(ctx context.Context, catalogPool *pgxpool.Pool, req *model.PullRecordsRequest) error { +func pullCore[Items model.Items]( + ctx context.Context, + c *PostgresConnector, + catalogPool *pgxpool.Pool, + req *model.PullRecordsRequest[Items], + processor replProcessor[Items], +) error { defer func() { req.RecordStream.Close() if c.replState != nil { @@ -353,8 +378,7 @@ func (c *PostgresConnector) PullRecords(ctx context.Context, catalogPool *pgxpoo c.replLock.Lock() defer c.replLock.Unlock() - err = c.MaybeStartReplication(ctx, slotName, publicationName, req) - if err != nil { + if err := c.MaybeStartReplication(ctx, slotName, publicationName, req.LastOffset); err != nil { c.logger.Error("error starting replication", slog.Any("error", err)) return err } @@ -371,8 +395,7 @@ func (c *PostgresConnector) PullRecords(ctx context.Context, catalogPool *pgxpoo RelationMessageMapping: c.relationMessageMapping, }) - err = cdc.PullRecords(ctx, req) - if err != nil { + if err := PullCdcRecords(ctx, cdc, req, processor); err != nil { c.logger.Error("error pulling records", slog.Any("error", err)) return err } @@ -398,8 +421,20 @@ func (c *PostgresConnector) UpdateReplStateLastOffset(lastOffset int64) { } } -// SyncRecords pushes records to the destination. -func (c *PostgresConnector) SyncRecords(ctx context.Context, req *model.SyncRecordsRequest) (*model.SyncResponse, error) { +func (c *PostgresConnector) SyncRecords(ctx context.Context, req *model.SyncRecordsRequest[model.RecordItems]) (*model.SyncResponse, error) { + return syncRecordsCore(ctx, c, req) +} + +func (c *PostgresConnector) SyncPg(ctx context.Context, req *model.SyncRecordsRequest[model.PgItems]) (*model.SyncResponse, error) { + return syncRecordsCore(ctx, c, req) +} + +// syncRecordsCore pushes records to the destination. +func syncRecordsCore[Items model.Items]( + ctx context.Context, + c *PostgresConnector, + req *model.SyncRecordsRequest[Items], +) (*model.SyncResponse, error) { rawTableIdentifier := getRawTableIdentifier(req.FlowJobName) c.logger.Info(fmt.Sprintf("pushing records to Postgres table %s via COPY", rawTableIdentifier)) @@ -413,7 +448,7 @@ func (c *PostgresConnector) SyncRecords(ctx context.Context, req *model.SyncReco } else { var row []any switch typedRecord := record.(type) { - case *model.InsertRecord: + case *model.InsertRecord[Items]: itemsJSON, err := typedRecord.Items.ToJSONWithOptions(model.ToJSONOptions{ UnnestColumns: nil, HStoreAsJSON: false, @@ -433,7 +468,7 @@ func (c *PostgresConnector) SyncRecords(ctx context.Context, req *model.SyncReco "", } - case *model.UpdateRecord: + case *model.UpdateRecord[Items]: newItemsJSON, err := typedRecord.NewItems.ToJSONWithOptions(model.ToJSONOptions{ UnnestColumns: nil, HStoreAsJSON: false, @@ -460,7 +495,7 @@ func (c *PostgresConnector) SyncRecords(ctx context.Context, req *model.SyncReco utils.KeysToString(typedRecord.UnchangedToastColumns), } - case *model.DeleteRecord: + case *model.DeleteRecord[Items]: itemsJSON, err := typedRecord.Items.ToJSONWithOptions(model.ToJSONOptions{ UnnestColumns: nil, HStoreAsJSON: false, @@ -494,12 +529,7 @@ func (c *PostgresConnector) SyncRecords(ctx context.Context, req *model.SyncReco if err != nil { return nil, fmt.Errorf("error starting transaction for syncing records: %w", err) } - defer func() { - deferErr := syncRecordsTx.Rollback(ctx) - if deferErr != pgx.ErrTxClosed && deferErr != nil { - c.logger.Error("error rolling back transaction for syncing records", slog.Any("error", err)) - } - }() + defer shared.RollbackTx(syncRecordsTx, c.logger) syncedRecordsCount, err := syncRecordsTx.CopyFrom(ctx, pgx.Identifier{c.metadataSchema, rawTableIdentifier}, []string{ @@ -544,7 +574,10 @@ func (c *PostgresConnector) SyncRecords(ctx context.Context, req *model.SyncReco }, nil } -func (c *PostgresConnector) NormalizeRecords(ctx context.Context, req *model.NormalizeRecordsRequest) (*model.NormalizeResponse, error) { +func (c *PostgresConnector) NormalizeRecords( + ctx context.Context, + req *model.NormalizeRecordsRequest, +) (*model.NormalizeResponse, error) { rawTableIdentifier := getRawTableIdentifier(req.FlowJobName) jobMetadataExists, err := c.jobMetadataExists(ctx, req.FlowJobName) @@ -580,7 +613,7 @@ func (c *PostgresConnector) NormalizeRecords(ctx context.Context, req *model.Nor if err != nil { return nil, err } - unchangedToastColsMap, err := c.getTableNametoUnchangedCols(ctx, req.FlowJobName, + unchangedToastColumnsMap, err := c.getTableNametoUnchangedCols(ctx, req.FlowJobName, req.SyncBatchID, normBatchID) if err != nil { return nil, err @@ -590,23 +623,18 @@ func (c *PostgresConnector) NormalizeRecords(ctx context.Context, req *model.Nor if err != nil { return nil, fmt.Errorf("error starting transaction for normalizing records: %w", err) } - defer func() { - deferErr := normalizeRecordsTx.Rollback(ctx) - if deferErr != pgx.ErrTxClosed && deferErr != nil { - c.logger.Error("error rolling back transaction for normalizing records", slog.Any("error", err)) - } - }() + defer shared.RollbackTx(normalizeRecordsTx, c.logger) pgversion, err := c.MajorVersion(ctx) if err != nil { return nil, err } totalRowsAffected := 0 - normalizeStmtGen := &normalizeStmtGenerator{ + normalizeStmtGen := normalizeStmtGenerator{ Logger: c.logger, rawTableName: rawTableIdentifier, tableSchemaMapping: req.TableNameSchemaMapping, - unchangedToastColumnsMap: unchangedToastColsMap, + unchangedToastColumnsMap: unchangedToastColumnsMap, peerdbCols: &protos.PeerDBColumns{ SoftDeleteColName: req.SoftDeleteColName, SyncedAtColName: req.SyncedAtColName, @@ -615,6 +643,7 @@ func (c *PostgresConnector) NormalizeRecords(ctx context.Context, req *model.Nor supportsMerge: pgversion >= shared.POSTGRES_15, metadataSchema: c.metadataSchema, } + for _, destinationTableName := range destinationTableNames { normalizeStatements := normalizeStmtGen.generateNormalizeStatements(destinationTableName) for _, normalizeStatement := range normalizeStatements { @@ -663,12 +692,7 @@ func (c *PostgresConnector) CreateRawTable(ctx context.Context, req *protos.Crea if err != nil { return nil, fmt.Errorf("error starting transaction for creating raw table: %w", err) } - defer func() { - deferErr := createRawTableTx.Rollback(ctx) - if deferErr != pgx.ErrTxClosed && deferErr != nil { - c.logger.Error("error rolling back transaction for creating raw table.", slog.Any("error", err)) - } - }() + defer shared.RollbackTx(createRawTableTx, c.logger) _, err = createRawTableTx.Exec(ctx, fmt.Sprintf(createRawTableSQL, c.metadataSchema, rawTableIdentifier)) if err != nil { @@ -693,19 +717,17 @@ func (c *PostgresConnector) CreateRawTable(ctx context.Context, req *protos.Crea return nil, nil } -// GetTableSchema returns the schema for a table, implementing the Connector interface. func (c *PostgresConnector) GetTableSchema( ctx context.Context, req *protos.GetTableSchemaBatchInput, ) (*protos.GetTableSchemaBatchOutput, error) { res := make(map[string]*protos.TableSchema) for _, tableName := range req.TableIdentifiers { - tableSchema, err := c.getTableSchemaForTable(ctx, tableName) + tableSchema, err := c.getTableSchemaForTable(ctx, tableName, req.System) if err != nil { return nil, err } res[tableName] = tableSchema - utils.RecordHeartbeat(ctx, "fetched schema for table "+tableName) c.logger.Info("fetched schema for table " + tableName) } @@ -717,6 +739,7 @@ func (c *PostgresConnector) GetTableSchema( func (c *PostgresConnector) getTableSchemaForTable( ctx context.Context, tableName string, + system protos.TypeSystem, ) (*protos.TableSchema, error) { schemaTable, err := utils.ParseSchemaTable(tableName) if err != nil { @@ -745,20 +768,34 @@ func (c *PostgresConnector) getTableSchemaForTable( columnNames := make([]string, 0, len(fields)) columns := make([]*protos.FieldDescription, 0, len(fields)) for _, fieldDescription := range fields { - genericColType := c.postgresOIDToQValueKind(fieldDescription.DataTypeOID) - if genericColType == qvalue.QValueKindInvalid { - typeName, ok := c.customTypesMapping[fieldDescription.DataTypeOID] - if ok { - genericColType = customTypeToQKind(typeName) - } else { - genericColType = qvalue.QValueKindString + var colType string + switch system { + case protos.TypeSystem_PG: + colType = c.postgresOIDToName(fieldDescription.DataTypeOID) + if colType == "" { + typeName, ok := c.customTypesMapping[fieldDescription.DataTypeOID] + if !ok { + return nil, fmt.Errorf("error getting type name for %d", fieldDescription.DataTypeOID) + } + colType = typeName + } + case protos.TypeSystem_Q: + qColType := c.postgresOIDToQValueKind(fieldDescription.DataTypeOID) + if qColType == qvalue.QValueKindInvalid { + typeName, ok := c.customTypesMapping[fieldDescription.DataTypeOID] + if ok { + qColType = customTypeToQKind(typeName) + } else { + qColType = qvalue.QValueKindString + } } + colType = string(qColType) } columnNames = append(columnNames, fieldDescription.Name) columns = append(columns, &protos.FieldDescription{ Name: fieldDescription.Name, - Type: string(genericColType), + Type: colType, TypeModifier: fieldDescription.TypeModifier, }) } @@ -776,6 +813,7 @@ func (c *PostgresConnector) getTableSchemaForTable( PrimaryKeyColumns: pKeyCols, IsReplicaIdentityFull: replicaIdentityType == ReplicaIdentityFull, Columns: columns, + System: system, }, nil } @@ -785,10 +823,7 @@ func (c *PostgresConnector) StartSetupNormalizedTables(ctx context.Context) (any } func (c *PostgresConnector) CleanupSetupNormalizedTables(ctx context.Context, tx any) { - err := tx.(pgx.Tx).Rollback(ctx) - if err != pgx.ErrTxClosed && err != nil { - c.logger.Error("error rolling back transaction for creating raw table", slog.Any("error", err)) - } + shared.RollbackTx(tx.(pgx.Tx), c.logger) } func (c *PostgresConnector) FinishSetupNormalizedTables(ctx context.Context, tx any) error { @@ -828,7 +863,7 @@ func (c *PostgresConnector) SetupNormalizedTable( return false, nil } -// ReplayTableSchemaDelta changes a destination table to match the schema at source +// replayTableSchemaDeltaCore changes a destination table to match the schema at source // This could involve adding or dropping multiple columns. func (c *PostgresConnector) ReplayTableSchemaDeltas( ctx context.Context, @@ -845,12 +880,7 @@ func (c *PostgresConnector) ReplayTableSchemaDeltas( return fmt.Errorf("error starting transaction for schema modification: %w", err) } - defer func() { - deferErr := tableSchemaModifyTx.Rollback(ctx) - if deferErr != pgx.ErrTxClosed && deferErr != nil { - c.logger.Error("error rolling back transaction for table schema modification", slog.Any("error", err)) - } - }() + defer shared.RollbackTx(tableSchemaModifyTx, c.logger) for _, schemaDelta := range schemaDeltas { if schemaDelta == nil || len(schemaDelta.AddedColumns) == 0 { @@ -858,28 +888,28 @@ func (c *PostgresConnector) ReplayTableSchemaDeltas( } for _, addedColumn := range schemaDelta.AddedColumns { + columnType := addedColumn.Type + if schemaDelta.System == protos.TypeSystem_Q { + columnType = qValueKindToPostgresType(columnType) + } _, err = tableSchemaModifyTx.Exec(ctx, fmt.Sprintf( "ALTER TABLE %s ADD COLUMN IF NOT EXISTS %s %s", - schemaDelta.DstTableName, QuoteIdentifier(addedColumn.ColumnName), - qValueKindToPostgresType(addedColumn.ColumnType))) + schemaDelta.DstTableName, QuoteIdentifier(addedColumn.Name), columnType)) if err != nil { - return fmt.Errorf("failed to add column %s for table %s: %w", addedColumn.ColumnName, + return fmt.Errorf("failed to add column %s for table %s: %w", addedColumn.Name, schemaDelta.DstTableName, err) } c.logger.Info(fmt.Sprintf("[schema delta replay] added column %s with data type %s", - addedColumn.ColumnName, addedColumn.ColumnType), + addedColumn.Name, addedColumn.Type), slog.String("srcTableName", schemaDelta.SrcTableName), slog.String("dstTableName", schemaDelta.DstTableName), ) } } - err = tableSchemaModifyTx.Commit(ctx) - if err != nil { - return fmt.Errorf("failed to commit transaction for table schema modification: %w", - err) + if err := tableSchemaModifyTx.Commit(ctx); err != nil { + return fmt.Errorf("failed to commit transaction for table schema modification: %w", err) } - return nil } @@ -987,10 +1017,8 @@ func (c *PostgresConnector) FinishExport(tx any) error { // SetupReplication sets up replication for the source connector. func (c *PostgresConnector) SetupReplication(ctx context.Context, signal SlotSignal, req *protos.SetupReplicationInput) error { - // ensure that the flowjob name is [a-z0-9_] only - reg := regexp.MustCompile(`^[a-z0-9_]+$`) - if !reg.MatchString(req.FlowJobName) { - return fmt.Errorf("invalid flow job name: `%s`, it should be [a-z0-9_]+", req.FlowJobName) + if !shared.IsValidReplicationName(req.FlowJobName) { + return fmt.Errorf("invalid flow job name: `%s`, it should be ^[a-z_][a-z0-9_]*$", req.FlowJobName) } // Slotname would be the job name prefixed with "peerflow_slot_" @@ -1052,12 +1080,7 @@ func (c *PostgresConnector) SyncFlowCleanup(ctx context.Context, jobName string) if err != nil { return fmt.Errorf("unable to begin transaction for sync flow cleanup: %w", err) } - defer func() { - deferErr := syncFlowCleanupTx.Rollback(ctx) - if deferErr != pgx.ErrTxClosed && deferErr != nil { - c.logger.Error("error while rolling back transaction for flow cleanup", slog.Any("error", deferErr)) - } - }() + defer shared.RollbackTx(syncFlowCleanupTx, c.logger) _, err = syncFlowCleanupTx.Exec(ctx, fmt.Sprintf(dropTableIfExistsSQL, c.metadataSchema, getRawTableIdentifier(jobName))) diff --git a/flow/connectors/postgres/postgres_schema_delta_test.go b/flow/connectors/postgres/postgres_schema_delta_test.go index c71c77c25c..ffd32572d1 100644 --- a/flow/connectors/postgres/postgres_schema_delta_test.go +++ b/flow/connectors/postgres/postgres_schema_delta_test.go @@ -61,20 +61,25 @@ func (s PostgresSchemaDeltaTestSuite) TestSimpleAddColumn() { err = s.connector.ReplayTableSchemaDeltas(context.Background(), "schema_delta_flow", []*protos.TableSchemaDelta{{ SrcTableName: tableName, DstTableName: tableName, - AddedColumns: []*protos.DeltaAddedColumn{{ - ColumnName: "hi", - ColumnType: string(qvalue.QValueKindInt64), - }}, + AddedColumns: []*protos.FieldDescription{ + { + Name: "hi", + Type: string(qvalue.QValueKindInt64), + TypeModifier: -1, + }, + }, }}) require.NoError(s.t, err) output, err := s.connector.GetTableSchema(context.Background(), &protos.GetTableSchemaBatchInput{ TableIdentifiers: []string{tableName}, + System: protos.TypeSystem_Q, }) require.NoError(s.t, err) require.Equal(s.t, &protos.TableSchema{ TableIdentifier: tableName, PrimaryKeyColumns: []string{"id"}, + System: protos.TypeSystem_Q, Columns: []*protos.FieldDescription{ { Name: "id", @@ -100,14 +105,17 @@ func (s PostgresSchemaDeltaTestSuite) TestAddAllColumnTypes() { TableIdentifier: tableName, PrimaryKeyColumns: []string{"id"}, Columns: AddAllColumnTypesFields, + System: protos.TypeSystem_Q, } - addedColumns := make([]*protos.DeltaAddedColumn, 0) + addedColumns := make([]*protos.FieldDescription, 0) for _, column := range expectedTableSchema.Columns { if column.Name != "id" { - addedColumns = append(addedColumns, &protos.DeltaAddedColumn{ - ColumnName: column.Name, - ColumnType: column.Type, - }) + addedColumns = append(addedColumns, &protos.FieldDescription{ + Name: column.Name, + Type: column.Type, + TypeModifier: -1, + }, + ) } } @@ -120,6 +128,7 @@ func (s PostgresSchemaDeltaTestSuite) TestAddAllColumnTypes() { output, err := s.connector.GetTableSchema(context.Background(), &protos.GetTableSchemaBatchInput{ TableIdentifiers: []string{tableName}, + System: protos.TypeSystem_Q, }) require.NoError(s.t, err) require.Equal(s.t, expectedTableSchema, output.TableNameSchemaMapping[tableName]) @@ -135,14 +144,17 @@ func (s PostgresSchemaDeltaTestSuite) TestAddTrickyColumnNames() { TableIdentifier: tableName, PrimaryKeyColumns: []string{"id"}, Columns: TrickyFields, + System: protos.TypeSystem_Q, } - addedColumns := make([]*protos.DeltaAddedColumn, 0) + addedColumns := make([]*protos.FieldDescription, 0) for _, column := range expectedTableSchema.Columns { if column.Name != "id" { - addedColumns = append(addedColumns, &protos.DeltaAddedColumn{ - ColumnName: column.Name, - ColumnType: column.Type, - }) + addedColumns = append(addedColumns, &protos.FieldDescription{ + Name: column.Name, + Type: column.Type, + TypeModifier: -1, + }, + ) } } @@ -155,6 +167,7 @@ func (s PostgresSchemaDeltaTestSuite) TestAddTrickyColumnNames() { output, err := s.connector.GetTableSchema(context.Background(), &protos.GetTableSchemaBatchInput{ TableIdentifiers: []string{tableName}, + System: protos.TypeSystem_Q, }) require.NoError(s.t, err) require.Equal(s.t, expectedTableSchema, output.TableNameSchemaMapping[tableName]) @@ -170,14 +183,17 @@ func (s PostgresSchemaDeltaTestSuite) TestAddDropWhitespaceColumnNames() { TableIdentifier: tableName, PrimaryKeyColumns: []string{" "}, Columns: WhitespaceFields, + System: protos.TypeSystem_Q, } - addedColumns := make([]*protos.DeltaAddedColumn, 0) + addedColumns := make([]*protos.FieldDescription, 0) for _, column := range expectedTableSchema.Columns { if column.Name != " " { - addedColumns = append(addedColumns, &protos.DeltaAddedColumn{ - ColumnName: column.Name, - ColumnType: column.Type, - }) + addedColumns = append(addedColumns, &protos.FieldDescription{ + Name: column.Name, + Type: column.Type, + TypeModifier: -1, + }, + ) } } @@ -190,6 +206,7 @@ func (s PostgresSchemaDeltaTestSuite) TestAddDropWhitespaceColumnNames() { output, err := s.connector.GetTableSchema(context.Background(), &protos.GetTableSchemaBatchInput{ TableIdentifiers: []string{tableName}, + System: protos.TypeSystem_Q, }) require.NoError(s.t, err) require.Equal(s.t, expectedTableSchema, output.TableNameSchemaMapping[tableName]) diff --git a/flow/connectors/postgres/qrep.go b/flow/connectors/postgres/qrep.go index 7ce588d006..8ec936d2ea 100644 --- a/flow/connectors/postgres/qrep.go +++ b/flow/connectors/postgres/qrep.go @@ -39,21 +39,16 @@ func (c *PostgresConnector) GetQRepPartitions( } // begin a transaction - tx, err := c.conn.BeginTx(ctx, pgx.TxOptions{ + getPartitionsTx, err := c.conn.BeginTx(ctx, pgx.TxOptions{ AccessMode: pgx.ReadOnly, IsoLevel: pgx.RepeatableRead, }) if err != nil { return nil, fmt.Errorf("failed to begin transaction: %w", err) } - defer func() { - deferErr := tx.Rollback(ctx) - if deferErr != pgx.ErrTxClosed && deferErr != nil { - c.logger.Error("error rolling back transaction for get partitions", slog.Any("error", deferErr)) - } - }() + defer shared.RollbackTx(getPartitionsTx, c.logger) - err = c.setTransactionSnapshot(ctx, tx) + err = c.setTransactionSnapshot(ctx, getPartitionsTx) if err != nil { return nil, fmt.Errorf("failed to set transaction snapshot: %w", err) } @@ -66,7 +61,7 @@ func (c *PostgresConnector) GetQRepPartitions( // log.Warnf("failed to lock table %s: %v", config.WatermarkTable, err) // } - return c.getNumRowsPartitions(ctx, tx, config, last) + return c.getNumRowsPartitions(ctx, getPartitionsTx, config, last) } func (c *PostgresConnector) setTransactionSnapshot(ctx context.Context, tx pgx.Tx) error { @@ -128,10 +123,7 @@ func (c *PostgresConnector) getNumRowsPartitions( } // Calculate the number of partitions - numPartitions := totalRows.Int64 / numRowsPerPartition - if totalRows.Int64%numRowsPerPartition != 0 { - numPartitions++ - } + numPartitions := shared.DivCeil(totalRows.Int64, numRowsPerPartition) c.logger.Info(fmt.Sprintf("total rows: %d, num partitions: %d, num rows per partition: %d", totalRows.Int64, numPartitions, numRowsPerPartition)) @@ -284,18 +276,13 @@ func (c *PostgresConnector) CheckForUpdatedMaxValue( config *protos.QRepConfig, last *protos.QRepPartition, ) (bool, error) { - tx, err := c.conn.Begin(ctx) + checkTx, err := c.conn.Begin(ctx) if err != nil { return false, fmt.Errorf("unable to begin transaction for getting max value: %w", err) } - defer func() { - deferErr := tx.Rollback(ctx) - if deferErr != pgx.ErrTxClosed && deferErr != nil { - c.logger.Error("error rolling back transaction for getting max value", "error", err) - } - }() + defer shared.RollbackTx(checkTx, c.logger) - _, maxValue, err := c.getMinMaxValues(ctx, tx, config, last) + _, maxValue, err := c.getMinMaxValues(ctx, checkTx, config, last) if err != nil { return false, fmt.Errorf("error while getting min and max values: %w", err) } diff --git a/flow/connectors/postgres/qrep_query_executor.go b/flow/connectors/postgres/qrep_query_executor.go index c77eb3b054..05c4d5b71a 100644 --- a/flow/connectors/postgres/qrep_query_executor.go +++ b/flow/connectors/postgres/qrep_query_executor.go @@ -61,7 +61,7 @@ func (qe *QRepQueryExecutor) executeQueryInTx(ctx context.Context, tx pgx.Tx, cu } // FieldDescriptionsToSchema converts a slice of pgconn.FieldDescription to a QRecordSchema. -func (qe *QRepQueryExecutor) fieldDescriptionsToSchema(fds []pgconn.FieldDescription) *qvalue.QRecordSchema { +func (qe *QRepQueryExecutor) fieldDescriptionsToSchema(fds []pgconn.FieldDescription) qvalue.QRecordSchema { qfields := make([]qvalue.QField, len(fds)) for i, fd := range fds { cname := fd.Name @@ -120,8 +120,8 @@ func (qe *QRepQueryExecutor) ProcessRows( } batch := &model.QRecordBatch{ - Records: records, Schema: qe.fieldDescriptionsToSchema(fieldDescriptions), + Records: records, } qe.logger.Info(fmt.Sprintf("[postgres] pulled %d records", len(batch.Records))) @@ -148,16 +148,11 @@ func (qe *QRepQueryExecutor) processRowsStream( record, err := qe.mapRowToQRecord(rows, fieldDescriptions) if err != nil { qe.logger.Error("[pg_query_executor] failed to map row to QRecord", slog.Any("error", err)) - stream.Records <- model.QRecordOrError{ - Err: fmt.Errorf("failed to map row to QRecord: %w", err), - } + stream.Close(fmt.Errorf("failed to map row to QRecord: %w", err)) return 0, fmt.Errorf("failed to map row to QRecord: %w", err) } - stream.Records <- model.QRecordOrError{ - Record: record, - Err: nil, - } + stream.Records <- record if numRows%heartBeatNumRows == 0 { qe.logger.Info("processing row stream", slog.String("cursor", cursorName), slog.Int("records", numRows)) @@ -180,9 +175,7 @@ func (qe *QRepQueryExecutor) processFetchedRows( ) (int, error) { rows, err := qe.executeQueryInTx(ctx, tx, cursorName, fetchSize) if err != nil { - stream.Records <- model.QRecordOrError{ - Err: err, - } + stream.Close(err) qe.logger.Error("[pg_query_executor] failed to execute query in tx", slog.Any("error", err), slog.String("query", query)) return 0, fmt.Errorf("[pg_query_executor] failed to execute query in tx: %w", err) @@ -192,8 +185,7 @@ func (qe *QRepQueryExecutor) processFetchedRows( fieldDescriptions := rows.FieldDescriptions() if !stream.IsSchemaSet() { - schema := qe.fieldDescriptionsToSchema(fieldDescriptions) - _ = stream.SetSchema(schema) + stream.SetSchema(qe.fieldDescriptionsToSchema(fieldDescriptions)) } numRows, err := qe.processRowsStream(ctx, cursorName, stream, rows, fieldDescriptions) @@ -202,10 +194,8 @@ func (qe *QRepQueryExecutor) processFetchedRows( return 0, fmt.Errorf("failed to process rows: %w", err) } - if rows.Err() != nil { - stream.Records <- model.QRecordOrError{ - Err: rows.Err(), - } + if err := rows.Err(); err != nil { + stream.Close(err) qe.logger.Error("[pg_query_executor] row iteration failed", slog.String("query", query), slog.Any("error", rows.Err())) return 0, fmt.Errorf("[pg_query_executor] row iteration failed '%s': %w", query, rows.Err()) @@ -236,25 +226,20 @@ func (qe *QRepQueryExecutor) ExecuteAndProcessQuery( select { case err := <-errors: return nil, err - case schema := <-stream.SchemaChan(): - if schema.Err != nil { - qe.logger.Error("[pg_query_executor] failed to get schema from stream", slog.Any("error", schema.Err)) - <-errors - return nil, fmt.Errorf("failed to get schema from stream: %w", schema.Err) - } + case <-stream.SchemaChan(): batch := &model.QRecordBatch{ + Schema: stream.Schema(), Records: make([][]qvalue.QValue, 0), - Schema: schema.Schema, } for record := range stream.Records { - if record.Err == nil { - batch.Records = append(batch.Records, record.Record) - } else { - <-errors - return nil, fmt.Errorf("[pg] failed to get record from stream: %w", record.Err) - } + batch.Records = append(batch.Records, record) + } + if err := <-errors; err != nil { + return nil, err + } + if err := stream.Err(); err != nil { + return nil, fmt.Errorf("[pg] failed to get record from stream: %w", err) } - <-errors return batch, nil } } @@ -266,7 +251,7 @@ func (qe *QRepQueryExecutor) ExecuteAndProcessQueryStream( args ...interface{}, ) (int, error) { qe.logger.Info("Executing and processing query stream", slog.String("query", query)) - defer close(stream.Records) + defer stream.Close(nil) tx, err := qe.conn.BeginTx(ctx, pgx.TxOptions{ AccessMode: pgx.ReadOnly, @@ -277,8 +262,7 @@ func (qe *QRepQueryExecutor) ExecuteAndProcessQueryStream( return 0, fmt.Errorf("[pg_query_executor] failed to begin transaction: %w", err) } - totalRecordsFetched, err := qe.ExecuteAndProcessQueryStreamWithTx(ctx, tx, stream, query, args...) - return totalRecordsFetched, err + return qe.ExecuteAndProcessQueryStreamWithTx(ctx, tx, stream, query, args...) } func (qe *QRepQueryExecutor) ExecuteAndProcessQueryStreamGettingCurrentSnapshotXmin( @@ -289,7 +273,7 @@ func (qe *QRepQueryExecutor) ExecuteAndProcessQueryStreamGettingCurrentSnapshotX ) (int, int64, error) { var currentSnapshotXmin pgtype.Int8 qe.logger.Info("Executing and processing query stream", slog.String("query", query)) - defer close(stream.Records) + defer stream.Close(nil) tx, err := qe.conn.BeginTx(ctx, pgx.TxOptions{ AccessMode: pgx.ReadOnly, @@ -319,32 +303,25 @@ func (qe *QRepQueryExecutor) ExecuteAndProcessQueryStreamWithTx( ) (int, error) { var err error - defer func() { - err := tx.Rollback(ctx) - if err != nil && err != pgx.ErrTxClosed { - qe.logger.Error("[pg_query_executor] failed to rollback transaction", slog.Any("error", err)) - } - }() + defer shared.RollbackTx(tx, qe.logger) if qe.snapshot != "" { _, err = tx.Exec(ctx, "SET TRANSACTION SNAPSHOT "+QuoteLiteral(qe.snapshot)) if err != nil { - stream.Records <- model.QRecordOrError{ - Err: fmt.Errorf("failed to set snapshot: %w", err), - } qe.logger.Error("[pg_query_executor] failed to set snapshot", slog.Any("error", err), slog.String("query", query)) - return 0, fmt.Errorf("[pg_query_executor] failed to set snapshot: %w", err) + err := fmt.Errorf("[pg_query_executor] failed to set snapshot: %w", err) + stream.Close(err) + return 0, err } } randomUint, err := shared.RandomUInt64() if err != nil { qe.logger.Error("[pg_query_executor] failed to generate random uint", slog.Any("error", err)) - stream.Records <- model.QRecordOrError{ - Err: fmt.Errorf("failed to generate random uint: %w", err), - } - return 0, fmt.Errorf("[pg_query_executor] failed to generate random uint: %w", err) + err = fmt.Errorf("[pg_query_executor] failed to generate random uint: %w", err) + stream.Close(err) + return 0, err } cursorName := fmt.Sprintf("peerdb_cursor_%d", randomUint) @@ -353,12 +330,11 @@ func (qe *QRepQueryExecutor) ExecuteAndProcessQueryStreamWithTx( qe.logger.Info(fmt.Sprintf("[pg_query_executor] executing cursor declaration for %v with args %v", cursorQuery, args)) _, err = tx.Exec(ctx, cursorQuery, args...) if err != nil { - stream.Records <- model.QRecordOrError{ - Err: fmt.Errorf("failed to declare cursor: %w", err), - } qe.logger.Info("[pg_query_executor] failed to declare cursor", slog.String("cursorQuery", cursorQuery), slog.Any("error", err)) - return 0, fmt.Errorf("[pg_query_executor] failed to declare cursor: %w", err) + err = fmt.Errorf("[pg_query_executor] failed to declare cursor: %w", err) + stream.Close(err) + return 0, err } qe.logger.Info(fmt.Sprintf("[pg_query_executor] declared cursor '%s' for query '%s'", cursorName, query)) @@ -383,10 +359,9 @@ func (qe *QRepQueryExecutor) ExecuteAndProcessQueryStreamWithTx( err = tx.Commit(ctx) if err != nil { qe.logger.Error("[pg_query_executor] failed to commit transaction", slog.Any("error", err)) - stream.Records <- model.QRecordOrError{ - Err: fmt.Errorf("failed to commit transaction: %w", err), - } - return 0, fmt.Errorf("[pg_query_executor] failed to commit transaction: %w", err) + err = fmt.Errorf("[pg_query_executor] failed to commit transaction: %w", err) + stream.Close(err) + return 0, err } qe.logger.Info(fmt.Sprintf("[pg_query_executor] committed transaction for query '%s', rows = %d", diff --git a/flow/connectors/postgres/qrep_sql_sync.go b/flow/connectors/postgres/qrep_sql_sync.go index 9f7d4b4e48..431c3b0570 100644 --- a/flow/connectors/postgres/qrep_sql_sync.go +++ b/flow/connectors/postgres/qrep_sql_sync.go @@ -46,11 +46,7 @@ func (s *QRepStagingTableSync) SyncQRepRecords( ) partitionID := partition.PartitionId startTime := time.Now() - schema, err := stream.Schema() - if err != nil { - logger.LoggerFromCtx(ctx).Error("failed to get schema from stream", slog.Any("error", err), syncLog) - return 0, fmt.Errorf("failed to get schema from stream: %w", err) - } + schema := stream.Schema() txConfig := s.connector.conn.Config() txConn, err := pgx.ConnectConfig(ctx, txConfig) @@ -97,7 +93,7 @@ func (s *QRepStagingTableSync) SyncQRepRecords( // Perform the COPY FROM operation numRowsSynced, err = tx.CopyFrom( - context.Background(), + ctx, pgx.Identifier{dstTableName.Schema, dstTableName.Table}, schema.GetColumnNames(), copySource, @@ -113,7 +109,7 @@ func (s *QRepStagingTableSync) SyncQRepRecords( QuoteIdentifier(syncedAtCol), QuoteIdentifier(syncedAtCol), ) - _, err = tx.Exec(context.Background(), updateSyncedAtStmt) + _, err = tx.Exec(ctx, updateSyncedAtStmt) if err != nil { return -1, fmt.Errorf("failed to update synced_at column: %w", err) } @@ -132,14 +128,14 @@ func (s *QRepStagingTableSync) SyncQRepRecords( s.connector.logger.Info(fmt.Sprintf("Creating staging table %s - '%s'", stagingTableName, createStagingTableStmt), syncLog) - _, err = tx.Exec(context.Background(), createStagingTableStmt) + _, err = tx.Exec(ctx, createStagingTableStmt) if err != nil { return -1, fmt.Errorf("failed to create staging table: %w", err) } // Step 2.2: Insert records into the staging table numRowsSynced, err = tx.CopyFrom( - context.Background(), + ctx, stagingTableIdentifier, schema.GetColumnNames(), copySource, @@ -182,7 +178,7 @@ func (s *QRepStagingTableSync) SyncQRepRecords( setClause, ) s.connector.logger.Info("Performing upsert operation", slog.String("upsertStmt", upsertStmt), syncLog) - res, err := tx.Exec(context.Background(), upsertStmt) + res, err := tx.Exec(ctx, upsertStmt) if err != nil { return -1, fmt.Errorf("failed to perform upsert operation: %w", err) } @@ -195,7 +191,7 @@ func (s *QRepStagingTableSync) SyncQRepRecords( stagingTableIdentifier.Sanitize(), ) s.connector.logger.Info("Dropping staging table", slog.String("stagingTable", stagingTableName), syncLog) - _, err = tx.Exec(context.Background(), dropStagingTableStmt) + _, err = tx.Exec(ctx, dropStagingTableStmt) if err != nil { return -1, fmt.Errorf("failed to drop staging table: %w", err) } @@ -216,7 +212,7 @@ func (s *QRepStagingTableSync) SyncQRepRecords( ) s.connector.logger.Info("Executing transaction inside QRep sync", syncLog) _, err = tx.Exec( - context.Background(), + ctx, insertMetadataStmt, flowJobName, partitionID, @@ -228,7 +224,7 @@ func (s *QRepStagingTableSync) SyncQRepRecords( return -1, fmt.Errorf("failed to execute statements in a transaction: %w", err) } - err = tx.Commit(context.Background()) + err = tx.Commit(ctx) if err != nil { return -1, fmt.Errorf("failed to commit transaction: %w", err) } diff --git a/flow/connectors/postgres/qvalue_convert.go b/flow/connectors/postgres/qvalue_convert.go index 2a03a1a7d4..ef12fd61c4 100644 --- a/flow/connectors/postgres/qvalue_convert.go +++ b/flow/connectors/postgres/qvalue_convert.go @@ -19,6 +19,29 @@ import ( "github.com/PeerDB-io/peer-flow/shared" ) +func (c *PostgresConnector) postgresOIDToName(recvOID uint32) string { + if ty, ok := pgtype.NewMap().TypeForOID(recvOID); ok { + return ty.Name + } + // workaround for some types not being defined by pgtype + switch recvOID { + case uint32(oid.T_timetz): + return "timetz" + case uint32(oid.T_xml): + return "xml" + case uint32(oid.T_money): + return "money" + case uint32(oid.T_txid_snapshot): + return "txid_snapshot" + case uint32(oid.T_tsvector): + return "tsvector" + case uint32(oid.T_tsquery): + return "tsquery" + default: + return "" + } +} + func (c *PostgresConnector) postgresOIDToQValueKind(recvOID uint32) qvalue.QValueKind { switch recvOID { case pgtype.BoolOID: @@ -339,7 +362,7 @@ func parseFieldFromQValueKind(qvalueKind qvalue.QValueKind, value interface{}) ( case net.HardwareAddr: return qvalue.QValueMacaddr{Val: v.String()}, nil default: - return nil, fmt.Errorf("failed to parse MACADDR: %v", value) + return nil, fmt.Errorf("failed to parse MACADDR: %v %T", value, v) } case qvalue.QValueKindBytes: rawBytes := value.([]byte) diff --git a/flow/connectors/pubsub/pubsub.go b/flow/connectors/pubsub/pubsub.go index ba2bc8426d..b08150960c 100644 --- a/flow/connectors/pubsub/pubsub.go +++ b/flow/connectors/pubsub/pubsub.go @@ -72,7 +72,12 @@ func (c *PubSubConnector) ReplayTableSchemaDeltas(_ context.Context, flowJobName return nil } -func lvalueToPubSubMessage(ls *lua.LState, value lua.LValue) (string, *pubsub.Message, error) { +type PubSubMessage struct { + *pubsub.Message + Topic string +} + +func lvalueToPubSubMessage(ls *lua.LState, value lua.LValue) (PubSubMessage, error) { var topic string var msg *pubsub.Message switch v := value.(type) { @@ -83,15 +88,15 @@ func lvalueToPubSubMessage(ls *lua.LState, value lua.LValue) (string, *pubsub.Me case *lua.LTable: key, err := utils.LVAsStringOrNil(ls, ls.GetField(v, "key")) if err != nil { - return "", nil, fmt.Errorf("invalid key, %w", err) + return PubSubMessage{}, fmt.Errorf("invalid key, %w", err) } value, err := utils.LVAsReadOnlyBytes(ls, ls.GetField(v, "value")) if err != nil { - return "", nil, fmt.Errorf("invalid value, %w", err) + return PubSubMessage{}, fmt.Errorf("invalid value, %w", err) } topic, err = utils.LVAsStringOrNil(ls, ls.GetField(v, "topic")) if err != nil { - return "", nil, fmt.Errorf("invalid topic, %w", err) + return PubSubMessage{}, fmt.Errorf("invalid topic, %w", err) } msg = &pubsub.Message{ OrderingKey: key, @@ -104,13 +109,16 @@ func lvalueToPubSubMessage(ls *lua.LState, value lua.LValue) (string, *pubsub.Me msg.Attributes[k.String()] = v.String() }) } else if lua.LVAsBool(lheaders) { - return "", nil, fmt.Errorf("invalid headers, must be nil or table: %s", lheaders) + return PubSubMessage{}, fmt.Errorf("invalid headers, must be nil or table: %s", lheaders) } case *lua.LNilType: default: - return "", nil, fmt.Errorf("script returned invalid value: %s", value) + return PubSubMessage{}, fmt.Errorf("script returned invalid value: %s", value) } - return topic, msg, nil + return PubSubMessage{ + Message: msg, + Topic: topic, + }, nil } type topicCache struct { @@ -162,65 +170,77 @@ func (tc *topicCache) GetOrSet(topic string, f func() (*pubsub.Topic, error)) (* return client, nil } -func (c *PubSubConnector) SyncRecords(ctx context.Context, req *model.SyncRecordsRequest) (*model.SyncResponse, error) { - numRecords := int64(0) +func (c *PubSubConnector) SyncRecords(ctx context.Context, req *model.SyncRecordsRequest[model.RecordItems]) (*model.SyncResponse, error) { + numRecords := atomic.Int64{} tableNameRowsMapping := utils.InitialiseTableRowsMap(req.TableMappings) + topiccache := topicCache{cache: make(map[string]*pubsub.Topic)} + publish := make(chan *pubsub.PublishResult, 32) - ls, err := utils.LoadScript(ctx, req.Script, func(ls *lua.LState) int { - top := ls.GetTop() - ss := make([]string, top) - for i := range top { - ss[i] = ls.ToStringMeta(ls.Get(i + 1)).String() + waitChan := make(chan struct{}) + wgCtx, wgErr := context.WithCancelCause(ctx) + + pool, err := utils.LuaPool(func() (*lua.LState, error) { + ls, err := utils.LoadScript(ctx, req.Script, func(ls *lua.LState) int { + top := ls.GetTop() + ss := make([]string, top) + for i := range top { + ss[i] = ls.ToStringMeta(ls.Get(i + 1)).String() + } + _ = c.LogFlowInfo(ctx, req.FlowJobName, strings.Join(ss, "\t")) + return 0 + }) + if err != nil { + return nil, err + } + if req.Script == "" { + ls.Env.RawSetString("onRecord", ls.NewFunction(utils.DefaultOnRecord)) + } + return ls, nil + }, func(messages []PubSubMessage) { + for _, message := range messages { + topicClient, err := topiccache.GetOrSet(message.Topic, func() (*pubsub.Topic, error) { + topicClient := c.client.Topic(message.Topic) + exists, err := topicClient.Exists(wgCtx) + if err != nil { + return nil, fmt.Errorf("error checking if topic exists: %w", err) + } + if !exists { + topicClient, err = c.client.CreateTopic(wgCtx, message.Topic) + if err != nil { + return nil, fmt.Errorf("error creating topic: %w", err) + } + } + return topicClient, nil + }) + if err != nil { + wgErr(err) + return + } + + publish <- topicClient.Publish(ctx, message.Message) } - _ = c.LogFlowInfo(ctx, req.FlowJobName, strings.Join(ss, "\t")) - return 0 }) if err != nil { return nil, err } - defer ls.Close() - if req.Script == "" { - ls.Env.RawSetString("onRecord", ls.NewFunction(utils.DefaultOnRecord)) - } - - lfn := ls.Env.RawGetString("onRecord") - fn, ok := lfn.(*lua.LFunction) - if !ok { - return nil, fmt.Errorf("script should define `onRecord` as function, not %s", lfn) - } + defer pool.Close() - var wg sync.WaitGroup - wgCtx, wgErr := context.WithCancelCause(ctx) - publish := make(chan *pubsub.PublishResult, 60) go func() { - var curpub *pubsub.PublishResult - for { - select { - case curpub, ok = <-publish: - if !ok { - return - } - case <-ctx.Done(): - wgErr(ctx.Err()) - return - } - _, err := curpub.Get(ctx) - if err != nil { + for curpub := range publish { + if _, err := curpub.Get(ctx); err != nil { wgErr(err) - return + break } - wg.Done() } + close(waitChan) }() - topiccache := topicCache{cache: make(map[string]*pubsub.Topic)} lastSeenLSN := atomic.Int64{} flushLoopDone := make(chan struct{}) go func() { ticker := time.NewTicker(peerdbenv.PeerDBQueueFlushTimeoutSeconds()) defer ticker.Stop() - lastUpdatedOffset := int64(0) for { select { case <-ctx.Done(): @@ -230,12 +250,12 @@ func (c *PubSubConnector) SyncRecords(ctx context.Context, req *model.SyncRecord // flush loop doesn't block processing new messages case <-ticker.C: lastSeen := lastSeenLSN.Load() - if lastSeen > lastUpdatedOffset { + if lastSeen > req.ConsumedOffset.Load() { if err := c.SetLastOffset(ctx, req.FlowJobName, lastSeen); err != nil { c.logger.Warn("[pubsub] SetLastOffset error", slog.Any("error", err)) } else { - lastUpdatedOffset = lastSeen - c.logger.Info("processBatch", slog.Int64("updated last offset", lastUpdatedOffset)) + shared.AtomicInt64Max(req.ConsumedOffset, lastSeen) + c.logger.Info("processBatch", slog.Int64("updated last offset", lastSeen)) } } } @@ -250,63 +270,56 @@ Loop: c.logger.Info("flushing batches because no more records") break Loop } - ls.Push(fn) - ls.Push(pua.LuaRecord.New(ls, record)) - err := ls.PCall(1, -1, nil) - if err != nil { - return nil, fmt.Errorf("script failed: %w", err) - } - args := ls.GetTop() - for i := range args { - topic, msg, err := lvalueToPubSubMessage(ls, ls.Get(i-args)) + + pool.Run(func(ls *lua.LState) []PubSubMessage { + lfn := ls.Env.RawGetString("onRecord") + fn, ok := lfn.(*lua.LFunction) + if !ok { + wgErr(fmt.Errorf("script should define `onRecord` as function, not %s", lfn)) + return nil + } + + ls.Push(fn) + ls.Push(pua.LuaRecord.New(ls, record)) + err := ls.PCall(1, -1, nil) if err != nil { - return nil, err + wgErr(fmt.Errorf("script failed: %w", err)) + return nil } - if msg != nil { - if topic == "" { - topic = record.GetDestinationTableName() + + args := ls.GetTop() + results := make([]PubSubMessage, 0, args) + for i := range args { + msg, err := lvalueToPubSubMessage(ls, ls.Get(i-args)) + if err != nil { + wgErr(err) + return nil } - topicClient, err := topiccache.GetOrSet(topic, func() (*pubsub.Topic, error) { - topicClient := c.client.Topic(topic) - exists, err := topicClient.Exists(wgCtx) - if err != nil { - return nil, fmt.Errorf("error checking if topic exists: %w", err) + if msg.Message != nil { + if msg.Topic == "" { + msg.Topic = record.GetDestinationTableName() } - if !exists { - topicClient, err = c.client.CreateTopic(wgCtx, topic) - if err != nil { - return nil, fmt.Errorf("error creating topic: %w", err) - } - } - return topicClient, nil - }) - if err != nil { - return nil, err + results = append(results, msg) + record.PopulateCountMap(tableNameRowsMapping) } - - pubresult := topicClient.Publish(ctx, msg) - wg.Add(1) - publish <- pubresult - record.PopulateCountMap(tableNameRowsMapping) } - } - ls.SetTop(0) - numRecords += 1 - shared.AtomicInt64Max(&lastSeenLSN, record.GetCheckpointID()) + ls.SetTop(0) + numRecords.Add(1) + shared.AtomicInt64Max(&lastSeenLSN, record.GetCheckpointID()) + return results + }) case <-wgCtx.Done(): - return nil, wgCtx.Err() + break Loop } } close(flushLoopDone) close(publish) + if err := pool.Wait(wgCtx); err != nil { + return nil, err + } topiccache.Stop(wgCtx) - waitChan := make(chan struct{}) - go func() { - wg.Wait() - close(waitChan) - }() select { case <-wgCtx.Done(): return nil, wgCtx.Err() @@ -314,15 +327,14 @@ Loop: } lastCheckpoint := req.Records.GetLastCheckpoint() - err = c.FinishBatch(ctx, req.FlowJobName, req.SyncBatchID, lastCheckpoint) - if err != nil { + if err := c.FinishBatch(ctx, req.FlowJobName, req.SyncBatchID, lastCheckpoint); err != nil { return nil, err } return &model.SyncResponse{ CurrentSyncBatchID: req.SyncBatchID, LastSyncedCheckpointID: lastCheckpoint, - NumRecordsSynced: numRecords, + NumRecordsSynced: numRecords.Load(), TableNameRowsMapping: tableNameRowsMapping, TableSchemaDeltas: req.Records.SchemaDeltas, }, nil diff --git a/flow/connectors/s3/qrep.go b/flow/connectors/s3/qrep.go index c1aeced652..14c7b31ef2 100644 --- a/flow/connectors/s3/qrep.go +++ b/flow/connectors/s3/qrep.go @@ -3,14 +3,12 @@ package conns3 import ( "context" "fmt" - "log/slog" "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" ) func (c *S3Connector) SyncQRepRecords( @@ -19,13 +17,7 @@ func (c *S3Connector) SyncQRepRecords( partition *protos.QRepPartition, stream *model.QRecordStream, ) (int, error) { - schema, err := stream.Schema() - if err != nil { - c.logger.Error("failed to get schema from stream", - slog.Any("error", err), - slog.String(string(shared.PartitionIDKey), partition.PartitionId)) - return 0, fmt.Errorf("failed to get schema from stream: %w", err) - } + schema := stream.Schema() dstTableName := config.DestinationTableIdentifier avroSchema, err := getAvroSchema(dstTableName, schema) @@ -43,7 +35,7 @@ func (c *S3Connector) SyncQRepRecords( func getAvroSchema( dstTableName string, - schema *qvalue.QRecordSchema, + schema qvalue.QRecordSchema, ) (*model.QRecordAvroSchemaDefinition, error) { avroSchema, err := model.GetAvroSchemaDefinition(dstTableName, schema, protos.DBType_S3) if err != nil { diff --git a/flow/connectors/s3/s3.go b/flow/connectors/s3/s3.go index d560c38fbf..93e041a62e 100644 --- a/flow/connectors/s3/s3.go +++ b/flow/connectors/s3/s3.go @@ -111,7 +111,7 @@ func (c *S3Connector) ConnectionActive(ctx context.Context) error { return nil } -func (c *S3Connector) SyncRecords(ctx context.Context, req *model.SyncRecordsRequest) (*model.SyncResponse, error) { +func (c *S3Connector) SyncRecords(ctx context.Context, req *model.SyncRecordsRequest[model.RecordItems]) (*model.SyncResponse, error) { tableNameRowsMapping := utils.InitialiseTableRowsMap(req.TableMappings) streamReq := model.NewRecordsToStreamRequest(req.Records.GetRecords(), tableNameRowsMapping, req.SyncBatchID) recordStream, err := utils.RecordsToRawTableStream(streamReq) diff --git a/flow/connectors/snowflake/avro_file_writer_test.go b/flow/connectors/snowflake/avro_file_writer_test.go index 0b9e2f1aa8..a4d89bd773 100644 --- a/flow/connectors/snowflake/avro_file_writer_test.go +++ b/flow/connectors/snowflake/avro_file_writer_test.go @@ -74,7 +74,7 @@ func generateRecords( nullable bool, numRows uint32, allnulls bool, -) (*model.QRecordStream, *qvalue.QRecordSchema) { +) (*model.QRecordStream, qvalue.QRecordSchema) { t.Helper() allQValueKinds := []qvalue.QValueKind{ @@ -102,16 +102,9 @@ func generateRecords( numKinds := len(allQValueKinds) - schema := &qvalue.QRecordSchema{ + schema := qvalue.QRecordSchema{ Fields: make([]qvalue.QField, numKinds), } - - // Create sample records - records := &model.QRecordBatch{ - Records: make([][]qvalue.QValue, numRows), - Schema: schema, - } - for i, kind := range allQValueKinds { schema.Fields[i] = qvalue.QField{ Name: string(kind), @@ -120,6 +113,12 @@ func generateRecords( } } + // Create sample records + records := &model.QRecordBatch{ + Schema: schema, + Records: make([][]qvalue.QValue, numRows), + } + for row := range numRows { entries := make([]qvalue.QValue, len(allQValueKinds)) diff --git a/flow/connectors/snowflake/get_schema_for_tests.go b/flow/connectors/snowflake/get_schema_for_tests.go index 476f16f165..c7875f173f 100644 --- a/flow/connectors/snowflake/get_schema_for_tests.go +++ b/flow/connectors/snowflake/get_schema_for_tests.go @@ -3,34 +3,36 @@ package connsnowflake import ( "context" + "github.com/PeerDB-io/peer-flow/datatypes" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model/qvalue" ) func (c *SnowflakeConnector) getTableSchemaForTable(ctx context.Context, tableName string) (*protos.TableSchema, error) { - colNames, colTypes, err := c.getColsFromTable(ctx, tableName) + columns, err := c.getColsFromTable(ctx, tableName) if err != nil { return nil, err } - colFields := make([]*protos.FieldDescription, 0, len(colNames)) - for i, sfType := range colTypes { - genericColType, err := snowflakeTypeToQValueKind(sfType) + colFields := make([]*protos.FieldDescription, 0, len(columns)) + for i, sfColumn := range columns { + genericColType, err := snowflakeTypeToQValueKind(sfColumn.ColumnType) if err != nil { // we use string for invalid types genericColType = qvalue.QValueKindString } - colTypes[i] = string(genericColType) + colFields = append(colFields, &protos.FieldDescription{ - Name: colNames[i], - Type: colTypes[i], - TypeModifier: -1, + Name: columns[i].ColumnName, + Type: string(genericColType), + TypeModifier: datatypes.MakeNumericTypmod(sfColumn.NumericPrecision, sfColumn.NumericScale), }) } return &protos.TableSchema{ TableIdentifier: tableName, Columns: colFields, + System: protos.TypeSystem_Q, }, nil } diff --git a/flow/connectors/snowflake/qrep.go b/flow/connectors/snowflake/qrep.go index df6304e1fb..daf3e330db 100644 --- a/flow/connectors/snowflake/qrep.go +++ b/flow/connectors/snowflake/qrep.go @@ -17,6 +17,13 @@ import ( "github.com/PeerDB-io/peer-flow/shared" ) +type SnowflakeTableColumn struct { + ColumnName string + ColumnType string + NumericPrecision int32 + NumericScale int32 +} + func (c *SnowflakeConnector) SyncQRepRecords( ctx context.Context, config *protos.QRepConfig, @@ -181,11 +188,11 @@ func (c *SnowflakeConnector) CleanupQRepFlow(ctx context.Context, config *protos return c.dropStage(ctx, config.StagingPath, config.FlowJobName) } -func (c *SnowflakeConnector) getColsFromTable(ctx context.Context, tableName string) ([]string, []string, error) { +func (c *SnowflakeConnector) getColsFromTable(ctx context.Context, tableName string) ([]SnowflakeTableColumn, error) { // parse the table name to get the schema and table name schemaTable, err := utils.ParseSchemaTable(tableName) if err != nil { - return nil, nil, fmt.Errorf("failed to parse table name: %w", err) + return nil, fmt.Errorf("failed to parse table name: %w", err) } rows, err := c.database.QueryContext( @@ -195,31 +202,35 @@ func (c *SnowflakeConnector) getColsFromTable(ctx context.Context, tableName str strings.ToUpper(schemaTable.Table), ) if err != nil { - return nil, nil, fmt.Errorf("failed to execute query: %w", err) + return nil, fmt.Errorf("failed to execute query: %w", err) } defer rows.Close() var colName, colType pgtype.Text - colNames := make([]string, 0, 8) - colTypes := make([]string, 0, 8) + var numericPrecision, numericScale pgtype.Int4 + var cols []SnowflakeTableColumn for rows.Next() { - if err := rows.Scan(&colName, &colType); err != nil { - return nil, nil, fmt.Errorf("failed to scan row: %w", err) + if err := rows.Scan(&colName, &colType, &numericPrecision, &numericScale); err != nil { + return nil, fmt.Errorf("failed to scan row: %w", err) } - colNames = append(colNames, colName.String) - colTypes = append(colTypes, colType.String) + cols = append(cols, SnowflakeTableColumn{ + ColumnName: colName.String, + ColumnType: colType.String, + NumericPrecision: numericPrecision.Int32, + NumericScale: numericScale.Int32, + }) } err = rows.Err() if err != nil { - return nil, nil, fmt.Errorf("failed to read rows: %w", err) + return nil, fmt.Errorf("failed to read rows: %w", err) } - if len(colNames) == 0 { - return nil, nil, fmt.Errorf("cannot load schema: table %s.%s does not exist", schemaTable.Schema, schemaTable.Table) + if len(cols) == 0 { + return nil, fmt.Errorf("cannot load schema: table %s.%s does not exist", schemaTable.Schema, schemaTable.Table) } - return colNames, colTypes, nil + return cols, nil } // dropStage drops the stage for the given job. diff --git a/flow/connectors/snowflake/qrep_avro_consolidate.go b/flow/connectors/snowflake/qrep_avro_consolidate.go index 6f261d2cf8..a2a4e0138a 100644 --- a/flow/connectors/snowflake/qrep_avro_consolidate.go +++ b/flow/connectors/snowflake/qrep_avro_consolidate.go @@ -39,10 +39,17 @@ func NewSnowflakeAvroConsolidateHandler( func (s *SnowflakeAvroConsolidateHandler) CopyStageToDestination(ctx context.Context) error { s.connector.logger.Info("Copying stage to destination " + s.dstTableName) - colNames, colTypes, colsErr := s.connector.getColsFromTable(ctx, s.dstTableName) + columns, colsErr := s.connector.getColsFromTable(ctx, s.dstTableName) if colsErr != nil { return fmt.Errorf("failed to get columns from destination table: %w", colsErr) } + + colNames := make([]string, 0, len(columns)) + colTypes := make([]string, 0, len(columns)) + for _, col := range columns { + colNames = append(colNames, col.ColumnName) + colTypes = append(colTypes, col.ColumnType) + } s.allColNames = colNames s.allColTypes = colTypes diff --git a/flow/connectors/snowflake/qrep_avro_sync.go b/flow/connectors/snowflake/qrep_avro_sync.go index ea8665d434..f42c9f4cd8 100644 --- a/flow/connectors/snowflake/qrep_avro_sync.go +++ b/flow/connectors/snowflake/qrep_avro_sync.go @@ -44,10 +44,7 @@ func (s *SnowflakeAvroSyncHandler) SyncRecords( tableLog := slog.String("destinationTable", s.config.DestinationTableIdentifier) dstTableName := s.config.DestinationTableIdentifier - schema, err := stream.Schema() - if err != nil { - return -1, fmt.Errorf("failed to get schema from stream: %w", err) - } + schema := stream.Schema() s.connector.logger.Info("sync function called and schema acquired", tableLog) @@ -99,13 +96,10 @@ func (s *SnowflakeAvroSyncHandler) SyncQRepRecords( startTime := time.Now() dstTableName := config.DestinationTableIdentifier - schema, err := stream.Schema() - if err != nil { - return -1, fmt.Errorf("failed to get schema from stream: %w", err) - } + schema := stream.Schema() s.connector.logger.Info("sync function called and schema acquired", partitionLog) - err = s.addMissingColumns(ctx, schema, dstTableSchema, dstTableName, partition) + err := s.addMissingColumns(ctx, schema, dstTableSchema, dstTableName, partition) if err != nil { return 0, err } @@ -141,7 +135,7 @@ func (s *SnowflakeAvroSyncHandler) SyncQRepRecords( func (s *SnowflakeAvroSyncHandler) addMissingColumns( ctx context.Context, - schema *qvalue.QRecordSchema, + schema qvalue.QRecordSchema, dstTableSchema []*sql.ColumnType, dstTableName string, partition *protos.QRepPartition, @@ -205,7 +199,7 @@ func (s *SnowflakeAvroSyncHandler) addMissingColumns( func (s *SnowflakeAvroSyncHandler) getAvroSchema( dstTableName string, - schema *qvalue.QRecordSchema, + schema qvalue.QRecordSchema, ) (*model.QRecordAvroSchemaDefinition, error) { avroSchema, err := model.GetAvroSchemaDefinition(dstTableName, schema, protos.DBType_SNOWFLAKE) if err != nil { diff --git a/flow/connectors/snowflake/snowflake.go b/flow/connectors/snowflake/snowflake.go index d29e3d8e09..27bb69d07f 100644 --- a/flow/connectors/snowflake/snowflake.go +++ b/flow/connectors/snowflake/snowflake.go @@ -7,7 +7,6 @@ import ( "errors" "fmt" "log/slog" - "regexp" "strings" "sync/atomic" "time" @@ -62,7 +61,7 @@ const ( ARRAY_AGG(DISTINCT _PEERDB_UNCHANGED_TOAST_COLUMNS) FROM %s.%s WHERE _PEERDB_BATCH_ID = %d AND _PEERDB_RECORD_TYPE != 2 GROUP BY _PEERDB_DESTINATION_TABLE_NAME` - getTableSchemaSQL = `SELECT COLUMN_NAME, DATA_TYPE FROM INFORMATION_SCHEMA.COLUMNS + getTableSchemaSQL = `SELECT COLUMN_NAME, DATA_TYPE, NUMERIC_PRECISION, NUMERIC_SCALE FROM INFORMATION_SCHEMA.COLUMNS WHERE UPPER(TABLE_SCHEMA)=? AND UPPER(TABLE_NAME)=? ORDER BY ORDINAL_POSITION` checkIfTableExistsSQL = `SELECT TO_BOOLEAN(COUNT(1)) FROM INFORMATION_SCHEMA.TABLES @@ -372,20 +371,26 @@ func (c *SnowflakeConnector) ReplayTableSchemaDeltas( } for _, addedColumn := range schemaDelta.AddedColumns { - sfColtype, err := qvalue.QValueKind(addedColumn.ColumnType).ToDWHColumnType(protos.DBType_SNOWFLAKE) + sfColtype, err := qvalue.QValueKind(addedColumn.Type).ToDWHColumnType(protos.DBType_SNOWFLAKE) if err != nil { return fmt.Errorf("failed to convert column type %s to snowflake type: %w", - addedColumn.ColumnType, err) + addedColumn.Type, err) } + + if addedColumn.Type == string(qvalue.QValueKindNumeric) { + precision, scale := numeric.GetNumericTypeForWarehouse(addedColumn.TypeModifier, numeric.SnowflakeNumericCompatibility{}) + sfColtype = fmt.Sprintf("NUMERIC(%d,%d)", precision, scale) + } + _, err = tableSchemaModifyTx.ExecContext(ctx, fmt.Sprintf("ALTER TABLE %s ADD COLUMN IF NOT EXISTS \"%s\" %s", - schemaDelta.DstTableName, strings.ToUpper(addedColumn.ColumnName), sfColtype)) + schemaDelta.DstTableName, strings.ToUpper(addedColumn.Name), sfColtype)) if err != nil { - return fmt.Errorf("failed to add column %s for table %s: %w", addedColumn.ColumnName, + return fmt.Errorf("failed to add column %s for table %s: %w", addedColumn.Name, schemaDelta.DstTableName, err) } - c.logger.Info(fmt.Sprintf("[schema delta replay] added column %s with data type %s", addedColumn.ColumnName, - addedColumn.ColumnType), + c.logger.Info(fmt.Sprintf("[schema delta replay] added column %s with data type %s", addedColumn.Name, + sfColtype), "destination table name", schemaDelta.DstTableName, "source table name", schemaDelta.SrcTableName) } @@ -400,7 +405,7 @@ func (c *SnowflakeConnector) ReplayTableSchemaDeltas( return nil } -func (c *SnowflakeConnector) SyncRecords(ctx context.Context, req *model.SyncRecordsRequest) (*model.SyncResponse, error) { +func (c *SnowflakeConnector) SyncRecords(ctx context.Context, req *model.SyncRecordsRequest[model.RecordItems]) (*model.SyncResponse, error) { rawTableIdentifier := getRawTableIdentifier(req.FlowJobName) c.logger.Info("pushing records to Snowflake table " + rawTableIdentifier) @@ -419,7 +424,7 @@ func (c *SnowflakeConnector) SyncRecords(ctx context.Context, req *model.SyncRec func (c *SnowflakeConnector) syncRecordsViaAvro( ctx context.Context, - req *model.SyncRecordsRequest, + req *model.SyncRecordsRequest[model.RecordItems], rawTableIdentifier string, syncBatchID int64, ) (*model.SyncResponse, error) { @@ -707,8 +712,7 @@ func generateCreateTableSQLForNormalizedTable( } func getRawTableIdentifier(jobName string) string { - jobName = regexp.MustCompile("[^a-zA-Z0-9_]+").ReplaceAllString(jobName, "_") - return fmt.Sprintf("%s_%s", rawTablePrefix, jobName) + return rawTablePrefix + "_" + shared.ReplaceIllegalCharactersWithUnderscores(jobName) } func (c *SnowflakeConnector) RenameTables(ctx context.Context, req *protos.RenameTablesInput) (*protos.RenameTablesOutput, error) { diff --git a/flow/connectors/sql/query_executor.go b/flow/connectors/sql/query_executor.go index 01cad5ebaa..7f56398f40 100644 --- a/flow/connectors/sql/query_executor.go +++ b/flow/connectors/sql/query_executor.go @@ -262,8 +262,8 @@ func (g *GenericSQLQueryExecutor) processRows(ctx context.Context, rows *sqlx.Ro } return &model.QRecordBatch{ - Records: records, Schema: qvalue.NewQRecordSchema(qfields), + Records: records, }, nil } diff --git a/flow/connectors/utils/avro/avro_writer.go b/flow/connectors/utils/avro/avro_writer.go index cb0c4afff3..ad2597478a 100644 --- a/flow/connectors/utils/avro/avro_writer.go +++ b/flow/connectors/utils/avro/avro_writer.go @@ -120,13 +120,7 @@ func (p *peerDBOCFWriter) createOCFWriter(w io.Writer) (*goavro.OCFWriter, error func (p *peerDBOCFWriter) writeRecordsToOCFWriter(ctx context.Context, ocfWriter *goavro.OCFWriter) (int, error) { logger := logger.LoggerFromCtx(ctx) - schema, err := p.stream.Schema() - if err != nil { - logger.Error("failed to get schema from stream", slog.Any("error", err)) - return 0, fmt.Errorf("failed to get schema from stream: %w", err) - } - - numRows := 0 + schema := p.stream.Schema() avroConverter := model.NewQRecordAvroConverter( p.avroSchema, @@ -135,13 +129,9 @@ func (p *peerDBOCFWriter) writeRecordsToOCFWriter(ctx context.Context, ocfWriter logger, ) - for qRecordOrErr := range p.stream.Records { - if qRecordOrErr.Err != nil { - logger.Error("[avro] failed to get record from stream", slog.Any("error", qRecordOrErr.Err)) - return 0, fmt.Errorf("[avro] failed to get record from stream: %w", qRecordOrErr.Err) - } - - avroMap, err := avroConverter.Convert(qRecordOrErr.Record) + numRows := 0 + for qrecord := range p.stream.Records { + avroMap, err := avroConverter.Convert(qrecord) if err != nil { logger.Error("failed to convert QRecord to Avro compatible map: ", slog.Any("error", err)) return 0, fmt.Errorf("failed to convert QRecord to Avro compatible map: %w", err) @@ -155,6 +145,10 @@ func (p *peerDBOCFWriter) writeRecordsToOCFWriter(ctx context.Context, ocfWriter numRows += 1 } + if err := p.stream.Err(); err != nil { + logger.Error("[avro] failed to get record from stream", slog.Any("error", err)) + return 0, fmt.Errorf("[avro] failed to get record from stream: %w", err) + } return numRows, nil } diff --git a/flow/connectors/utils/cdc_records/cdc_records_storage.go b/flow/connectors/utils/cdc_store.go similarity index 87% rename from flow/connectors/utils/cdc_records/cdc_records_storage.go rename to flow/connectors/utils/cdc_store.go index 4e1516324d..a98f10c656 100644 --- a/flow/connectors/utils/cdc_records/cdc_records_storage.go +++ b/flow/connectors/utils/cdc_store.go @@ -1,4 +1,4 @@ -package cdc_records +package utils import ( "bytes" @@ -31,8 +31,8 @@ func encVal(val any) ([]byte, error) { return buf.Bytes(), nil } -type cdcRecordsStore struct { - inMemoryRecords map[model.TableWithPkey]model.Record +type cdcStore[Items model.Items] struct { + inMemoryRecords map[model.TableWithPkey]model.Record[Items] pebbleDB *pebble.DB flowJobName string dbFolderName string @@ -43,9 +43,9 @@ type cdcRecordsStore struct { numRecordsSwitchThreshold int } -func NewCDCRecordsStore(flowJobName string) *cdcRecordsStore { - return &cdcRecordsStore{ - inMemoryRecords: make(map[model.TableWithPkey]model.Record), +func NewCDCStore[Items model.Items](flowJobName string) *cdcStore[Items] { + return &cdcStore[Items]{ + inMemoryRecords: make(map[model.TableWithPkey]model.Record[Items]), pebbleDB: nil, numRecords: atomic.Int32{}, flowJobName: flowJobName, @@ -60,19 +60,12 @@ func NewCDCRecordsStore(flowJobName string) *cdcRecordsStore { return 0 }(), thresholdReason: "", - memStats: []metrics.Sample{{Name: "/gc/heap/allocs:bytes"}}, + memStats: []metrics.Sample{{Name: "/memory/classes/heap/objects:bytes"}}, } } -func (c *cdcRecordsStore) initPebbleDB() error { - if c.pebbleDB != nil { - return nil - } - +func init() { // register future record classes here as well, if they are passed/stored as interfaces - gob.Register(&model.InsertRecord{}) - gob.Register(&model.UpdateRecord{}) - gob.Register(&model.DeleteRecord{}) gob.Register(time.Time{}) gob.Register(decimal.Decimal{}) gob.Register(qvalue.QValueNull("")) @@ -114,6 +107,17 @@ func (c *cdcRecordsStore) initPebbleDB() error { gob.Register(qvalue.QValueArrayTimestamp{}) gob.Register(qvalue.QValueArrayTimestampTZ{}) gob.Register(qvalue.QValueArrayBoolean{}) +} + +func (c *cdcStore[T]) initPebbleDB() error { + if c.pebbleDB != nil { + return nil + } + + gob.Register(&model.InsertRecord[T]{}) + gob.Register(&model.UpdateRecord[T]{}) + gob.Register(&model.DeleteRecord[T]{}) + gob.Register(&model.RelationRecord[T]{}) var err error // we don't want a WAL since cache, we don't want to overwrite another DB either @@ -128,7 +132,7 @@ func (c *cdcRecordsStore) initPebbleDB() error { return nil } -func (c *cdcRecordsStore) diskSpillThresholdsExceeded() bool { +func (c *cdcStore[T]) diskSpillThresholdsExceeded() bool { if len(c.inMemoryRecords) >= c.numRecordsSwitchThreshold { c.thresholdReason = fmt.Sprintf("more than %d primary keys read, spilling to disk", c.numRecordsSwitchThreshold) @@ -146,7 +150,7 @@ func (c *cdcRecordsStore) diskSpillThresholdsExceeded() bool { return false } -func (c *cdcRecordsStore) Set(logger log.Logger, key model.TableWithPkey, rec model.Record) error { +func (c *cdcStore[T]) Set(logger log.Logger, key model.TableWithPkey, rec model.Record[T]) error { if key.TableName != "" { _, ok := c.inMemoryRecords[key] if ok || !c.diskSpillThresholdsExceeded() { @@ -180,12 +184,13 @@ func (c *cdcRecordsStore) Set(logger log.Logger, key model.TableWithPkey, rec mo } } } + c.numRecords.Add(1) return nil } // bool is to indicate if a record is found or not [similar to ok] -func (c *cdcRecordsStore) Get(key model.TableWithPkey) (model.Record, bool, error) { +func (c *cdcStore[T]) Get(key model.TableWithPkey) (model.Record[T], bool, error) { rec, ok := c.inMemoryRecords[key] if ok { return rec, true, nil @@ -212,7 +217,7 @@ func (c *cdcRecordsStore) Get(key model.TableWithPkey) (model.Record, bool, erro }() dec := gob.NewDecoder(bytes.NewReader(encodedRec)) - var rec model.Record + var rec model.Record[T] err = dec.Decode(&rec) if err != nil { return nil, false, fmt.Errorf("failed to decode record: %w", err) @@ -223,15 +228,15 @@ func (c *cdcRecordsStore) Get(key model.TableWithPkey) (model.Record, bool, erro return nil, false, nil } -func (c *cdcRecordsStore) Len() int { +func (c *cdcStore[T]) Len() int { return int(c.numRecords.Load()) } -func (c *cdcRecordsStore) IsEmpty() bool { +func (c *cdcStore[T]) IsEmpty() bool { return c.Len() == 0 } -func (c *cdcRecordsStore) Close() error { +func (c *cdcStore[T]) Close() error { c.inMemoryRecords = nil if c.pebbleDB != nil { err := c.pebbleDB.Close() diff --git a/flow/connectors/utils/cdc_records/cdc_records_storage_test.go b/flow/connectors/utils/cdc_store_test.go similarity index 89% rename from flow/connectors/utils/cdc_records/cdc_records_storage_test.go rename to flow/connectors/utils/cdc_store_test.go index b8209bfed0..ab6560f3d1 100644 --- a/flow/connectors/utils/cdc_records/cdc_records_storage_test.go +++ b/flow/connectors/utils/cdc_store_test.go @@ -1,4 +1,4 @@ -package cdc_records +package utils import ( "crypto/rand" @@ -32,7 +32,7 @@ func getDecimalForTesting(t *testing.T) decimal.Decimal { return decimal.New(9876543210, 123) } -func genKeyAndRec(t *testing.T) (model.TableWithPkey, model.Record) { +func genKeyAndRec(t *testing.T) (model.TableWithPkey, model.Record[model.RecordItems]) { t.Helper() pkeyColVal := make([]byte, 32) @@ -46,7 +46,7 @@ func genKeyAndRec(t *testing.T) (model.TableWithPkey, model.Record) { TableName: "test_src_tbl", PkeyColVal: [32]byte(pkeyColVal), } - rec := &model.InsertRecord{ + rec := &model.InsertRecord[model.RecordItems]{ BaseRecord: model.BaseRecord{ CheckpointID: 1, CommitTimeNano: time.Now().UnixNano(), @@ -67,7 +67,7 @@ func genKeyAndRec(t *testing.T) (model.TableWithPkey, model.Record) { func TestSingleRecord(t *testing.T) { t.Parallel() - cdcRecordsStore := NewCDCRecordsStore("test_single_record") + cdcRecordsStore := NewCDCStore[model.RecordItems]("test_single_record") cdcRecordsStore.numRecordsSwitchThreshold = 10 key, rec := genKeyAndRec(t) @@ -87,7 +87,7 @@ func TestSingleRecord(t *testing.T) { func TestRecordsTillSpill(t *testing.T) { t.Parallel() - cdcRecordsStore := NewCDCRecordsStore("test_records_till_spill") + cdcRecordsStore := NewCDCStore[model.RecordItems]("test_records_till_spill") cdcRecordsStore.numRecordsSwitchThreshold = 10 // add records upto set limit @@ -118,7 +118,7 @@ func TestRecordsTillSpill(t *testing.T) { func TestTimeAndDecimalEncoding(t *testing.T) { t.Parallel() - cdcRecordsStore := NewCDCRecordsStore("test_time_encoding") + cdcRecordsStore := NewCDCStore[model.RecordItems]("test_time_encoding") cdcRecordsStore.numRecordsSwitchThreshold = 0 key, rec := genKeyAndRec(t) @@ -130,7 +130,7 @@ func TestTimeAndDecimalEncoding(t *testing.T) { require.True(t, ok) require.Equal(t, rec, retreived) - _, err = retreived.GetItems().ToJSON() + _, err = model.ItemsToJSON(retreived.GetItems()) require.NoError(t, err) require.NoError(t, cdcRecordsStore.Close()) @@ -139,7 +139,7 @@ func TestTimeAndDecimalEncoding(t *testing.T) { func TestNullKeyDoesntStore(t *testing.T) { t.Parallel() - cdcRecordsStore := NewCDCRecordsStore("test_time_encoding") + cdcRecordsStore := NewCDCStore[model.RecordItems]("test_time_encoding") cdcRecordsStore.numRecordsSwitchThreshold = 0 key, rec := genKeyAndRec(t) diff --git a/flow/connectors/utils/lua.go b/flow/connectors/utils/lua.go index 846761dc96..47676721b3 100644 --- a/flow/connectors/utils/lua.go +++ b/flow/connectors/utils/lua.go @@ -9,6 +9,7 @@ import ( "github.com/PeerDB-io/gluaflatbuffers" "github.com/PeerDB-io/gluajson" "github.com/PeerDB-io/peer-flow/model" + "github.com/PeerDB-io/peer-flow/peerdbenv" "github.com/PeerDB-io/peer-flow/pua" "github.com/PeerDB-io/peer-flow/shared" ) @@ -72,7 +73,7 @@ func LoadScript(ctx context.Context, script string, printfn lua.LGFunction) (*lu func DefaultOnRecord(ls *lua.LState) int { ud, record := pua.LuaRecord.Check(ls, 1) - if _, ok := record.(*model.RelationRecord); ok { + if _, ok := record.(*model.RelationRecord[model.RecordItems]); ok { return 0 } ls.Push(ls.NewFunction(gluajson.LuaJsonEncode)) @@ -80,3 +81,96 @@ func DefaultOnRecord(ls *lua.LState) int { ls.Call(1, 1) return 1 } + +type LPoolMessage[T any] struct { + f func(*lua.LState) T + ret chan<- T +} +type LPool[T any] struct { + messages chan LPoolMessage[T] + returns chan<- (<-chan T) + wait <-chan struct{} + cons func() (*lua.LState, error) + maxSize int + size int + closed bool +} + +func LuaPool[T any](cons func() (*lua.LState, error), merge func(T)) (*LPool[T], error) { + maxSize := peerdbenv.PeerDBQueueParallelism() + returns := make(chan (<-chan T), maxSize) + wait := make(chan struct{}) + go func() { + for ret := range returns { + for val := range ret { + merge(val) + } + } + close(wait) + }() + + pool := &LPool[T]{ + messages: make(chan LPoolMessage[T]), + returns: returns, + wait: wait, + cons: cons, + maxSize: maxSize, + size: 0, + closed: false, + } + if err := pool.Spawn(); err != nil { + pool.Close() + return nil, err + } + return pool, nil +} + +func (pool *LPool[T]) Spawn() error { + ls, err := pool.cons() + if err != nil { + return err + } + pool.size += 1 + go func() { + defer ls.Close() + for message := range pool.messages { + message.ret <- message.f(ls) + close(message.ret) + } + }() + return nil +} + +func (pool *LPool[T]) Close() { + if !pool.closed { + close(pool.returns) + close(pool.messages) + pool.closed = true + } +} + +func (pool *LPool[T]) Run(f func(*lua.LState) T) { + ret := make(chan T, 1) + msg := LPoolMessage[T]{f: f, ret: ret} + if pool.size < pool.maxSize { + select { + case pool.messages <- msg: + pool.returns <- ret + return + default: + _ = pool.Spawn() + } + } + pool.messages <- msg + pool.returns <- ret +} + +func (pool *LPool[T]) Wait(ctx context.Context) error { + pool.Close() + select { + case <-pool.wait: + return nil + case <-ctx.Done(): + return ctx.Err() + } +} diff --git a/flow/connectors/utils/monitoring/monitoring.go b/flow/connectors/utils/monitoring/monitoring.go index 2adf240430..7ecf248c0a 100644 --- a/flow/connectors/utils/monitoring/monitoring.go +++ b/flow/connectors/utils/monitoring/monitoring.go @@ -114,7 +114,7 @@ func AddCDCBatchTablesForFlow(ctx context.Context, pool *pgxpool.Pool, flowJobNa return fmt.Errorf("error while beginning transaction for inserting statistics into cdc_batch_table: %w", err) } defer func() { - err = insertBatchTablesTx.Rollback(ctx) + err = insertBatchTablesTx.Rollback(context.Background()) if err != pgx.ErrTxClosed && err != nil { logger.LoggerFromCtx(ctx).Error("error during transaction rollback", slog.Any("error", err), diff --git a/flow/connectors/utils/stream.go b/flow/connectors/utils/stream.go index 9cbe22a362..483d0f58ee 100644 --- a/flow/connectors/utils/stream.go +++ b/flow/connectors/utils/stream.go @@ -11,9 +11,9 @@ import ( "github.com/PeerDB-io/peer-flow/model/qvalue" ) -func RecordsToRawTableStream(req *model.RecordsToStreamRequest) (*model.QRecordStream, error) { +func RecordsToRawTableStream[Items model.Items](req *model.RecordsToStreamRequest[Items]) (*model.QRecordStream, error) { recordStream := model.NewQRecordStream(1 << 17) - err := recordStream.SetSchema(&qvalue.QRecordSchema{ + recordStream.SetSchema(qvalue.QRecordSchema{ Fields: []qvalue.QField{ { Name: "_peerdb_uid", @@ -57,15 +57,17 @@ func RecordsToRawTableStream(req *model.RecordsToStreamRequest) (*model.QRecordS }, }, }) - if err != nil { - return nil, err - } go func() { for record := range req.GetRecords() { record.PopulateCountMap(req.TableMapping) - qRecordOrError := recordToQRecordOrError(req.BatchID, record) - recordStream.Records <- qRecordOrError + qRecord, err := recordToQRecordOrError(req.BatchID, record) + if err != nil { + recordStream.Close(err) + return + } else { + recordStream.Records <- qRecord + } } close(recordStream.Records) @@ -73,34 +75,28 @@ func RecordsToRawTableStream(req *model.RecordsToStreamRequest) (*model.QRecordS return recordStream, nil } -func recordToQRecordOrError(batchID int64, record model.Record) model.QRecordOrError { +func recordToQRecordOrError[Items model.Items](batchID int64, record model.Record[Items]) ([]qvalue.QValue, error) { var entries [8]qvalue.QValue switch typedRecord := record.(type) { - case *model.InsertRecord: + case *model.InsertRecord[Items]: // json.Marshal converts bytes in Hex automatically to BASE64 string. - itemsJSON, err := typedRecord.Items.ToJSON() + itemsJSON, err := model.ItemsToJSON(typedRecord.Items) if err != nil { - return model.QRecordOrError{ - Err: fmt.Errorf("failed to serialize insert record items to JSON: %w", err), - } + return nil, fmt.Errorf("failed to serialize insert record items to JSON: %w", err) } entries[3] = qvalue.QValueString{Val: itemsJSON} entries[4] = qvalue.QValueInt64{Val: 0} entries[5] = qvalue.QValueString{Val: ""} entries[7] = qvalue.QValueString{Val: ""} - case *model.UpdateRecord: - newItemsJSON, err := typedRecord.NewItems.ToJSON() + case *model.UpdateRecord[Items]: + newItemsJSON, err := model.ItemsToJSON(typedRecord.NewItems) if err != nil { - return model.QRecordOrError{ - Err: fmt.Errorf("failed to serialize update record new items to JSON: %w", err), - } + return nil, fmt.Errorf("failed to serialize update record new items to JSON: %w", err) } - oldItemsJSON, err := typedRecord.OldItems.ToJSON() + oldItemsJSON, err := model.ItemsToJSON(typedRecord.OldItems) if err != nil { - return model.QRecordOrError{ - Err: fmt.Errorf("failed to serialize update record old items to JSON: %w", err), - } + return nil, fmt.Errorf("failed to serialize update record old items to JSON: %w", err) } entries[3] = qvalue.QValueString{Val: newItemsJSON} @@ -108,12 +104,10 @@ func recordToQRecordOrError(batchID int64, record model.Record) model.QRecordOrE entries[5] = qvalue.QValueString{Val: oldItemsJSON} entries[7] = qvalue.QValueString{Val: KeysToString(typedRecord.UnchangedToastColumns)} - case *model.DeleteRecord: - itemsJSON, err := typedRecord.Items.ToJSON() + case *model.DeleteRecord[Items]: + itemsJSON, err := model.ItemsToJSON(typedRecord.Items) if err != nil { - return model.QRecordOrError{ - Err: fmt.Errorf("failed to serialize delete record items to JSON: %w", err), - } + return nil, fmt.Errorf("failed to serialize delete record items to JSON: %w", err) } entries[3] = qvalue.QValueString{Val: itemsJSON} @@ -122,9 +116,7 @@ func recordToQRecordOrError(batchID int64, record model.Record) model.QRecordOrE entries[7] = qvalue.QValueString{Val: KeysToString(typedRecord.UnchangedToastColumns)} default: - return model.QRecordOrError{ - Err: fmt.Errorf("unknown record type: %T", typedRecord), - } + return nil, fmt.Errorf("unknown record type: %T", typedRecord) } entries[0] = qvalue.QValueString{Val: uuid.New().String()} @@ -132,9 +124,7 @@ func recordToQRecordOrError(batchID int64, record model.Record) model.QRecordOrE entries[2] = qvalue.QValueString{Val: record.GetDestinationTableName()} entries[6] = qvalue.QValueInt64{Val: batchID} - return model.QRecordOrError{ - Record: entries[:], - } + return entries[:], nil } func InitialiseTableRowsMap(tableMaps []*protos.TableMapping) map[string]*model.RecordTypeCounts { diff --git a/flow/datatypes/numeric.go b/flow/datatypes/numeric.go index 65d64d4164..0a072ae42f 100644 --- a/flow/datatypes/numeric.go +++ b/flow/datatypes/numeric.go @@ -8,6 +8,7 @@ const ( PeerDBSnowflakeScale = 20 PeerDBClickhousePrecision = 76 PeerDBClickhouseScale = 38 + VARHDRSZ = 4 ) type WarehouseNumericCompatibility interface { @@ -89,10 +90,17 @@ func (DefaultNumericCompatibility) IsValidPrevisionAndScale(precision, scale int return true } +func MakeNumericTypmod(precision int32, scale int32) int32 { + if precision == 0 && scale == 0 { + return -1 + } + return (precision << 16) | (scale & 0x7ff) + VARHDRSZ +} + // This is to reverse what make_numeric_typmod of Postgres does: // https://github.com/postgres/postgres/blob/21912e3c0262e2cfe64856e028799d6927862563/src/backend/utils/adt/numeric.c#L897 func ParseNumericTypmod(typmod int32) (int16, int16) { - offsetMod := typmod - 4 + offsetMod := typmod - VARHDRSZ precision := int16((offsetMod >> 16) & 0x7FFF) scale := int16(offsetMod & 0x7FFF) return precision, scale diff --git a/flow/e2e/bigquery/bigquery_helper.go b/flow/e2e/bigquery/bigquery_helper.go index c3a429d031..9fd5691114 100644 --- a/flow/e2e/bigquery/bigquery_helper.go +++ b/flow/e2e/bigquery/bigquery_helper.go @@ -231,11 +231,7 @@ func toQValue(bqValue bigquery.Value) (qvalue.QValue, error) { case time.Time: return qvalue.QValueTimestamp{Val: v}, nil case *big.Rat: - val, err := decimal.NewFromString(v.FloatString(32)) - if err != nil { - return nil, fmt.Errorf("bqHelper failed to parse as decimal %v", v) - } - return qvalue.QValueNumeric{Val: val}, nil + return qvalue.QValueNumeric{Val: decimal.NewFromBigRat(v, 32)}, nil case []uint8: return qvalue.QValueBytes{Val: v}, nil case []bigquery.Value: @@ -309,12 +305,12 @@ func toQValue(bqValue bigquery.Value) (qvalue.QValue, error) { } // bqSchemaToQRecordSchema converts a bigquery schema to a QRecordSchema. -func bqSchemaToQRecordSchema(schema bigquery.Schema) *qvalue.QRecordSchema { +func bqSchemaToQRecordSchema(schema bigquery.Schema) qvalue.QRecordSchema { fields := make([]qvalue.QField, 0, len(schema)) for _, fieldSchema := range schema { fields = append(fields, peer_bq.BigQueryFieldToQField(fieldSchema)) } - return &qvalue.QRecordSchema{Fields: fields} + return qvalue.QRecordSchema{Fields: fields} } func (b *BigQueryTestHelper) ExecuteAndProcessQuery(query string) (*model.QRecordBatch, error) { @@ -353,8 +349,8 @@ func (b *BigQueryTestHelper) ExecuteAndProcessQuery(query string) (*model.QRecor schema := bqSchemaToQRecordSchema(it.Schema) return &model.QRecordBatch{ - Records: records, Schema: schema, + Records: records, }, nil } diff --git a/flow/e2e/generic/generic_test.go b/flow/e2e/generic/generic_test.go index 97e64f5ed3..c175b17dbc 100644 --- a/flow/e2e/generic/generic_test.go +++ b/flow/e2e/generic/generic_test.go @@ -152,6 +152,7 @@ func (s Generic) Test_Simple_Schema_Changes() { } output, err := destinationSchemaConnector.GetTableSchema(context.Background(), &protos.GetTableSchemaBatchInput{ TableIdentifiers: []string{dstTableName}, + System: protos.TypeSystem_Q, }) e2e.EnvNoError(t, env, err) e2e.EnvTrue(t, env, e2e.CompareTableSchemas(expectedTableSchema, output.TableNameSchemaMapping[dstTableName])) @@ -195,6 +196,7 @@ func (s Generic) Test_Simple_Schema_Changes() { } output, err = destinationSchemaConnector.GetTableSchema(context.Background(), &protos.GetTableSchemaBatchInput{ TableIdentifiers: []string{dstTableName}, + System: protos.TypeSystem_Q, }) e2e.EnvNoError(t, env, err) e2e.EnvTrue(t, env, e2e.CompareTableSchemas(expectedTableSchema, output.TableNameSchemaMapping[dstTableName])) @@ -244,6 +246,7 @@ func (s Generic) Test_Simple_Schema_Changes() { } output, err = destinationSchemaConnector.GetTableSchema(context.Background(), &protos.GetTableSchemaBatchInput{ TableIdentifiers: []string{dstTableName}, + System: protos.TypeSystem_Q, }) e2e.EnvNoError(t, env, err) e2e.EnvTrue(t, env, e2e.CompareTableSchemas(expectedTableSchema, output.TableNameSchemaMapping[dstTableName])) @@ -293,6 +296,7 @@ func (s Generic) Test_Simple_Schema_Changes() { } output, err = destinationSchemaConnector.GetTableSchema(context.Background(), &protos.GetTableSchemaBatchInput{ TableIdentifiers: []string{dstTableName}, + System: protos.TypeSystem_Q, }) e2e.EnvNoError(t, env, err) e2e.EnvTrue(t, env, e2e.CompareTableSchemas(expectedTableSchema, output.TableNameSchemaMapping[dstTableName])) diff --git a/flow/e2e/postgres/peer_flow_pg_test.go b/flow/e2e/postgres/peer_flow_pg_test.go index fac0a270cb..b3dd3a2dc5 100644 --- a/flow/e2e/postgres/peer_flow_pg_test.go +++ b/flow/e2e/postgres/peer_flow_pg_test.go @@ -1025,3 +1025,51 @@ func (s PeerFlowE2ETestSuitePG) Test_Dynamic_Mirror_Config_Via_Signals() { env.Cancel() e2e.RequireEnvCanceled(s.t, env) } + +func (s PeerFlowE2ETestSuitePG) Test_TypeSystem_PG() { + srcTableName := s.attachSchemaSuffix("test_typesystem_pg") + dstTableName := s.attachSchemaSuffix("test_typesystem_pg_dst") + + _, err := s.Conn().Exec(context.Background(), fmt.Sprintf(` + create table %[1]s ( + id uuid not null primary key default gen_random_uuid(), + created_at timestamptz not null default now(), + updated_at timestamp, + j json, + jb jsonb, + aa32 integer[][], + currency char(3) + )`, srcTableName)) + require.NoError(s.t, err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_typesystem_pg"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + Destination: s.peer, + } + flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() + flowConnConfig.DoInitialSnapshot = true + flowConnConfig.System = protos.TypeSystem_PG + + tc := e2e.NewTemporalClient(s.t) + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + insert into %s (updated_at, j, jb, aa32, currency) values ( + NOW(),'{"b" : 123}','{"b" : 123}','{{3,2,1},{6,5,4},{9,8,7}}','ISK' + )`, srcTableName)) + e2e.EnvNoError(s.t, env, err) + + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize rows", func() bool { + err := s.comparePGTables(srcTableName, dstTableName, "id,created_at,updated_at,j::text,jb,aa32,currency") + if err != nil { + s.t.Log("PGPGPG", err) + } + return err == nil + }) + + env.Cancel() + e2e.RequireEnvCanceled(s.t, env) +} diff --git a/flow/e2e/snowflake/snowflake_schema_delta_test.go b/flow/e2e/snowflake/snowflake_schema_delta_test.go index d607e90451..5e4addec53 100644 --- a/flow/e2e/snowflake/snowflake_schema_delta_test.go +++ b/flow/e2e/snowflake/snowflake_schema_delta_test.go @@ -13,7 +13,10 @@ import ( "github.com/PeerDB-io/peer-flow/model/qvalue" ) -const schemaDeltaTestSchemaName = "PUBLIC" +const ( + schemaDeltaTestSchemaName = "PUBLIC" + numericAddedColumnTypeModifier int32 = 1048587 // Numeric(16,7) +) type SnowflakeSchemaDeltaTestSuite struct { t *testing.T @@ -53,10 +56,13 @@ func (s SnowflakeSchemaDeltaTestSuite) TestSimpleAddColumn() { err = s.connector.ReplayTableSchemaDeltas(context.Background(), "schema_delta_flow", []*protos.TableSchemaDelta{{ SrcTableName: tableName, DstTableName: tableName, - AddedColumns: []*protos.DeltaAddedColumn{{ - ColumnName: "HI", - ColumnType: string(qvalue.QValueKindJSON), - }}, + AddedColumns: []*protos.FieldDescription{ + { + Name: "HI", + Type: string(qvalue.QValueKindJSON), + TypeModifier: -1, + }, + }, }}) require.NoError(s.t, err) @@ -120,10 +126,9 @@ func (s SnowflakeSchemaDeltaTestSuite) TestAddAllColumnTypes() { TypeModifier: -1, }, { - Name: "C6", - Type: string(qvalue.QValueKindNumeric), - - TypeModifier: -1, + Name: "C6", + Type: string(qvalue.QValueKindNumeric), + TypeModifier: numericAddedColumnTypeModifier, // Numeric(16,7) }, { Name: "C7", @@ -147,13 +152,20 @@ func (s SnowflakeSchemaDeltaTestSuite) TestAddAllColumnTypes() { }, }, } - addedColumns := make([]*protos.DeltaAddedColumn, 0) + addedColumns := make([]*protos.FieldDescription, 0) for _, column := range expectedTableSchema.Columns { if column.Name != "ID" { - addedColumns = append(addedColumns, &protos.DeltaAddedColumn{ - ColumnName: column.Name, - ColumnType: column.Type, - }) + var typeModifierOfAddedCol int32 + typeModifierOfAddedCol = -1 + if column.Type == string(qvalue.QValueKindNumeric) { + typeModifierOfAddedCol = numericAddedColumnTypeModifier + } + addedColumns = append(addedColumns, &protos.FieldDescription{ + Name: column.Name, + Type: column.Type, + TypeModifier: typeModifierOfAddedCol, + }, + ) } } @@ -226,13 +238,15 @@ func (s SnowflakeSchemaDeltaTestSuite) TestAddTrickyColumnNames() { }, }, } - addedColumns := make([]*protos.DeltaAddedColumn, 0) + addedColumns := make([]*protos.FieldDescription, 0) for _, column := range expectedTableSchema.Columns { if column.Name != "ID" { - addedColumns = append(addedColumns, &protos.DeltaAddedColumn{ - ColumnName: column.Name, - ColumnType: column.Type, - }) + addedColumns = append(addedColumns, &protos.FieldDescription{ + Name: column.Name, + Type: column.Type, + TypeModifier: -1, + }, + ) } } @@ -280,13 +294,16 @@ func (s SnowflakeSchemaDeltaTestSuite) TestAddWhitespaceColumnNames() { }, }, } - addedColumns := make([]*protos.DeltaAddedColumn, 0) + + addedColumns := make([]*protos.FieldDescription, 0) for _, column := range expectedTableSchema.Columns { if column.Name != " " { - addedColumns = append(addedColumns, &protos.DeltaAddedColumn{ - ColumnName: column.Name, - ColumnType: column.Type, - }) + addedColumns = append(addedColumns, &protos.FieldDescription{ + Name: column.Name, + Type: column.Type, + TypeModifier: -1, + }, + ) } } diff --git a/flow/go.mod b/flow/go.mod index dfe2f1e6dc..88b1bf97b0 100644 --- a/flow/go.mod +++ b/flow/go.mod @@ -1,21 +1,21 @@ module github.com/PeerDB-io/peer-flow -go 1.22.0 +go 1.22.2 require ( cloud.google.com/go v0.112.2 cloud.google.com/go/bigquery v1.60.0 cloud.google.com/go/pubsub v1.37.0 cloud.google.com/go/storage v1.40.0 - github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.5.1 + github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.5.2 github.com/Azure/azure-sdk-for-go/sdk/messaging/azeventhubs v1.1.0 github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/eventhub/armeventhub v1.2.0 - github.com/ClickHouse/clickhouse-go/v2 v2.23.0 + github.com/ClickHouse/clickhouse-go/v2 v2.23.1 github.com/PeerDB-io/glua64 v1.0.1 github.com/PeerDB-io/gluabit32 v1.0.2 github.com/PeerDB-io/gluaflatbuffers v1.0.1 github.com/PeerDB-io/gluajson v1.0.2 - github.com/PeerDB-io/gluamsgpack v1.0.2 + github.com/PeerDB-io/gluamsgpack v1.0.4 github.com/PeerDB-io/gluautf8 v1.0.0 github.com/aws/aws-sdk-go-v2 v1.26.1 github.com/aws/aws-sdk-go-v2/config v1.27.11 @@ -25,6 +25,7 @@ require ( github.com/aws/aws-sdk-go-v2/service/ses v1.22.4 github.com/aws/aws-sdk-go-v2/service/sns v1.29.4 github.com/cockroachdb/pebble v1.1.0 + github.com/elastic/go-elasticsearch/v8 v8.13.1 github.com/google/uuid v1.6.0 github.com/grafana/pyroscope-go v1.1.1 github.com/grpc-ecosystem/grpc-gateway/v2 v2.19.1 @@ -33,12 +34,12 @@ require ( github.com/jackc/pgx/v5 v5.5.5 github.com/jmoiron/sqlx v1.3.5 github.com/joho/godotenv v1.5.1 - github.com/klauspost/compress v1.17.7 + github.com/klauspost/compress v1.17.8 github.com/lib/pq v1.10.9 github.com/linkedin/goavro/v2 v2.12.0 - github.com/microsoft/go-mssqldb v1.7.0 + github.com/microsoft/go-mssqldb v1.7.1 github.com/orcaman/concurrent-map/v2 v2.0.1 - github.com/shopspring/decimal v1.3.1 + github.com/shopspring/decimal v1.4.0 github.com/slack-go/slack v0.12.5 github.com/snowflakedb/gosnowflake v1.9.0 github.com/stretchr/testify v1.9.0 @@ -48,18 +49,20 @@ require ( github.com/urfave/cli/v3 v3.0.0-alpha9 github.com/youmark/pkcs8 v0.0.0-20201027041543-1326539a0a0a github.com/yuin/gopher-lua v1.1.1 - go.temporal.io/api v1.31.0 - go.temporal.io/sdk v1.26.0 + go.temporal.io/api v1.32.0 + go.temporal.io/sdk v1.26.1 go.uber.org/automaxprocs v1.5.3 golang.org/x/crypto v0.22.0 golang.org/x/sync v0.7.0 - google.golang.org/api v0.172.0 - google.golang.org/genproto/googleapis/api v0.0.0-20240401170217-c3f982113cda - google.golang.org/grpc v1.63.0 + google.golang.org/api v0.176.0 + google.golang.org/genproto/googleapis/api v0.0.0-20240415180920-8c6c420018be + google.golang.org/grpc v1.63.2 google.golang.org/protobuf v1.33.0 ) require ( + cloud.google.com/go/auth v0.2.2 // indirect + cloud.google.com/go/auth/oauth2adapt v0.2.1 // 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.5 // indirect @@ -79,7 +82,8 @@ require ( 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/dvsekhvalnov/jose2go v1.7.0 // indirect + github.com/elastic/elastic-transport-go/v8 v8.5.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.27.0 // indirect @@ -97,8 +101,8 @@ require ( github.com/pkg/errors v0.9.1 // indirect github.com/prometheus/client_golang v1.19.0 // indirect github.com/prometheus/client_model v0.6.1 // indirect - github.com/prometheus/common v0.52.2 // indirect - github.com/prometheus/procfs v0.13.0 // indirect + github.com/prometheus/common v0.53.0 // indirect + github.com/prometheus/procfs v0.14.0 // indirect github.com/rogpeppe/go-internal v1.12.0 // indirect github.com/segmentio/asm v1.2.0 // indirect github.com/sirupsen/logrus v1.9.3 // indirect @@ -112,12 +116,11 @@ require ( ) require ( - cloud.google.com/go/compute v1.25.1 // indirect - cloud.google.com/go/compute/metadata v0.2.3 // indirect + cloud.google.com/go/compute/metadata v0.3.0 // indirect cloud.google.com/go/iam v1.1.7 // indirect github.com/Azure/azure-sdk-for-go/sdk/azcore v1.11.1 // indirect - github.com/Azure/azure-sdk-for-go/sdk/internal v1.5.2 // indirect - github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.3.1 // indirect + github.com/Azure/azure-sdk-for-go/sdk/internal v1.6.0 // indirect + github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.3.2 // indirect github.com/Azure/go-amqp v1.0.5 // indirect github.com/AzureAD/microsoft-authentication-library-for-go v1.2.2 // indirect github.com/andybalholm/brotli v1.1.0 // indirect @@ -166,7 +169,7 @@ require ( github.com/xrash/smetrics v0.0.0-20240312152122-5f08fbb34913 // indirect github.com/zeebo/xxh3 v1.0.2 // indirect go.opencensus.io v0.24.0 // indirect - golang.org/x/exp v0.0.0-20240404231335-c0f41cb1a7a0 + golang.org/x/exp v0.0.0-20240416160154-fe59bbe5cc7f golang.org/x/mod v0.17.0 // indirect golang.org/x/net v0.24.0 // indirect golang.org/x/oauth2 v0.19.0 // indirect @@ -175,7 +178,7 @@ require ( golang.org/x/time v0.5.0 // indirect golang.org/x/tools v0.20.0 // indirect golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 // indirect - google.golang.org/genproto v0.0.0-20240401170217-c3f982113cda // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240401170217-c3f982113cda // indirect + google.golang.org/genproto v0.0.0-20240415180920-8c6c420018be // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240415180920-8c6c420018be // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/flow/go.sum b/flow/go.sum index e90024ec70..73a7c5df16 100644 --- a/flow/go.sum +++ b/flow/go.sum @@ -1,12 +1,15 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.112.2 h1:ZaGT6LiG7dBzi6zNOvVZwacaXlmf3lRqnC4DQzqyRQw= cloud.google.com/go v0.112.2/go.mod h1:iEqjp//KquGIJV/m+Pk3xecgKNhV+ry+vVTsy4TbDms= +cloud.google.com/go/auth v0.2.2 h1:gmxNJs4YZYcw6YvKRtVBaF2fyUE6UrWPyzU8jHvYfmI= +cloud.google.com/go/auth v0.2.2/go.mod h1:2bDNJWtWziDT3Pu1URxHHbkHE/BbOCuyUiKIGcNvafo= +cloud.google.com/go/auth/oauth2adapt v0.2.1 h1:VSPmMmUlT8CkIZ2PzD9AlLN+R3+D1clXMWHHa6vG/Ag= +cloud.google.com/go/auth/oauth2adapt v0.2.1/go.mod h1:tOdK/k+D2e4GEwfBRA48dKNQiDsqIXxLh7VU319eV0g= cloud.google.com/go/bigquery v1.60.0 h1:kA96WfgvCbkqfLnr7xI5uEfJ4h4FrnkdEb0yty0KSZo= cloud.google.com/go/bigquery v1.60.0/go.mod h1:Clwk2OeC0ZU5G5LDg7mo+h8U7KlAa5v06z5rptKdM3g= cloud.google.com/go/compute v1.25.1 h1:ZRpHJedLtTpKgr3RV1Fx23NuaAEN1Zfx9hw1u4aJdjU= -cloud.google.com/go/compute v1.25.1/go.mod h1:oopOIR53ly6viBYxaDhBfJwzUAxf1zE//uf3IB011ls= -cloud.google.com/go/compute/metadata v0.2.3 h1:mg4jlk7mCAj6xXp9UJ4fjI9VUI5rubuGBW5aJ7UnBMY= -cloud.google.com/go/compute/metadata v0.2.3/go.mod h1:VAV5nSsACxMJvgaAuX6Pk2AawlZn8kiOGuCv6gTkwuA= +cloud.google.com/go/compute/metadata v0.3.0 h1:Tz+eQXMEqDIKRsmY3cHTL6FVaynIjX2QxYC4trgAKZc= +cloud.google.com/go/compute/metadata v0.3.0/go.mod h1:zFmK7XCadkQkj6TtorcaGlCW1hT1fIilQDwofLpJ20k= cloud.google.com/go/datacatalog v1.20.0 h1:BGDsEjqpAo0Ka+b9yDLXnE5k+jU3lXGMh//NsEeDMIg= cloud.google.com/go/datacatalog v1.20.0/go.mod h1:fSHaKjIroFpmRrYlwz9XBB2gJBpXufpnxyAKaT4w6L0= cloud.google.com/go/iam v1.1.7 h1:z4VHOhwKLF/+UYXAJDFwGtNF0b6gjsW1Pk9Ml0U/IoM= @@ -25,10 +28,10 @@ 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.11.1 h1:E+OJmp2tPvt1W+amx48v1eqbjDYsgN+RzP4q16yV5eM= github.com/Azure/azure-sdk-for-go/sdk/azcore v1.11.1/go.mod h1:a6xsAQUZg+VsS3TJ05SRp524Hs4pZ/AeFSr5ENf0Yjo= -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.2 h1:LqbJ/WzJUwBf8UiaSzgX7aMclParm9/5Vgp+TY51uBQ= -github.com/Azure/azure-sdk-for-go/sdk/internal v1.5.2/go.mod h1:yInRyqWXAuaPrgI7p70+lDDgh3mlBohis29jGMISnmc= +github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.5.2 h1:FDif4R1+UUR+00q6wquyX90K7A8dN+R5E8GEadoP7sU= +github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.5.2/go.mod h1:aiYBYui4BJ/BJCAIKs92XiPyQfTaBWqvHujDwKb6CBU= +github.com/Azure/azure-sdk-for-go/sdk/internal v1.6.0 h1:sUFnFjzDUie80h24I7mrKtwCKgLY9L8h5Tp2x9+TWqk= +github.com/Azure/azure-sdk-for-go/sdk/internal v1.6.0/go.mod h1:52JbnQTp15qg5mRkMBHwp0j0ZFwHJ42Sx3zVV5RE9p0= github.com/Azure/azure-sdk-for-go/sdk/messaging/azeventhubs v1.1.0 h1:vEe09cdSBy7evqoVUvuitnsjyozsSzI4TbGgwu01+TI= github.com/Azure/azure-sdk-for-go/sdk/messaging/azeventhubs v1.1.0/go.mod h1:PgOlzIlvwIagKI8N6hCsfFDpAijHCmlHqOwA5GsSh9w= github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/eventhub/armeventhub v1.2.0 h1:+dggnR89/BIIlRlQ6d19dkhhdd/mQUiQbXhyHUFiB4w= @@ -43,8 +46,8 @@ github.com/Azure/azure-sdk-for-go/sdk/security/keyvault/azkeys v1.0.1 h1:MyVTgWR github.com/Azure/azure-sdk-for-go/sdk/security/keyvault/azkeys v1.0.1/go.mod h1:GpPjLhVR9dnUoJMyHWSPy71xY9/lcmpzIPZXmF0FCVY= github.com/Azure/azure-sdk-for-go/sdk/security/keyvault/internal v1.0.0 h1:D3occbWoio4EBLkbkevetNMAVX197GkzbUMtqjGWn80= github.com/Azure/azure-sdk-for-go/sdk/security/keyvault/internal v1.0.0/go.mod h1:bTSOgj05NGRuHHhQwAdPnYr9TOdNmKlZTgGLL6nyAdI= -github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.3.1 h1:fXPMAmuh0gDuRDey0atC8cXBuKIlqCzCkL8sm1n9Ov0= -github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.3.1/go.mod h1:SUZc9YRRHfx2+FAQKNDGrssXehqLpxmwRv2mC/5ntj4= +github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.3.2 h1:YUUxeiOWgdAQE3pXt2H7QXzZs0q8UBjgRbl56qo8GYM= +github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.3.2/go.mod h1:dmXQgZuiSubAecswZE+Sm8jkvEa7kQgTPVRvwL/nd0E= github.com/Azure/go-amqp v1.0.5 h1:po5+ljlcNSU8xtapHTe8gIc8yHxCzC03E8afH2g1ftU= github.com/Azure/go-amqp v1.0.5/go.mod h1:vZAogwdrkbyK3Mla8m/CxSc/aKdnTZ4IbPxl51Y5WZE= github.com/AzureAD/microsoft-authentication-library-for-go v1.2.2 h1:XHOnouVk1mxXfQidrMEnLlPk9UMeRtyBTnEFtxkV0kU= @@ -53,8 +56,8 @@ github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03 github.com/ClickHouse/ch-go v0.61.5 h1:zwR8QbYI0tsMiEcze/uIMK+Tz1D3XZXLdNrlaOpeEI4= github.com/ClickHouse/ch-go v0.61.5/go.mod h1:s1LJW/F/LcFs5HJnuogFMta50kKDO0lf9zzfrbl0RQg= github.com/ClickHouse/clickhouse-go v1.5.4 h1:cKjXeYLNWVJIx2J1K6H2CqyRmfwVJVY1OV1coaaFcI0= -github.com/ClickHouse/clickhouse-go/v2 v2.23.0 h1:srmRrkS0BR8gEut87u8jpcZ7geOob6nGj9ifrb+aKmg= -github.com/ClickHouse/clickhouse-go/v2 v2.23.0/go.mod h1:tBhdF3f3RdP7sS59+oBAtTyhWpy0024ZxDMhgxra0QE= +github.com/ClickHouse/clickhouse-go/v2 v2.23.1 h1:h+wOAjtycWeR8gNh0pKip+P4/Lyp9x9Ol5KyqaIJDeM= +github.com/ClickHouse/clickhouse-go/v2 v2.23.1/go.mod h1:aNap51J1OM3yxQJRgM+AlP/MPkGBCL8A74uQThoQhR0= github.com/DataDog/zstd v1.5.5 h1:oWf5W7GtOLgp6bciQYDmhHHjdhYkALu6S/5Ni9ZgSvQ= github.com/DataDog/zstd v1.5.5/go.mod h1:g4AWEaM3yOg3HYfnJ3YIawPnVdXJh9QME85blwSAmyw= github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c h1:RGWPOewvKIROun94nF7v2cua9qP+thov/7M50KEoeSU= @@ -67,8 +70,8 @@ github.com/PeerDB-io/gluaflatbuffers v1.0.1 h1:Oxlv0VlMYoQ05Q5n/k4hXAsvtDnuVNC99 github.com/PeerDB-io/gluaflatbuffers v1.0.1/go.mod h1:unZOM4Mm2Sn+aAFuVjoJDZ2Dji7jlDWrt4Hvq79as2g= github.com/PeerDB-io/gluajson v1.0.2 h1:Kv5Qabj2Md6gxRZsX5QVUOQDf5WMOQEF8lIkKXguajM= github.com/PeerDB-io/gluajson v1.0.2/go.mod h1:arRzpblxlLiWfBAluxP9Xibf6J8UkUIfoY4FPHTsz4Q= -github.com/PeerDB-io/gluamsgpack v1.0.2 h1:J5VhMSfJdWfCMJ1wszDNC2BVD4F+ATgXwGVs1lNft9g= -github.com/PeerDB-io/gluamsgpack v1.0.2/go.mod h1:1ufs5NK2DczzQS78Nhy0AkCA0dOVyt/KVEk39lbWzyU= +github.com/PeerDB-io/gluamsgpack v1.0.4 h1:JrZtdNAAkE6RtllVuhEuWWa26lQZ/K5BJWiI3q9EhL0= +github.com/PeerDB-io/gluamsgpack v1.0.4/go.mod h1:1ufs5NK2DczzQS78Nhy0AkCA0dOVyt/KVEk39lbWzyU= github.com/PeerDB-io/gluautf8 v1.0.0 h1:Ubhy6FVnrED5jrosdUOxzAkf3YdcgebYJzX2YBdGedE= github.com/PeerDB-io/gluautf8 v1.0.0/go.mod h1:+4RQlCVFCMikYFmiKUA9ADZftgGAZseMmdErxfE1EZQ= github.com/alecthomas/assert/v2 v2.6.0 h1:o3WJwILtexrEUk3cUVal3oiQY2tfgr/FHWiz/v2n4FU= @@ -156,8 +159,12 @@ github.com/djherbis/nio/v3 v3.0.1 h1:6wxhnuppteMa6RHA4L81Dq7ThkZH8SwnDzXDYy95vB4 github.com/djherbis/nio/v3 v3.0.1/go.mod h1:Ng4h80pbZFMla1yKzm61cF0tqqilXZYrogmWgZxOcmg= github.com/dnaeon/go-vcr v1.2.0 h1:zHCHvJYTMh1N7xnV7zf1m1GPBF9Ad0Jk/whtQ1663qI= github.com/dnaeon/go-vcr v1.2.0/go.mod h1:R4UdLID7HZT3taECzJs4YgbbH6PIGXB6W/sc5OLb6RQ= -github.com/dvsekhvalnov/jose2go v1.6.0 h1:Y9gnSnP4qEI0+/uQkHvFXeD2PLPJeXEL+ySMEA2EjTY= -github.com/dvsekhvalnov/jose2go v1.6.0/go.mod h1:QsHjhyTlD/lAVqn/NSbVZmSCGeDehTB/mPZadG+mhXU= +github.com/dvsekhvalnov/jose2go v1.7.0 h1:bnQc8+GMnidJZA8zc6lLEAb4xNrIqHwO+9TzqvtQZPo= +github.com/dvsekhvalnov/jose2go v1.7.0/go.mod h1:QsHjhyTlD/lAVqn/NSbVZmSCGeDehTB/mPZadG+mhXU= +github.com/elastic/elastic-transport-go/v8 v8.5.0 h1:v5membAl7lvQgBTexPRDBO/RdnlQX+FM9fUVDyXxvH0= +github.com/elastic/elastic-transport-go/v8 v8.5.0/go.mod h1:YLHer5cj0csTzNFXoNQ8qhtGY1GTvSqPnKWKaqQE3Hk= +github.com/elastic/go-elasticsearch/v8 v8.13.1 h1:du5F8IzUUyCkzxyHdrO9AtopcG95I/qwi2WK8Kf1xlg= +github.com/elastic/go-elasticsearch/v8 v8.13.1/go.mod h1:DIn7HopJs4oZC/w0WoJR13uMUxtHeq92eI5bqv5CRfI= github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= @@ -293,8 +300,8 @@ github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/compress v1.13.6/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= github.com/klauspost/compress v1.17.3/go.mod h1:/dCuZOvVtNoHsyb+cuJD3itjs3NbnF6KH9zAO4BDxPM= -github.com/klauspost/compress v1.17.7 h1:ehO88t2UGzQK66LMdE8tibEd1ErmzZjNEqWkjLAKQQg= -github.com/klauspost/compress v1.17.7/go.mod h1:Di0epgTjJY877eYKx5yC51cX2A2Vl2ibi7bDH9ttBbw= +github.com/klauspost/compress v1.17.8 h1:YcnTYrq7MikUT7k0Yb5eceMmALQPYBW/Xltxn0NAMnU= +github.com/klauspost/compress v1.17.8/go.mod h1:Di0epgTjJY877eYKx5yC51cX2A2Vl2ibi7bDH9ttBbw= github.com/klauspost/cpuid/v2 v2.2.7 h1:ZWSB3igEs+d0qvnxR/ZBzXVmxkgt8DdzP6m9pfuVLDM= github.com/klauspost/cpuid/v2 v2.2.7/go.mod h1:Lcz8mBdAVJIBVzewtcLocK12l3Y+JytZYpaMropDUws= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= @@ -314,8 +321,8 @@ 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/microsoft/go-mssqldb v1.7.0 h1:sgMPW0HA6Ihd37Yx0MzHyKD726C2kY/8KJsQtXHNaAs= -github.com/microsoft/go-mssqldb v1.7.0/go.mod h1:kOvZKUdrhhFQmxLZqbwUV0rHkNkZpthMITIb2Ko1IoA= +github.com/microsoft/go-mssqldb v1.7.1 h1:KU/g8aWeM3Hx7IMOFpiwYiUkU+9zeISb4+tx3ScVfsM= +github.com/microsoft/go-mssqldb v1.7.1/go.mod h1:kOvZKUdrhhFQmxLZqbwUV0rHkNkZpthMITIb2Ko1IoA= github.com/montanaflynn/stats v0.0.0-20171201202039-1bf9dbcd8cbe/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= github.com/mtibben/percent v0.2.1 h1:5gssi8Nqo8QU/r2pynCm+hBQHpkB/uNK7BJCFogWdzs= github.com/mtibben/percent v0.2.1/go.mod h1:KG9uO+SZkUp+VkRHsCdYQV3XSZrrSpR3O9ibNBTZrns= @@ -347,10 +354,10 @@ github.com/prometheus/client_golang v1.19.0/go.mod h1:ZRM9uEAypZakd+q/x7+gmsvXdU github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= -github.com/prometheus/common v0.52.2 h1:LW8Vk7BccEdONfrJBDffQGRtpSzi5CQaRZGtboOO2ck= -github.com/prometheus/common v0.52.2/go.mod h1:lrWtQx+iDfn2mbH5GUzlH9TSHyfZpHkSiG1W7y3sF2Q= -github.com/prometheus/procfs v0.13.0 h1:GqzLlQyfsPbaEHaQkO7tbDlriv/4o5Hudv6OXHGKX7o= -github.com/prometheus/procfs v0.13.0/go.mod h1:cd4PFCR54QLnGKPaKGA6l+cfuNXtht43ZKY6tow0Y1g= +github.com/prometheus/common v0.53.0 h1:U2pL9w9nmJwJDa4qqLQ3ZaePJ6ZTwt7cMD3AG3+aLCE= +github.com/prometheus/common v0.53.0/go.mod h1:BrxBKv3FWBIGXw89Mg1AeBq7FSyRzXWI3l3e7W3RN5U= +github.com/prometheus/procfs v0.14.0 h1:Lw4VdGGoKEZilJsayHf0B+9YgLGREba2C6xr+Fdfq6s= +github.com/prometheus/procfs v0.14.0/go.mod h1:XL+Iwz8k8ZabyZfMFHPiilCniixqQarAy5Mu67pHlNQ= github.com/robfig/cron v1.2.0 h1:ZjScXvvxeQ63Dbyxy76Fj3AT3Ut0aKsyd2/tl3DTMuQ= github.com/robfig/cron v1.2.0/go.mod h1:JGuDeoQd7Z6yL4zQhZ3OPEVHB7fL6Ka6skscFHfmt2k= github.com/rogpeppe/go-internal v1.9.0/go.mod h1:WtVeX8xhTBvf0smdhujwtBcq4Qrzq/fJaraNFVN+nFs= @@ -358,8 +365,8 @@ github.com/rogpeppe/go-internal v1.12.0 h1:exVL4IDcn6na9z1rAb56Vxr+CgyK3nn3O+epU github.com/rogpeppe/go-internal v1.12.0/go.mod h1:E+RYuTGaKKdloAfM02xzb0FW3Paa99yedzYV+kq4uf4= github.com/segmentio/asm v1.2.0 h1:9BQrFxC+YOHJlTlHGkTrFWf59nbL3XnCoFLTwDCI7ys= github.com/segmentio/asm v1.2.0/go.mod h1:BqMnlJP91P8d+4ibuonYZw9mfnzI9HfxselHZr5aAcs= -github.com/shopspring/decimal v1.3.1 h1:2Usl1nmF/WZucqkFZhnfFYxxxu8LG21F6nPQBE5gKV8= -github.com/shopspring/decimal v1.3.1/go.mod h1:DKyhrW/HYNuLGql+MJL6WCR6knT2jwCFRcu2hWCYk4o= +github.com/shopspring/decimal v1.4.0 h1:bxl37RwXBklmTi0C79JfXCEBD1cqqHt0bbgBAGFp81k= +github.com/shopspring/decimal v1.4.0/go.mod h1:gawqmDU56v4yIKSwfBSFip1HdCCXN8/+DMd9qYNcwME= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/sirupsen/logrus v1.9.3 h1:dueUQJ1C2q9oE3F7wvmSGAaVtTmUizReu6fjN8uqzbQ= github.com/sirupsen/logrus v1.9.3/go.mod h1:naHLuLoDiP4jHNo9R0sCBMtWGeIprob74mVsIT4qYEQ= @@ -429,12 +436,10 @@ go.opentelemetry.io/otel/sdk v1.24.0 h1:YMPPDNymmQN3ZgczicBY3B6sf9n62Dlj9pWD3ucg go.opentelemetry.io/otel/sdk v1.24.0/go.mod h1:KVrIYw6tEubO9E96HQpcmpTKDVn9gdv35HoYiQWGDFg= go.opentelemetry.io/otel/trace v1.25.0 h1:tqukZGLwQYRIFtSQM2u2+yfMVTgGVeqRLPUYx1Dq6RM= go.opentelemetry.io/otel/trace v1.25.0/go.mod h1:hCCs70XM/ljO+BeQkyFnbK28SBIJ/Emuha+ccrCRT7I= -go.temporal.io/api v1.30.1 h1:73UCTi+8l+Qy3GdDypW2FB5rj995A3Pi0mXkSu/qedw= -go.temporal.io/api v1.30.1/go.mod h1:xI9UdP3s07881dgWzG8idIBAnZq3/aop+O682EIDoT0= -go.temporal.io/api v1.31.0 h1:iIBX0XjUaOxZw6c6qzT47YAA/UN1zyF/bQttx7UdfIA= -go.temporal.io/api v1.31.0/go.mod h1:8EIvKai87mimizN6LZ4t5Zh4NdvjZa42kZs0xL2/RP0= -go.temporal.io/sdk v1.26.0 h1:QAi7irgKvJI+5cKmvy+1lkdCDJJDDNpIQAoXdr3dcyM= -go.temporal.io/sdk v1.26.0/go.mod h1:rcAf1YWlbWgMsjJEuz7XiQd6UYxTQDOk2AqRRIDwq/U= +go.temporal.io/api v1.32.0 h1:Jv0FieWDq0HJVqoHRE/kRHM+tIaRtR16RbXZZl+8Qb4= +go.temporal.io/api v1.32.0/go.mod h1:MClRjMCgXZTKmxyItEJPRR5NuJRBhSEpuF9wuh97N6U= +go.temporal.io/sdk v1.26.1 h1:ggmFBythnuuW3yQRp0VzOTrmbOf+Ddbe00TZl+CQ+6U= +go.temporal.io/sdk v1.26.1/go.mod h1:ph3K/74cry+JuSV9nJH+Q+Zeir2ddzoX2LjWL/e5yCo= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/automaxprocs v1.5.3 h1:kWazyxZUrS3Gs4qUpbwo5kEIMGe/DAvi5Z4tl2NW4j8= go.uber.org/automaxprocs v1.5.3/go.mod h1:eRbA25aqJrxAbsLO0xy5jVwPt7FQnRgjW+efnwa1WM0= @@ -449,8 +454,8 @@ golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0 golang.org/x/crypto v0.22.0 h1:g1v0xeRhjcugydODzvb3mEM9SQ0HGp9s/nh3COQ/C30= golang.org/x/crypto v0.22.0/go.mod h1:vr6Su+7cTlO45qkww3VDJlzDn0ctJvRgYbC2NvXHt+M= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20240404231335-c0f41cb1a7a0 h1:985EYyeCOxTpcgOTJpflJUwOeEz0CQOdPt73OzpE9F8= -golang.org/x/exp v0.0.0-20240404231335-c0f41cb1a7a0/go.mod h1:/lliqkxwWAhPjf5oSOIJup2XcqJaw8RGS6k3TGEc7GI= +golang.org/x/exp v0.0.0-20240416160154-fe59bbe5cc7f h1:99ci1mjWVBWwJiEKYY6jWa4d2nTQVIEhZIptnrVb1XY= +golang.org/x/exp v0.0.0-20240416160154-fe59bbe5cc7f/go.mod h1:/lliqkxwWAhPjf5oSOIJup2XcqJaw8RGS6k3TGEc7GI= 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= @@ -531,28 +536,28 @@ golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 h1:+cNy6SZtPcJQH3LJVLOSm golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028/go.mod h1:NDW/Ps6MPRej6fsCIbMTohpP40sJ/P/vI1MoTEGwX90= 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.172.0 h1:/1OcMZGPmW1rX2LCu2CmGUD1KXK1+pfzxotxyRUCCdk= -google.golang.org/api v0.172.0/go.mod h1:+fJZq6QXWfa9pXhnIzsjx4yI22d4aI9ZpLb58gvXjis= +google.golang.org/api v0.176.0 h1:dHj1/yv5Dm/eQTXiP9hNCRT3xzJHWXeNdRq29XbMxoE= +google.golang.org/api v0.176.0/go.mod h1:Rra+ltKu14pps/4xTycZfobMgLpbosoaaL7c+SEMrO8= 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/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= -google.golang.org/genproto v0.0.0-20240401170217-c3f982113cda h1:wu/KJm9KJwpfHWhkkZGohVC6KRrc1oJNr4jwtQMOQXw= -google.golang.org/genproto v0.0.0-20240401170217-c3f982113cda/go.mod h1:g2LLCvCeCSir/JJSWosk19BR4NVxGqHUC6rxIRsd7Aw= -google.golang.org/genproto/googleapis/api v0.0.0-20240401170217-c3f982113cda h1:b6F6WIV4xHHD0FA4oIyzU6mHWg2WI2X1RBehwa5QN38= -google.golang.org/genproto/googleapis/api v0.0.0-20240401170217-c3f982113cda/go.mod h1:AHcE/gZH76Bk/ROZhQphlRoWo5xKDEtz3eVEO1LfA8c= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240401170217-c3f982113cda h1:LI5DOvAxUPMv/50agcLLoo+AdWc1irS9Rzz4vPuD1V4= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240401170217-c3f982113cda/go.mod h1:WtryC6hu0hhx87FDGxWCDptyssuo68sk10vYjF+T9fY= +google.golang.org/genproto v0.0.0-20240415180920-8c6c420018be h1:g4aX8SUFA8V5F4LrSY5EclyGYw1OZN4HS1jTyjB9ZDc= +google.golang.org/genproto v0.0.0-20240415180920-8c6c420018be/go.mod h1:FeSdT5fk+lkxatqJP38MsUicGqHax5cLtmy/6TAuxO4= +google.golang.org/genproto/googleapis/api v0.0.0-20240415180920-8c6c420018be h1:Zz7rLWqp0ApfsR/l7+zSHhY3PMiH2xqgxlfYfAfNpoU= +google.golang.org/genproto/googleapis/api v0.0.0-20240415180920-8c6c420018be/go.mod h1:dvdCTIoAGbkWbcIKBniID56/7XHTt6WfxXNMxuziJ+w= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240415180920-8c6c420018be h1:LG9vZxsWGOmUKieR8wPAUR3u3MpnYFQZROPIMaXh7/A= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240415180920-8c6c420018be/go.mod h1:WtryC6hu0hhx87FDGxWCDptyssuo68sk10vYjF+T9fY= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= google.golang.org/grpc v1.33.2/go.mod h1:JMHMWHQWaTccqQQlmk3MJZS+GWXOdAesneDmEnv2fbc= -google.golang.org/grpc v1.63.0 h1:WjKe+dnvABXyPJMD7KDNLxtoGk5tgk+YFWN6cBWjZE8= -google.golang.org/grpc v1.63.0/go.mod h1:WAX/8DgncnokcFUldAxq7GeB5DXHDbMF+lLvDomNkRA= +google.golang.org/grpc v1.63.2 h1:MUeiw1B2maTVZthpU5xvASfTh3LDbxHd6IJ6QQVU+xM= +google.golang.org/grpc v1.63.2/go.mod h1:WAX/8DgncnokcFUldAxq7GeB5DXHDbMF+lLvDomNkRA= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= diff --git a/flow/model/cdc_record_stream.go b/flow/model/cdc_stream.go similarity index 58% rename from flow/model/cdc_record_stream.go rename to flow/model/cdc_stream.go index 47070512f4..72eb1ad490 100644 --- a/flow/model/cdc_record_stream.go +++ b/flow/model/cdc_stream.go @@ -8,23 +8,21 @@ import ( "github.com/PeerDB-io/peer-flow/shared" ) -type CDCRecordStream struct { +type CDCStream[T Items] struct { // empty signal to indicate if the records are going to be empty or not. emptySignal chan bool - // Records are a list of json objects. - records chan Record - // Schema changes from the slot - SchemaDeltas []*protos.TableSchemaDelta - // Indicates if the last checkpoint has been set. + records chan Record[T] + // Schema changes from slot + SchemaDeltas []*protos.TableSchemaDelta lastCheckpointSet bool // lastCheckpointID is the last ID of the commit that corresponds to this batch. lastCheckpointID atomic.Int64 } -func NewCDCRecordStream() *CDCRecordStream { +func NewCDCStream[T Items]() *CDCStream[T] { channelBuffer := peerdbenv.PeerDBCDCChannelBufferSize() - return &CDCRecordStream{ - records: make(chan Record, channelBuffer), + return &CDCStream[T]{ + records: make(chan Record[T], channelBuffer), SchemaDeltas: make([]*protos.TableSchemaDelta, 0), emptySignal: make(chan bool, 1), lastCheckpointSet: false, @@ -32,35 +30,35 @@ func NewCDCRecordStream() *CDCRecordStream { } } -func (r *CDCRecordStream) UpdateLatestCheckpoint(val int64) { +func (r *CDCStream[T]) UpdateLatestCheckpoint(val int64) { shared.AtomicInt64Max(&r.lastCheckpointID, val) } -func (r *CDCRecordStream) GetLastCheckpoint() int64 { +func (r *CDCStream[T]) GetLastCheckpoint() int64 { if !r.lastCheckpointSet { panic("last checkpoint not set, stream is still active") } return r.lastCheckpointID.Load() } -func (r *CDCRecordStream) AddRecord(record Record) { +func (r *CDCStream[T]) AddRecord(record Record[T]) { r.records <- record } -func (r *CDCRecordStream) SignalAsEmpty() { +func (r *CDCStream[T]) SignalAsEmpty() { r.emptySignal <- true } -func (r *CDCRecordStream) SignalAsNotEmpty() { +func (r *CDCStream[T]) SignalAsNotEmpty() { r.emptySignal <- false } -func (r *CDCRecordStream) WaitAndCheckEmpty() bool { +func (r *CDCStream[T]) WaitAndCheckEmpty() bool { isEmpty := <-r.emptySignal return isEmpty } -func (r *CDCRecordStream) Close() { +func (r *CDCStream[T]) Close() { if !r.lastCheckpointSet { close(r.emptySignal) close(r.records) @@ -68,11 +66,12 @@ func (r *CDCRecordStream) Close() { } } -func (r *CDCRecordStream) GetRecords() <-chan Record { +func (r *CDCStream[T]) GetRecords() <-chan Record[T] { return r.records } -func (r *CDCRecordStream) AddSchemaDelta(tableNameMapping map[string]NameAndExclude, +func (r *CDCStream[T]) AddSchemaDelta( + tableNameMapping map[string]NameAndExclude, delta *protos.TableSchemaDelta, ) { r.SchemaDeltas = append(r.SchemaDeltas, delta) diff --git a/flow/model/conversion_avro.go b/flow/model/conversion_avro.go index 17a3f2b3f3..8f52c44611 100644 --- a/flow/model/conversion_avro.go +++ b/flow/model/conversion_avro.go @@ -69,7 +69,7 @@ type QRecordAvroSchemaDefinition struct { func GetAvroSchemaDefinition( dstTableName string, - qRecordSchema *qvalue.QRecordSchema, + qRecordSchema qvalue.QRecordSchema, targetDWH protos.DBType, ) (*QRecordAvroSchemaDefinition, error) { avroFields := make([]QRecordAvroField, 0, len(qRecordSchema.Fields)) diff --git a/flow/model/model.go b/flow/model/model.go index 2dfb08e36e..54c2ef41ce 100644 --- a/flow/model/model.go +++ b/flow/model/model.go @@ -1,6 +1,7 @@ package model import ( + "sync/atomic" "time" "github.com/jackc/pglogrepl" @@ -24,9 +25,11 @@ func NewNameAndExclude(name string, exclude []string) NameAndExclude { return NameAndExclude{Name: name, Exclude: exset} } -type PullRecordsRequest struct { +type PullRecordsRequest[T Items] struct { // record batch for pushing changes into - RecordStream *CDCRecordStream + RecordStream *CDCStream[T] + // ConsumedOffset can be reported as committed to reduce slot size + ConsumedOffset *atomic.Int64 // FlowJobName is the name of the flow job. FlowJobName string // relId to name Mapping @@ -72,8 +75,10 @@ type TableWithPkey struct { PkeyColVal [32]byte } -type SyncRecordsRequest struct { - Records *CDCRecordStream +type SyncRecordsRequest[T Items] struct { + Records *CDCStream[T] + // ConsumedOffset allows destination to confirm lsn for slot + ConsumedOffset *atomic.Int64 // FlowJobName is the name of the flow job. FlowJobName string // Staging path for AVRO files in CDC diff --git a/flow/model/pg_items.go b/flow/model/pg_items.go new file mode 100644 index 0000000000..e607ad6964 --- /dev/null +++ b/flow/model/pg_items.go @@ -0,0 +1,77 @@ +package model + +import ( + "encoding/json" + "fmt" + + "github.com/PeerDB-io/peer-flow/shared" +) + +// encoding/gob cannot encode unexported fields +type PgItems struct { + ColToVal map[string][]byte +} + +func NewPgItems(capacity int) PgItems { + return PgItems{ + ColToVal: make(map[string][]byte, capacity), + } +} + +func (r PgItems) AddColumn(col string, val []byte) { + r.ColToVal[col] = val +} + +func (r PgItems) GetColumnValue(col string) []byte { + return r.ColToVal[col] +} + +// UpdateIfNotExists takes in a RecordItems as input and updates the values of the +// current RecordItems with the values from the input RecordItems for the columns +// that are present in the input RecordItems but not in the current RecordItems. +// We return the slice of col names that were updated. +func (r PgItems) UpdateIfNotExists(input_ Items) []string { + input := input_.(PgItems) + updatedCols := make([]string, 0, len(input.ColToVal)) + for col, val := range input.ColToVal { + if _, ok := r.ColToVal[col]; !ok { + r.ColToVal[col] = val + updatedCols = append(updatedCols, col) + } + } + return updatedCols +} + +func (r PgItems) GetBytesByColName(colName string) ([]byte, error) { + val, ok := r.ColToVal[colName] + if !ok { + return nil, fmt.Errorf("column name %s not found", colName) + } + return val, nil +} + +func (r PgItems) Len() int { + return len(r.ColToVal) +} + +func (r PgItems) ToJSONWithOptions(options ToJSONOptions) (string, error) { + bytes, err := r.MarshalJSON() + return shared.UnsafeFastReadOnlyBytesToString(bytes), err +} + +func (r PgItems) ToJSON() (string, error) { + return r.ToJSONWithOptions(NewToJSONOptions(nil, true)) +} + +func (r PgItems) MarshalJSON() ([]byte, error) { + jsonStruct := make(map[string]interface{}, len(r.ColToVal)) + for col, bytes := range r.ColToVal { + if bytes == nil { + jsonStruct[col] = nil + } else { + jsonStruct[col] = shared.UnsafeFastReadOnlyBytesToString(bytes) + } + } + + return json.Marshal(jsonStruct) +} diff --git a/flow/model/qrecord_batch.go b/flow/model/qrecord_batch.go index 7c28c64d89..1f787b5c1c 100644 --- a/flow/model/qrecord_batch.go +++ b/flow/model/qrecord_batch.go @@ -3,7 +3,6 @@ package model import ( "errors" "fmt" - "log/slog" "strings" "time" @@ -16,7 +15,7 @@ import ( // QRecordBatch holds a batch of []QValue slices type QRecordBatch struct { - Schema *qvalue.QRecordSchema + Schema qvalue.QRecordSchema Records [][]qvalue.QValue } @@ -27,13 +26,10 @@ func (q *QRecordBatch) ToQRecordStream(buffer int) *QRecordStream { } func (q *QRecordBatch) FeedToQRecordStream(stream *QRecordStream) { - err := stream.SetSchema(q.Schema) - if err != nil { - slog.Warn(err.Error()) - } + stream.SetSchema(q.Schema) for _, record := range q.Records { - stream.Records <- QRecordOrError{Record: record} + stream.Records <- record } close(stream.Records) } @@ -53,7 +49,7 @@ func constructArray[T any](qValue qvalue.QValue, typeName string) (*pgtype.Array type QRecordBatchCopyFromSource struct { err error stream *QRecordStream - currentRecord QRecordOrError + currentRecord []qvalue.QValue numRecords int } @@ -63,7 +59,7 @@ func NewQRecordBatchCopyFromSource( return &QRecordBatchCopyFromSource{ numRecords: 0, stream: stream, - currentRecord: QRecordOrError{}, + currentRecord: nil, err: nil, } } @@ -71,6 +67,7 @@ func NewQRecordBatchCopyFromSource( func (src *QRecordBatchCopyFromSource) Next() bool { rec, ok := <-src.stream.Records if !ok { + src.err = src.stream.Err() return false } @@ -80,16 +77,12 @@ func (src *QRecordBatchCopyFromSource) Next() bool { } func (src *QRecordBatchCopyFromSource) Values() ([]interface{}, error) { - if src.currentRecord.Err != nil { - src.err = src.currentRecord.Err + if src.err != nil { return nil, src.err } - record := src.currentRecord.Record - numEntries := len(record) - - values := make([]interface{}, numEntries) - for i, qValue := range record { + values := make([]interface{}, len(src.currentRecord)) + for i, qValue := range src.currentRecord { if qValue.Value() == nil { values[i] = nil continue @@ -115,11 +108,10 @@ func (src *QRecordBatchCopyFromSource) Values() ([]interface{}, error) { case qvalue.QValueCIDR, qvalue.QValueINET, qvalue.QValueMacaddr: str, ok := v.Value().(string) if !ok { - src.err = errors.New("invalid INET/CIDR value") + src.err = errors.New("invalid INET/CIDR/MACADDR value") return nil, src.err } values[i] = str - case qvalue.QValueTime: values[i] = pgtype.Time{Microseconds: v.Val.UnixMicro(), Valid: true} case qvalue.QValueTimestamp: @@ -130,14 +122,10 @@ func (src *QRecordBatchCopyFromSource) Values() ([]interface{}, error) { values[i] = uuid.UUID(v.Val) case qvalue.QValueNumeric: values[i] = v.Val - case qvalue.QValueBytes, qvalue.QValueBit: - bytes, ok := v.Value().([]byte) - if !ok { - src.err = errors.New("invalid Bytes value") - return nil, src.err - } - values[i] = bytes - + case qvalue.QValueBit: + values[i] = v.Val + case qvalue.QValueBytes: + values[i] = v.Val case qvalue.QValueDate: values[i] = pgtype.Date{Time: v.Val, Valid: true} case qvalue.QValueHStore: diff --git a/flow/model/qrecord_stream.go b/flow/model/qrecord_stream.go index b0d6076294..0a0c26d2e4 100644 --- a/flow/model/qrecord_stream.go +++ b/flow/model/qrecord_stream.go @@ -1,8 +1,6 @@ package model import ( - "errors" - "github.com/PeerDB-io/peer-flow/model/qvalue" ) @@ -12,80 +10,77 @@ type RecordTypeCounts struct { DeleteCount int } -type QRecordOrError struct { - Err error - Record []qvalue.QValue -} - -type QRecordSchemaOrError struct { - Schema *qvalue.QRecordSchema - Err error -} - type QRecordStream struct { - schema chan QRecordSchemaOrError - Records chan QRecordOrError - schemaCache *qvalue.QRecordSchema + schemaLatch chan struct{} + Records chan []qvalue.QValue + err error + schema qvalue.QRecordSchema schemaSet bool } -type RecordsToStreamRequest struct { - records <-chan Record +type RecordsToStreamRequest[T Items] struct { + records <-chan Record[T] TableMapping map[string]*RecordTypeCounts BatchID int64 } -func NewRecordsToStreamRequest( - records <-chan Record, +func NewRecordsToStreamRequest[T Items]( + records <-chan Record[T], tableMapping map[string]*RecordTypeCounts, batchID int64, -) *RecordsToStreamRequest { - return &RecordsToStreamRequest{ +) *RecordsToStreamRequest[T] { + return &RecordsToStreamRequest[T]{ records: records, TableMapping: tableMapping, BatchID: batchID, } } -func (r *RecordsToStreamRequest) GetRecords() <-chan Record { +func (r *RecordsToStreamRequest[T]) GetRecords() <-chan Record[T] { return r.records } func NewQRecordStream(buffer int) *QRecordStream { return &QRecordStream{ - schema: make(chan QRecordSchemaOrError, 1), - Records: make(chan QRecordOrError, buffer), + schemaLatch: make(chan struct{}), + Records: make(chan []qvalue.QValue, buffer), + schema: qvalue.QRecordSchema{}, + err: nil, schemaSet: false, - schemaCache: nil, } } -func (s *QRecordStream) Schema() (*qvalue.QRecordSchema, error) { - if s.schemaCache != nil { - return s.schemaCache, nil - } - - schemaOrError := <-s.schema - s.schemaCache = schemaOrError.Schema - return schemaOrError.Schema, schemaOrError.Err +func (s *QRecordStream) Schema() qvalue.QRecordSchema { + <-s.schemaLatch + return s.schema } -func (s *QRecordStream) SetSchema(schema *qvalue.QRecordSchema) error { - if s.schemaSet { - return errors.New("Schema already set") +func (s *QRecordStream) SetSchema(schema qvalue.QRecordSchema) { + if !s.schemaSet { + s.schema = schema + close(s.schemaLatch) + s.schemaSet = true } - - s.schema <- QRecordSchemaOrError{ - Schema: schema, - } - s.schemaSet = true - return nil } func (s *QRecordStream) IsSchemaSet() bool { return s.schemaSet } -func (s *QRecordStream) SchemaChan() <-chan QRecordSchemaOrError { - return s.schema +func (s *QRecordStream) SchemaChan() <-chan struct{} { + return s.schemaLatch +} + +func (s *QRecordStream) Err() error { + return s.err +} + +// Set error & close stream. Calling with multiple errors only tracks first error & does not panic. +// Close(nil) after an error won't panic, but Close after Close(nil) will panic, +// this is enough to be able to safely `defer stream.Close(nil)`. +func (s *QRecordStream) Close(err error) { + if s.err == nil { + s.err = err + close(s.Records) + } } diff --git a/flow/model/qvalue/equals.go b/flow/model/qvalue/equals.go index 20e7392906..97ba7ca73f 100644 --- a/flow/model/qvalue/equals.go +++ b/flow/model/qvalue/equals.go @@ -3,7 +3,6 @@ package qvalue import ( "bytes" "math" - "math/big" "reflect" "slices" "strconv" @@ -463,7 +462,7 @@ func getDecimal(v interface{}) (decimal.Decimal, bool) { case int64: return decimal.NewFromInt(value), true case uint64: - return decimal.NewFromBigInt(new(big.Int).SetUint64(value), 0), true + return decimal.NewFromUint64(value), true case float32: return decimal.NewFromFloat32(value), true case int32: diff --git a/flow/model/qvalue/qschema.go b/flow/model/qvalue/qschema.go index 3c121f28c5..a956968ac1 100644 --- a/flow/model/qvalue/qschema.go +++ b/flow/model/qvalue/qschema.go @@ -17,16 +17,12 @@ type QRecordSchema struct { } // NewQRecordSchema creates a new QRecordSchema. -func NewQRecordSchema(fields []QField) *QRecordSchema { - return &QRecordSchema{Fields: fields} +func NewQRecordSchema(fields []QField) QRecordSchema { + return QRecordSchema{Fields: fields} } // EqualNames returns true if the field names are equal. -func (q *QRecordSchema) EqualNames(other *QRecordSchema) bool { - if other == nil { - return q == nil - } - +func (q QRecordSchema) EqualNames(other QRecordSchema) bool { if len(q.Fields) != len(other.Fields) { return false } @@ -44,7 +40,7 @@ func (q *QRecordSchema) EqualNames(other *QRecordSchema) bool { } // GetColumnNames returns a slice of column names. -func (q *QRecordSchema) GetColumnNames() []string { +func (q QRecordSchema) GetColumnNames() []string { names := make([]string, 0, len(q.Fields)) for _, field := range q.Fields { names = append(names, field.Name) diff --git a/flow/model/qvalue/qvalue.go b/flow/model/qvalue/qvalue.go index 40ac4b6fc5..91b9e3fe31 100644 --- a/flow/model/qvalue/qvalue.go +++ b/flow/model/qvalue/qvalue.go @@ -1,15 +1,22 @@ package qvalue import ( + "encoding/json" "time" + "github.com/google/uuid" "github.com/shopspring/decimal" + "github.com/yuin/gopher-lua" + + "github.com/PeerDB-io/glua64" + "github.com/PeerDB-io/peer-flow/shared" ) -// if new types are added, register them in gob - cdc_records_storage.go +// if new types are added, register them in gob - cdc_store.go type QValue interface { Kind() QValueKind Value() any + LValue(ls *lua.LState) lua.LValue } type QValueNull QValueKind @@ -22,6 +29,10 @@ func (QValueNull) Value() any { return nil } +func (QValueNull) LValue(ls *lua.LState) lua.LValue { + return lua.LNil +} + type QValueInvalid struct { Val string } @@ -34,6 +45,10 @@ func (v QValueInvalid) Value() any { return v.Val } +func (v QValueInvalid) LValue(ls *lua.LState) lua.LValue { + return lua.LString(v.Val) +} + type QValueFloat32 struct { Val float32 } @@ -46,6 +61,10 @@ func (v QValueFloat32) Value() any { return v.Val } +func (v QValueFloat32) LValue(ls *lua.LState) lua.LValue { + return lua.LNumber(v.Val) +} + type QValueFloat64 struct { Val float64 } @@ -58,6 +77,10 @@ func (v QValueFloat64) Value() any { return v.Val } +func (v QValueFloat64) LValue(ls *lua.LState) lua.LValue { + return lua.LNumber(v.Val) +} + type QValueInt16 struct { Val int16 } @@ -70,6 +93,10 @@ func (v QValueInt16) Value() any { return v.Val } +func (v QValueInt16) LValue(ls *lua.LState) lua.LValue { + return lua.LNumber(v.Val) +} + type QValueInt32 struct { Val int32 } @@ -82,6 +109,10 @@ func (v QValueInt32) Value() any { return v.Val } +func (v QValueInt32) LValue(ls *lua.LState) lua.LValue { + return lua.LNumber(v.Val) +} + type QValueInt64 struct { Val int64 } @@ -94,6 +125,10 @@ func (v QValueInt64) Value() any { return v.Val } +func (v QValueInt64) LValue(ls *lua.LState) lua.LValue { + return glua64.I64.New(ls, v.Val) +} + type QValueBoolean struct { Val bool } @@ -106,6 +141,10 @@ func (v QValueBoolean) Value() any { return v.Val } +func (v QValueBoolean) LValue(ls *lua.LState) lua.LValue { + return lua.LBool(v.Val) +} + type QValueStruct struct { Val map[string]interface{} } @@ -118,6 +157,15 @@ func (v QValueStruct) Value() any { return v.Val } +func (v QValueStruct) LValue(ls *lua.LState) lua.LValue { + bytes, err := json.Marshal(v.Val) + if err != nil { + return lua.LString(err.Error()) + } else { + return lua.LString(shared.UnsafeFastReadOnlyBytesToString(bytes)) + } +} + type QValueQChar struct { Val uint8 } @@ -130,6 +178,10 @@ func (v QValueQChar) Value() any { return v.Val } +func (v QValueQChar) LValue(ls *lua.LState) lua.LValue { + return lua.LString(v.Val) +} + type QValueString struct { Val string } @@ -142,6 +194,10 @@ func (v QValueString) Value() any { return v.Val } +func (v QValueString) LValue(ls *lua.LState) lua.LValue { + return lua.LString(v.Val) +} + type QValueTimestamp struct { Val time.Time } @@ -154,6 +210,10 @@ func (v QValueTimestamp) Value() any { return v.Val } +func (v QValueTimestamp) LValue(ls *lua.LState) lua.LValue { + return shared.LuaTime.New(ls, v.Val) +} + type QValueTimestampTZ struct { Val time.Time } @@ -166,6 +226,10 @@ func (v QValueTimestampTZ) Value() any { return v.Val } +func (v QValueTimestampTZ) LValue(ls *lua.LState) lua.LValue { + return shared.LuaTime.New(ls, v.Val) +} + type QValueDate struct { Val time.Time } @@ -178,6 +242,10 @@ func (v QValueDate) Value() any { return v.Val } +func (v QValueDate) LValue(ls *lua.LState) lua.LValue { + return shared.LuaTime.New(ls, v.Val) +} + type QValueTime struct { Val time.Time } @@ -190,6 +258,10 @@ func (v QValueTime) Value() any { return v.Val } +func (v QValueTime) LValue(ls *lua.LState) lua.LValue { + return shared.LuaTime.New(ls, v.Val) +} + type QValueTimeTZ struct { Val time.Time } @@ -202,6 +274,10 @@ func (v QValueTimeTZ) Value() any { return v.Val } +func (v QValueTimeTZ) LValue(ls *lua.LState) lua.LValue { + return shared.LuaTime.New(ls, v.Val) +} + type QValueInterval struct { Val string } @@ -214,6 +290,10 @@ func (v QValueInterval) Value() any { return v.Val } +func (v QValueInterval) LValue(ls *lua.LState) lua.LValue { + return lua.LString(v.Val) +} + type QValueNumeric struct { Val decimal.Decimal } @@ -226,6 +306,10 @@ func (v QValueNumeric) Value() any { return v.Val } +func (v QValueNumeric) LValue(ls *lua.LState) lua.LValue { + return shared.LuaDecimal.New(ls, v.Val) +} + type QValueBytes struct { Val []byte } @@ -238,6 +322,10 @@ func (v QValueBytes) Value() any { return v.Val } +func (v QValueBytes) LValue(ls *lua.LState) lua.LValue { + return lua.LString(shared.UnsafeFastReadOnlyBytesToString(v.Val)) +} + type QValueUUID struct { Val [16]byte } @@ -250,6 +338,10 @@ func (v QValueUUID) Value() any { return v.Val } +func (v QValueUUID) LValue(ls *lua.LState) lua.LValue { + return shared.LuaUuid.New(ls, uuid.UUID(v.Val)) +} + type QValueJSON struct { Val string } @@ -262,6 +354,10 @@ func (v QValueJSON) Value() any { return v.Val } +func (v QValueJSON) LValue(ls *lua.LState) lua.LValue { + return lua.LString(v.Val) +} + type QValueBit struct { Val []byte } @@ -274,6 +370,10 @@ func (v QValueBit) Value() any { return v.Val } +func (v QValueBit) LValue(ls *lua.LState) lua.LValue { + return lua.LString(shared.UnsafeFastReadOnlyBytesToString(v.Val)) +} + type QValueHStore struct { Val string } @@ -286,6 +386,10 @@ func (v QValueHStore) Value() any { return v.Val } +func (v QValueHStore) LValue(ls *lua.LState) lua.LValue { + return lua.LString(v.Val) +} + type QValueGeography struct { Val string } @@ -298,6 +402,10 @@ func (v QValueGeography) Value() any { return v.Val } +func (v QValueGeography) LValue(ls *lua.LState) lua.LValue { + return lua.LString(v.Val) +} + type QValueGeometry struct { Val string } @@ -310,6 +418,10 @@ func (v QValueGeometry) Value() any { return v.Val } +func (v QValueGeometry) LValue(ls *lua.LState) lua.LValue { + return lua.LString(v.Val) +} + type QValuePoint struct { Val string } @@ -322,6 +434,10 @@ func (v QValuePoint) Value() any { return v.Val } +func (v QValuePoint) LValue(ls *lua.LState) lua.LValue { + return lua.LString(v.Val) +} + type QValueCIDR struct { Val string } @@ -334,6 +450,10 @@ func (v QValueCIDR) Value() any { return v.Val } +func (v QValueCIDR) LValue(ls *lua.LState) lua.LValue { + return lua.LString(v.Val) +} + type QValueINET struct { Val string } @@ -346,6 +466,10 @@ func (v QValueINET) Value() any { return v.Val } +func (v QValueINET) LValue(ls *lua.LState) lua.LValue { + return lua.LString(v.Val) +} + type QValueMacaddr struct { Val string } @@ -358,6 +482,10 @@ func (v QValueMacaddr) Value() any { return v.Val } +func (v QValueMacaddr) LValue(ls *lua.LState) lua.LValue { + return lua.LString(v.Val) +} + type QValueArrayFloat32 struct { Val []float32 } @@ -370,6 +498,12 @@ func (v QValueArrayFloat32) Value() any { return v.Val } +func (v QValueArrayFloat32) LValue(ls *lua.LState) lua.LValue { + return shared.SliceToLTable(ls, v.Val, func(f float32) lua.LValue { + return lua.LNumber(f) + }) +} + type QValueArrayFloat64 struct { Val []float64 } @@ -382,6 +516,12 @@ func (v QValueArrayFloat64) Value() any { return v.Val } +func (v QValueArrayFloat64) LValue(ls *lua.LState) lua.LValue { + return shared.SliceToLTable(ls, v.Val, func(x float64) lua.LValue { + return lua.LNumber(x) + }) +} + type QValueArrayInt16 struct { Val []int16 } @@ -394,6 +534,12 @@ func (v QValueArrayInt16) Value() any { return v.Val } +func (v QValueArrayInt16) LValue(ls *lua.LState) lua.LValue { + return shared.SliceToLTable(ls, v.Val, func(x int16) lua.LValue { + return lua.LNumber(x) + }) +} + type QValueArrayInt32 struct { Val []int32 } @@ -406,6 +552,12 @@ func (v QValueArrayInt32) Value() any { return v.Val } +func (v QValueArrayInt32) LValue(ls *lua.LState) lua.LValue { + return shared.SliceToLTable(ls, v.Val, func(x int32) lua.LValue { + return lua.LNumber(x) + }) +} + type QValueArrayInt64 struct { Val []int64 } @@ -418,6 +570,12 @@ func (v QValueArrayInt64) Value() any { return v.Val } +func (v QValueArrayInt64) LValue(ls *lua.LState) lua.LValue { + return shared.SliceToLTable(ls, v.Val, func(x int64) lua.LValue { + return glua64.I64.New(ls, x) + }) +} + type QValueArrayString struct { Val []string } @@ -430,6 +588,12 @@ func (v QValueArrayString) Value() any { return v.Val } +func (v QValueArrayString) LValue(ls *lua.LState) lua.LValue { + return shared.SliceToLTable(ls, v.Val, func(x string) lua.LValue { + return lua.LString(x) + }) +} + type QValueArrayDate struct { Val []time.Time } @@ -442,6 +606,12 @@ func (v QValueArrayDate) Value() any { return v.Val } +func (v QValueArrayDate) LValue(ls *lua.LState) lua.LValue { + return shared.SliceToLTable(ls, v.Val, func(x time.Time) lua.LValue { + return shared.LuaTime.New(ls, x) + }) +} + type QValueArrayTimestamp struct { Val []time.Time } @@ -454,6 +624,12 @@ func (v QValueArrayTimestamp) Value() any { return v.Val } +func (v QValueArrayTimestamp) LValue(ls *lua.LState) lua.LValue { + return shared.SliceToLTable(ls, v.Val, func(x time.Time) lua.LValue { + return shared.LuaTime.New(ls, x) + }) +} + type QValueArrayTimestampTZ struct { Val []time.Time } @@ -466,6 +642,12 @@ func (v QValueArrayTimestampTZ) Value() any { return v.Val } +func (v QValueArrayTimestampTZ) LValue(ls *lua.LState) lua.LValue { + return shared.SliceToLTable(ls, v.Val, func(x time.Time) lua.LValue { + return shared.LuaTime.New(ls, x) + }) +} + type QValueArrayBoolean struct { Val []bool } @@ -477,3 +659,9 @@ func (QValueArrayBoolean) Kind() QValueKind { func (v QValueArrayBoolean) Value() any { return v.Val } + +func (v QValueArrayBoolean) LValue(ls *lua.LState) lua.LValue { + return shared.SliceToLTable(ls, v.Val, func(x bool) lua.LValue { + return lua.LBool(x) + }) +} diff --git a/flow/model/record.go b/flow/model/record.go index 85e8723853..30f3493cf4 100644 --- a/flow/model/record.go +++ b/flow/model/record.go @@ -6,13 +6,13 @@ import ( "github.com/PeerDB-io/peer-flow/generated/protos" ) -type Record interface { +type Record[T Items] interface { GetCheckpointID() int64 GetCommitTime() time.Time GetDestinationTableName() string GetSourceTableName() string // get columns and values for the record - GetItems() RecordItems + GetItems() T PopulateCountMap(mapOfCounts map[string]*RecordTypeCounts) } @@ -31,9 +31,9 @@ func (r *BaseRecord) GetCommitTime() time.Time { return time.Unix(0, r.CommitTimeNano) } -type InsertRecord struct { +type InsertRecord[T Items] struct { // Items is a map of column name to value. - Items RecordItems + Items T // Name of the source table SourceTableName string // Name of the destination table @@ -43,30 +43,30 @@ type InsertRecord struct { BaseRecord } -func (r *InsertRecord) GetDestinationTableName() string { +func (r *InsertRecord[T]) GetDestinationTableName() string { return r.DestinationTableName } -func (r *InsertRecord) GetSourceTableName() string { +func (r *InsertRecord[T]) GetSourceTableName() string { return r.SourceTableName } -func (r *InsertRecord) GetItems() RecordItems { +func (r *InsertRecord[T]) GetItems() T { return r.Items } -func (r *InsertRecord) PopulateCountMap(mapOfCounts map[string]*RecordTypeCounts) { +func (r *InsertRecord[T]) PopulateCountMap(mapOfCounts map[string]*RecordTypeCounts) { recordCount, ok := mapOfCounts[r.DestinationTableName] if ok { recordCount.InsertCount++ } } -type UpdateRecord struct { +type UpdateRecord[T Items] struct { // OldItems is a map of column name to value. - OldItems RecordItems + OldItems T // NewItems is a map of column name to value. - NewItems RecordItems + NewItems T // unchanged toast columns UnchangedToastColumns map[string]struct{} // Name of the source table @@ -76,28 +76,28 @@ type UpdateRecord struct { BaseRecord } -func (r *UpdateRecord) GetDestinationTableName() string { +func (r *UpdateRecord[T]) GetDestinationTableName() string { return r.DestinationTableName } -func (r *UpdateRecord) GetSourceTableName() string { +func (r *UpdateRecord[T]) GetSourceTableName() string { return r.SourceTableName } -func (r *UpdateRecord) GetItems() RecordItems { +func (r *UpdateRecord[T]) GetItems() T { return r.NewItems } -func (r *UpdateRecord) PopulateCountMap(mapOfCounts map[string]*RecordTypeCounts) { +func (r *UpdateRecord[T]) PopulateCountMap(mapOfCounts map[string]*RecordTypeCounts) { recordCount, ok := mapOfCounts[r.DestinationTableName] if ok { recordCount.UpdateCount++ } } -type DeleteRecord struct { +type DeleteRecord[T Items] struct { // Items is a map of column name to value. - Items RecordItems + Items T // unchanged toast columns, filled from latest UpdateRecord UnchangedToastColumns map[string]struct{} // Name of the source table @@ -107,19 +107,19 @@ type DeleteRecord struct { BaseRecord } -func (r *DeleteRecord) GetDestinationTableName() string { +func (r *DeleteRecord[T]) GetDestinationTableName() string { return r.DestinationTableName } -func (r *DeleteRecord) GetSourceTableName() string { +func (r *DeleteRecord[T]) GetSourceTableName() string { return r.SourceTableName } -func (r *DeleteRecord) GetItems() RecordItems { +func (r *DeleteRecord[T]) GetItems() T { return r.Items } -func (r *DeleteRecord) PopulateCountMap(mapOfCounts map[string]*RecordTypeCounts) { +func (r *DeleteRecord[T]) PopulateCountMap(mapOfCounts map[string]*RecordTypeCounts) { recordCount, ok := mapOfCounts[r.DestinationTableName] if ok { recordCount.DeleteCount++ @@ -127,22 +127,23 @@ func (r *DeleteRecord) PopulateCountMap(mapOfCounts map[string]*RecordTypeCounts } // being clever and passing the delta back as a regular record instead of heavy CDC refactoring. -type RelationRecord struct { +type RelationRecord[T Items] struct { TableSchemaDelta *protos.TableSchemaDelta `json:"tableSchemaDelta"` BaseRecord } -func (r *RelationRecord) GetDestinationTableName() string { +func (r *RelationRecord[T]) GetDestinationTableName() string { return r.TableSchemaDelta.DstTableName } -func (r *RelationRecord) GetSourceTableName() string { +func (r *RelationRecord[T]) GetSourceTableName() string { return r.TableSchemaDelta.SrcTableName } -func (r *RelationRecord) GetItems() RecordItems { - return RecordItems{ColToVal: nil} +func (r *RelationRecord[T]) GetItems() T { + var none T + return none } -func (r *RelationRecord) PopulateCountMap(mapOfCounts map[string]*RecordTypeCounts) { +func (r *RelationRecord[T]) PopulateCountMap(mapOfCounts map[string]*RecordTypeCounts) { } diff --git a/flow/model/record_items.go b/flow/model/record_items.go index d6702cf777..54f8820f58 100644 --- a/flow/model/record_items.go +++ b/flow/model/record_items.go @@ -11,6 +11,18 @@ import ( "github.com/PeerDB-io/peer-flow/model/qvalue" ) +type Items interface { + json.Marshaler + UpdateIfNotExists(Items) []string + GetBytesByColName(string) ([]byte, error) + ToJSONWithOptions(ToJSONOptions) (string, error) +} + +func ItemsToJSON(items Items) (string, error) { + bytes, err := items.MarshalJSON() + return string(bytes), err +} + // encoding/gob cannot encode unexported fields type RecordItems struct { ColToVal map[string]qvalue.QValue @@ -22,14 +34,6 @@ func NewRecordItems(capacity int) RecordItems { } } -func NewRecordItemWithData(cols []string, val []qvalue.QValue) RecordItems { - recordItem := NewRecordItems(len(cols)) - for i, col := range cols { - recordItem.ColToVal[col] = val[i] - } - return recordItem -} - func (r RecordItems) AddColumn(col string, val qvalue.QValue) { r.ColToVal[col] = val } @@ -42,8 +46,9 @@ func (r RecordItems) GetColumnValue(col string) qvalue.QValue { // current RecordItems with the values from the input RecordItems for the columns // that are present in the input RecordItems but not in the current RecordItems. // We return the slice of col names that were updated. -func (r RecordItems) UpdateIfNotExists(input RecordItems) []string { - updatedCols := make([]string, 0) +func (r RecordItems) UpdateIfNotExists(input_ Items) []string { + input := input_.(RecordItems) + updatedCols := make([]string, 0, len(input.ColToVal)) for col, val := range input.ColToVal { if _, ok := r.ColToVal[col]; !ok { r.ColToVal[col] = val @@ -61,11 +66,19 @@ func (r RecordItems) GetValueByColName(colName string) (qvalue.QValue, error) { return val, nil } +func (r RecordItems) GetBytesByColName(colName string) ([]byte, error) { + val, err := r.GetValueByColName(colName) + if err != nil { + return nil, err + } + return []byte(fmt.Sprint(val.Value())), nil +} + func (r RecordItems) Len() int { return len(r.ColToVal) } -func (r RecordItems) toMap(hstoreAsJSON bool, opts ToJSONOptions) (map[string]interface{}, error) { +func (r RecordItems) toMap(opts ToJSONOptions) (map[string]interface{}, error) { jsonStruct := make(map[string]interface{}, len(r.ColToVal)) for col, qv := range r.ColToVal { if qv == nil { @@ -129,7 +142,7 @@ func (r RecordItems) toMap(hstoreAsJSON bool, opts ToJSONOptions) (map[string]in case qvalue.QValueHStore: hstoreVal := v.Val - if !hstoreAsJSON { + if !opts.HStoreAsJSON { jsonStruct[col] = hstoreVal } else { jsonVal, err := datatypes.ParseHstore(hstoreVal) @@ -211,16 +224,12 @@ func (r RecordItems) ToJSONWithOptions(options ToJSONOptions) (string, error) { return string(bytes), err } -func (r RecordItems) ToJSON() (string, error) { - return r.ToJSONWithOptions(NewToJSONOptions(nil, true)) -} - func (r RecordItems) MarshalJSON() ([]byte, error) { return r.MarshalJSONWithOptions(NewToJSONOptions(nil, true)) } func (r RecordItems) MarshalJSONWithOptions(opts ToJSONOptions) ([]byte, error) { - jsonStruct, err := r.toMap(opts.HStoreAsJSON, opts) + jsonStruct, err := r.toMap(opts) if err != nil { return nil, err } diff --git a/flow/peerdbenv/config.go b/flow/peerdbenv/config.go index db252c84f0..be580d5884 100644 --- a/flow/peerdbenv/config.go +++ b/flow/peerdbenv/config.go @@ -40,6 +40,10 @@ func PeerDBQueueFlushTimeoutSeconds() time.Duration { return time.Duration(x) * time.Second } +func PeerDBQueueParallelism() int { + return getEnvInt("PEERDB_QUEUE_PARALLELISM", 4) +} + // env variable doesn't exist anymore, but tests appear to depend on this // in lieu of an actual value of IdleTimeoutSeconds func PeerDBCDCIdleTimeoutSeconds(providedValue int) time.Duration { diff --git a/flow/pua/peerdb.go b/flow/pua/peerdb.go index 7a6dab9652..540065970f 100644 --- a/flow/pua/peerdb.go +++ b/flow/pua/peerdb.go @@ -3,7 +3,6 @@ package pua import ( "bytes" "fmt" - "math/big" "time" "github.com/google/uuid" @@ -19,15 +18,12 @@ import ( "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/PeerDB-io/peer-flow/peerdbenv" + "github.com/PeerDB-io/peer-flow/shared" ) var ( - LuaRecord = glua64.UserDataType[model.Record]{Name: "peerdb_record"} - LuaRow = glua64.UserDataType[model.RecordItems]{Name: "peerdb_row"} - LuaTime = glua64.UserDataType[time.Time]{Name: "peerdb_time"} - LuaUuid = glua64.UserDataType[uuid.UUID]{Name: "peerdb_uuid"} - LuaBigInt = glua64.UserDataType[*big.Int]{Name: "peerdb_bigint"} - LuaDecimal = glua64.UserDataType[decimal.Decimal]{Name: "peerdb_decimal"} + LuaRecord = glua64.UserDataType[model.Record[model.RecordItems]]{Name: "peerdb_record"} + LuaRow = glua64.UserDataType[model.RecordItems]{Name: "peerdb_row"} ) func RegisterTypes(ls *lua.LState) { @@ -45,7 +41,11 @@ func RegisterTypes(ls *lua.LState) { ls.PreloadModule("msgpack", gluamsgpack.Loader) ls.PreloadModule("utf8", gluautf8.Loader) - mt := LuaRecord.NewMetatable(ls) + mt := ls.NewTypeMetatable("Array") + mt.RawSetString("__json", ls.NewFunction(LuaArrayJson)) + mt.RawSetString("__msgpack", ls.NewFunction(LuaArrayMsgpack)) + + mt = LuaRecord.NewMetatable(ls) mt.RawSetString("__index", ls.NewFunction(LuaRecordIndex)) mt.RawSetString("__json", ls.NewFunction(LuaRecordJson)) @@ -53,27 +53,27 @@ func RegisterTypes(ls *lua.LState) { mt.RawSetString("__index", ls.NewFunction(LuaRowIndex)) mt.RawSetString("__len", ls.NewFunction(LuaRowLen)) - mt = LuaUuid.NewMetatable(ls) + mt = shared.LuaUuid.NewMetatable(ls) mt.RawSetString("__index", ls.NewFunction(LuaUuidIndex)) mt.RawSetString("__tostring", ls.NewFunction(LuaUuidString)) mt.RawSetString("__eq", ls.NewFunction(LuaUuidEq)) mt.RawSetString("__msgpack", ls.NewFunction(LuaUuidMsgpack)) - mt = LuaTime.NewMetatable(ls) + mt = shared.LuaTime.NewMetatable(ls) mt.RawSetString("__index", ls.NewFunction(LuaTimeIndex)) mt.RawSetString("__tostring", ls.NewFunction(LuaTimeString)) mt.RawSetString("__eq", ls.NewFunction(LuaTimeEq)) mt.RawSetString("__le", ls.NewFunction(LuaTimeLe)) mt.RawSetString("__lt", ls.NewFunction(LuaTimeLt)) - mt = LuaBigInt.NewMetatable(ls) + mt = shared.LuaBigInt.NewMetatable(ls) mt.RawSetString("__index", ls.NewFunction(LuaBigIntIndex)) mt.RawSetString("__tostring", ls.NewFunction(LuaBigIntString)) mt.RawSetString("__eq", ls.NewFunction(LuaBigIntEq)) mt.RawSetString("__le", ls.NewFunction(LuaBigIntLe)) mt.RawSetString("__lt", ls.NewFunction(LuaBigIntLt)) - mt = LuaDecimal.NewMetatable(ls) + mt = shared.LuaDecimal.NewMetatable(ls) mt.RawSetString("__index", ls.NewFunction(LuaDecimalIndex)) mt.RawSetString("__tostring", ls.NewFunction(LuaDecimalString)) mt.RawSetString("__eq", ls.NewFunction(LuaDecimalEq)) @@ -82,6 +82,7 @@ func RegisterTypes(ls *lua.LState) { mt.RawSetString("__msgpack", ls.NewFunction(LuaDecimalString)) peerdb := ls.NewTable() + peerdb.RawSetString("RowTable", ls.NewFunction(LuaRowTable)) peerdb.RawSetString("RowColumns", ls.NewFunction(LuaRowColumns)) peerdb.RawSetString("RowColumnKind", ls.NewFunction(LuaRowColumnKind)) peerdb.RawSetString("Now", ls.NewFunction(LuaNow)) @@ -131,7 +132,7 @@ func GetRowQ(ls *lua.LState, row model.RecordItems, col string) qvalue.QValue { func LuaRowIndex(ls *lua.LState) int { row, key := LuaRow.StartIndex(ls) - ls.Push(LuaQValue(ls, GetRowQ(ls, row, key))) + ls.Push(GetRowQ(ls, row, key).LValue(ls)) return 1 } @@ -141,6 +142,16 @@ func LuaRowLen(ls *lua.LState) int { return 1 } +func LuaRowTable(ls *lua.LState) int { + row := LuaRow.StartMethod(ls) + tbl := ls.CreateTable(0, len(row.ColToVal)) + for col, val := range row.ColToVal { + tbl.RawSetString(col, val.LValue(ls)) + } + ls.Push(tbl) + return 1 +} + func LuaRowColumns(ls *lua.LState) int { row := LuaRow.StartMethod(ls) tbl := ls.CreateTable(len(row.ColToVal), 0) @@ -159,18 +170,36 @@ func LuaRowColumnKind(ls *lua.LState) int { return 1 } +func LuaArrayJson(ls *lua.LState) int { + ls.Push(&lua.LUserData{ + Value: gluajson.Array(ls.CheckTable(1)), + Env: ls.Env, + Metatable: nil, + }) + return 1 +} + +func LuaArrayMsgpack(ls *lua.LState) int { + ls.Push(&lua.LUserData{ + Value: gluamsgpack.Array(ls.CheckTable(1)), + Env: ls.Env, + Metatable: nil, + }) + return 1 +} + func LuaRecordIndex(ls *lua.LState) int { record, key := LuaRecord.StartIndex(ls) switch key { case "kind": switch record.(type) { - case *model.InsertRecord: + case *model.InsertRecord[model.RecordItems]: ls.Push(lua.LString("insert")) - case *model.UpdateRecord: + case *model.UpdateRecord[model.RecordItems]: ls.Push(lua.LString("update")) - case *model.DeleteRecord: + case *model.DeleteRecord[model.RecordItems]: ls.Push(lua.LString("delete")) - case *model.RelationRecord: + case *model.RelationRecord[model.RecordItems]: ls.Push(lua.LString("relation")) } case "row": @@ -183,9 +212,9 @@ func LuaRecordIndex(ls *lua.LState) int { case "old": var items model.RecordItems switch rec := record.(type) { - case *model.UpdateRecord: + case *model.UpdateRecord[model.RecordItems]: items = rec.OldItems - case *model.DeleteRecord: + case *model.DeleteRecord[model.RecordItems]: items = rec.Items } if items.ColToVal != nil { @@ -196,9 +225,9 @@ func LuaRecordIndex(ls *lua.LState) int { case "new": var items model.RecordItems switch rec := record.(type) { - case *model.InsertRecord: + case *model.InsertRecord[model.RecordItems]: items = rec.Items - case *model.UpdateRecord: + case *model.UpdateRecord[model.RecordItems]: items = rec.NewItems } if items.ColToVal != nil { @@ -209,13 +238,13 @@ func LuaRecordIndex(ls *lua.LState) int { case "checkpoint": ls.Push(glua64.I64.New(ls, record.GetCheckpointID())) case "commit_time": - ls.Push(LuaTime.New(ls, record.GetCommitTime())) + ls.Push(shared.LuaTime.New(ls, record.GetCommitTime())) case "target": ls.Push(lua.LString(record.GetDestinationTableName())) case "source": ls.Push(lua.LString(record.GetSourceTableName())) case "unchanged_columns": - if ur, ok := record.(*model.UpdateRecord); ok { + if ur, ok := record.(*model.UpdateRecord[model.RecordItems]); ok { tbl := ls.CreateTable(0, len(ur.UnchangedToastColumns)) for col := range ur.UnchangedToastColumns { tbl.RawSetString(col, lua.LTrue) @@ -238,7 +267,7 @@ func LuaRecordJson(ls *lua.LState) int { } { tbl.RawSetString(key, ls.GetField(ud, key)) } - if ur, ok := ud.Value.(*model.UpdateRecord); ok { + if ur, ok := ud.Value.(*model.UpdateRecord[model.RecordItems]); ok { if len(ur.UnchangedToastColumns) > 0 { unchanged := ls.CreateTable(len(ur.UnchangedToastColumns), 0) for col := range ur.UnchangedToastColumns { @@ -251,85 +280,8 @@ func LuaRecordJson(ls *lua.LState) int { return 1 } -func qvToLTable[T any](ls *lua.LState, s []T, f func(x T) lua.LValue) *lua.LTable { - tbl := ls.CreateTable(len(s), 0) - for idx, val := range s { - tbl.RawSetInt(idx+1, f(val)) - } - return tbl -} - -func LuaQValue(ls *lua.LState, qv qvalue.QValue) lua.LValue { - switch v := qv.Value().(type) { - case nil: - return lua.LNil - case bool: - return lua.LBool(v) - case uint8: - if qv.Kind() == qvalue.QValueKindQChar { - return lua.LString(rune(v)) - } else { - return lua.LNumber(v) - } - case int16: - return lua.LNumber(v) - case int32: - return lua.LNumber(v) - case int64: - return glua64.I64.New(ls, v) - case float32: - return lua.LNumber(v) - case float64: - return lua.LNumber(v) - case string: - return lua.LString(v) - case time.Time: - return LuaTime.New(ls, v) - case decimal.Decimal: - return LuaDecimal.New(ls, v) - case [16]byte: - return LuaUuid.New(ls, uuid.UUID(v)) - case []byte: - return lua.LString(v) - case []float32: - return qvToLTable(ls, v, func(f float32) lua.LValue { - return lua.LNumber(f) - }) - case []float64: - return qvToLTable(ls, v, func(f float64) lua.LValue { - return lua.LNumber(f) - }) - case []int16: - return qvToLTable(ls, v, func(x int16) lua.LValue { - return lua.LNumber(x) - }) - case []int32: - return qvToLTable(ls, v, func(x int32) lua.LValue { - return lua.LNumber(x) - }) - case []int64: - return qvToLTable(ls, v, func(x int64) lua.LValue { - return glua64.I64.New(ls, x) - }) - case []string: - return qvToLTable(ls, v, func(x string) lua.LValue { - return lua.LString(x) - }) - case []time.Time: - return qvToLTable(ls, v, func(x time.Time) lua.LValue { - return LuaTime.New(ls, x) - }) - case []bool: - return qvToLTable(ls, v, func(x bool) lua.LValue { - return lua.LBool(x) - }) - default: - return lua.LString(fmt.Sprint(qv.Value())) - } -} - func LuaUuidIndex(ls *lua.LState) int { - _, val := LuaUuid.Check(ls, 1) + _, val := shared.LuaUuid.Check(ls, 1) key := ls.CheckNumber(2) ki := int(key) if ki >= 0 && ki < 16 { @@ -340,20 +292,20 @@ func LuaUuidIndex(ls *lua.LState) int { } func LuaUuidString(ls *lua.LState) int { - val := LuaUuid.StartMethod(ls) + val := shared.LuaUuid.StartMethod(ls) ls.Push(lua.LString(val.String())) return 1 } func LuaUuidEq(ls *lua.LState) int { - val1 := LuaUuid.StartMethod(ls) - _, val2 := LuaUuid.Check(ls, 2) + val1 := shared.LuaUuid.StartMethod(ls) + _, val2 := shared.LuaUuid.Check(ls, 2) ls.Push(lua.LBool(val1 == val2)) return 1 } func LuaUuidMsgpack(ls *lua.LState) int { - val := LuaUuid.StartMethod(ls) + val := shared.LuaUuid.StartMethod(ls) ls.Push(&lua.LUserData{ Value: gluamsgpack.Bin(val[:]), Env: ls.Env, @@ -363,15 +315,15 @@ func LuaUuidMsgpack(ls *lua.LState) int { } func LuaNow(ls *lua.LState) int { - ls.Push(LuaTime.New(ls, time.Now())) + ls.Push(shared.LuaTime.New(ls, time.Now())) return 1 } func LuaUUID(ls *lua.LState) int { if ls.GetTop() == 0 { - ls.Push(LuaUuid.New(ls, uuid.New())) + ls.Push(shared.LuaUuid.New(ls, uuid.New())) } else if v, ok := ls.Get(1).(lua.LString); ok { - ls.Push(LuaUuid.New(ls, uuid.MustParse(string(v)))) + ls.Push(shared.LuaUuid.New(ls, uuid.MustParse(string(v)))) } else { ls.RaiseError("uuid must be created from string") } @@ -381,13 +333,13 @@ func LuaUUID(ls *lua.LState) int { func LuaParseDecimal(ls *lua.LState) int { switch v := ls.Get(1).(type) { case lua.LNumber: - ls.Push(LuaDecimal.New(ls, decimal.NewFromFloat(float64(v)))) + ls.Push(shared.LuaDecimal.New(ls, decimal.NewFromFloat(float64(v)))) case lua.LString: d, err := decimal.NewFromString(string(v)) if err != nil { ls.RaiseError(err.Error()) } - ls.Push(LuaDecimal.New(ls, d)) + ls.Push(shared.LuaDecimal.New(ls, d)) default: ls.RaiseError("cannot create decimal from " + v.Type().String()) } @@ -413,7 +365,7 @@ func LuaToString(ls *lua.LState) int { } func LuaTimeIndex(ls *lua.LState) int { - tm, key := LuaTime.StartIndex(ls) + tm, key := shared.LuaTime.StartIndex(ls) switch key { case "unix_nano": ls.Push(glua64.I64.New(ls, tm.UnixNano())) @@ -448,34 +400,34 @@ func LuaTimeIndex(ls *lua.LState) int { } func LuaTimeString(ls *lua.LState) int { - tm := LuaTime.StartMethod(ls) + tm := shared.LuaTime.StartMethod(ls) ls.Push(lua.LString(tm.String())) return 1 } func LuaTimeEq(ls *lua.LState) int { - t1 := LuaTime.StartMethod(ls) - _, t2 := LuaTime.Check(ls, 2) + t1 := shared.LuaTime.StartMethod(ls) + _, t2 := shared.LuaTime.Check(ls, 2) ls.Push(lua.LBool(t1.Compare(t2) == 0)) return 1 } func LuaTimeLe(ls *lua.LState) int { - t1 := LuaTime.StartMethod(ls) - _, t2 := LuaTime.Check(ls, 2) + t1 := shared.LuaTime.StartMethod(ls) + _, t2 := shared.LuaTime.Check(ls, 2) ls.Push(lua.LBool(t1.Compare(t2) <= 0)) return 1 } func LuaTimeLt(ls *lua.LState) int { - t1 := LuaTime.StartMethod(ls) - _, t2 := LuaTime.Check(ls, 2) + t1 := shared.LuaTime.StartMethod(ls) + _, t2 := shared.LuaTime.Check(ls, 2) ls.Push(lua.LBool(t1.Compare(t2) == -1)) return 1 } func LuaBigIntIndex(ls *lua.LState) int { - bi, key := LuaBigInt.StartIndex(ls) + bi, key := shared.LuaBigInt.StartIndex(ls) switch key { case "sign": ls.Push(lua.LNumber(bi.Sign())) @@ -492,43 +444,43 @@ func LuaBigIntIndex(ls *lua.LState) int { } func LuaBigIntString(ls *lua.LState) int { - bi := LuaBigInt.StartMethod(ls) + bi := shared.LuaBigInt.StartMethod(ls) ls.Push(lua.LString(bi.String())) return 1 } func LuaBigIntEq(ls *lua.LState) int { - t1 := LuaBigInt.StartMethod(ls) - _, t2 := LuaBigInt.Check(ls, 2) + t1 := shared.LuaBigInt.StartMethod(ls) + _, t2 := shared.LuaBigInt.Check(ls, 2) ls.Push(lua.LBool(t1.Cmp(t2) == 0)) return 1 } func LuaBigIntLe(ls *lua.LState) int { - t1 := LuaBigInt.StartMethod(ls) - _, t2 := LuaBigInt.Check(ls, 2) + t1 := shared.LuaBigInt.StartMethod(ls) + _, t2 := shared.LuaBigInt.Check(ls, 2) ls.Push(lua.LBool(t1.Cmp(t2) <= 0)) return 1 } func LuaBigIntLt(ls *lua.LState) int { - t1 := LuaBigInt.StartMethod(ls) - _, t2 := LuaBigInt.Check(ls, 2) + t1 := shared.LuaBigInt.StartMethod(ls) + _, t2 := shared.LuaBigInt.Check(ls, 2) ls.Push(lua.LBool(t1.Cmp(t2) == -1)) return 1 } func LuaDecimalIndex(ls *lua.LState) int { - num, key := LuaDecimal.StartIndex(ls) + num, key := shared.LuaDecimal.StartIndex(ls) switch key { case "coefficient": - ls.Push(LuaBigInt.New(ls, num.Coefficient())) + ls.Push(shared.LuaBigInt.New(ls, num.Coefficient())) case "coefficient64": ls.Push(glua64.I64.New(ls, num.CoefficientInt64())) case "exponent": ls.Push(lua.LNumber(num.Exponent())) case "bigint": - ls.Push(LuaBigInt.New(ls, num.BigInt())) + ls.Push(shared.LuaBigInt.New(ls, num.BigInt())) case "int64": ls.Push(glua64.I64.New(ls, num.IntPart())) case "float64": @@ -540,28 +492,28 @@ func LuaDecimalIndex(ls *lua.LState) int { } func LuaDecimalString(ls *lua.LState) int { - num := LuaDecimal.StartMethod(ls) + num := shared.LuaDecimal.StartMethod(ls) ls.Push(lua.LString(num.String())) return 1 } func LuaDecimalEq(ls *lua.LState) int { - t1 := LuaDecimal.StartMethod(ls) - _, t2 := LuaDecimal.Check(ls, 2) + t1 := shared.LuaDecimal.StartMethod(ls) + _, t2 := shared.LuaDecimal.Check(ls, 2) ls.Push(lua.LBool(t1.Cmp(t2) == 0)) return 1 } func LuaDecimalLe(ls *lua.LState) int { - t1 := LuaDecimal.StartMethod(ls) - _, t2 := LuaDecimal.Check(ls, 2) + t1 := shared.LuaDecimal.StartMethod(ls) + _, t2 := shared.LuaDecimal.Check(ls, 2) ls.Push(lua.LBool(t1.Cmp(t2) <= 0)) return 1 } func LuaDecimalLt(ls *lua.LState) int { - t1 := LuaDecimal.StartMethod(ls) - _, t2 := LuaDecimal.Check(ls, 2) + t1 := shared.LuaDecimal.StartMethod(ls) + _, t2 := shared.LuaDecimal.Check(ls, 2) ls.Push(lua.LBool(t1.Cmp(t2) == -1)) return 1 } diff --git a/flow/pua/peerdb_test.go b/flow/pua/peerdb_test.go index ee9a7b084b..d08c9b4db6 100644 --- a/flow/pua/peerdb_test.go +++ b/flow/pua/peerdb_test.go @@ -28,6 +28,10 @@ func Test(t *testing.T) { row.AddColumn("a", qvalue.QValueInt64{Val: 5040}) ls.Env.RawSetString("row", LuaRow.New(ls, row)) + row_empty_array := model.NewRecordItems(1) + row_empty_array.AddColumn("a", qvalue.QValueArrayInt32{Val: nil}) + ls.Env.RawSetString("row_empty_array", LuaRow.New(ls, row_empty_array)) + assert(t, ls, ` assert(require('bit32').band(173, 21) == 5) assert(dofile == nil) @@ -62,5 +66,6 @@ assert(msgpack.encode(uuid) == string.char(0xc4, 16, 2, 3, 5, 7, 11, 13, 17, 19, local json = require "json" assert(json.encode(row) == "{\"a\":5040}") +assert(json.encode(row_empty_array.a) == "[]") `) } diff --git a/flow/shared/lua.go b/flow/shared/lua.go new file mode 100644 index 0000000000..2b95c3464c --- /dev/null +++ b/flow/shared/lua.go @@ -0,0 +1,28 @@ +package shared + +import ( + "math/big" + "time" + + "github.com/google/uuid" + "github.com/shopspring/decimal" + "github.com/yuin/gopher-lua" + + "github.com/PeerDB-io/glua64" +) + +var ( + LuaTime = glua64.UserDataType[time.Time]{Name: "peerdb_time"} + LuaUuid = glua64.UserDataType[uuid.UUID]{Name: "peerdb_uuid"} + LuaBigInt = glua64.UserDataType[*big.Int]{Name: "peerdb_bigint"} + LuaDecimal = glua64.UserDataType[decimal.Decimal]{Name: "peerdb_decimal"} +) + +func SliceToLTable[T any](ls *lua.LState, s []T, f func(x T) lua.LValue) *lua.LTable { + tbl := ls.CreateTable(len(s), 0) + tbl.Metatable = ls.GetTypeMetatable("Array") + for idx, val := range s { + tbl.RawSetInt(idx+1, f(val)) + } + return tbl +} diff --git a/flow/shared/postgres.go b/flow/shared/postgres.go index f74d94de50..296d496fb2 100644 --- a/flow/shared/postgres.go +++ b/flow/shared/postgres.go @@ -4,12 +4,15 @@ import ( "context" "errors" "fmt" + "log/slog" "net/url" + "time" "github.com/jackc/pgerrcode" "github.com/jackc/pgx/v5" "github.com/jackc/pgx/v5/pgconn" "github.com/jackc/pgx/v5/pgtype" + "go.temporal.io/sdk/log" "github.com/PeerDB-io/peer-flow/generated/protos" ) @@ -91,3 +94,12 @@ func GetMajorVersion(ctx context.Context, conn *pgx.Conn) (PGVersion, error) { return PGVersion(version), nil } + +func RollbackTx(tx pgx.Tx, logger log.Logger) { + ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + defer cancel() + err := tx.Rollback(ctx) + if err != nil && err != pgx.ErrTxClosed { + logger.Error("error while rolling back transaction", slog.Any("error", err)) + } +} diff --git a/flow/shared/schema_helpers.go b/flow/shared/schema_helpers.go index 2c92195e6f..196e57d431 100644 --- a/flow/shared/schema_helpers.go +++ b/flow/shared/schema_helpers.go @@ -60,6 +60,7 @@ func BuildProcessedSchemaMapping(tableMappings []*protos.TableMapping, TableIdentifier: tableSchema.TableIdentifier, PrimaryKeyColumns: tableSchema.PrimaryKeyColumns, IsReplicaIdentityFull: tableSchema.IsReplicaIdentityFull, + System: tableSchema.System, Columns: columns, } } diff --git a/flow/shared/string.go b/flow/shared/string.go index 6f3a5c7f29..b6295ab00d 100644 --- a/flow/shared/string.go +++ b/flow/shared/string.go @@ -1,7 +1,27 @@ package shared -import "unsafe" +import ( + "regexp" + "unsafe" +) func UnsafeFastStringToReadOnlyBytes(s string) []byte { return unsafe.Slice(unsafe.StringData(s), len(s)) } + +func UnsafeFastReadOnlyBytesToString(s []byte) string { + return unsafe.String(unsafe.SliceData(s), len(s)) +} + +var ( + reIllegalIdentifierCharacters = regexp.MustCompile("[^a-zA-Z0-9_]+") + reLegalIdentifierLower = regexp.MustCompile("^[a-z_][a-z0-9_]*$") +) + +func ReplaceIllegalCharactersWithUnderscores(s string) string { + return reIllegalIdentifierCharacters.ReplaceAllString(s, "_") +} + +func IsValidReplicationName(s string) bool { + return reLegalIdentifierLower.MatchString(s) +} diff --git a/flow/workflows/qrep_flow.go b/flow/workflows/qrep_flow.go index 4b3a59873b..355d63c1c1 100644 --- a/flow/workflows/qrep_flow.go +++ b/flow/workflows/qrep_flow.go @@ -110,6 +110,7 @@ func (q *QRepFlowExecution) getTableSchema(ctx workflow.Context, tableName strin PeerConnectionConfig: q.config.SourcePeer, TableIdentifiers: []string{tableName}, FlowName: q.config.FlowJobName, + System: q.config.System, } future := workflow.ExecuteActivity(ctx, flowable.GetTableSchema, tableSchemaInput) diff --git a/flow/workflows/setup_flow.go b/flow/workflows/setup_flow.go index 2c2b0876d6..5df1746f4f 100644 --- a/flow/workflows/setup_flow.go +++ b/flow/workflows/setup_flow.go @@ -185,6 +185,7 @@ func (s *SetupFlowExecution) fetchTableSchemaAndSetupNormalizedTables( PeerConnectionConfig: flowConnectionConfigs.Source, TableIdentifiers: sourceTables, FlowName: s.cdcFlowName, + System: flowConnectionConfigs.System, } future := workflow.ExecuteActivity(ctx, flowable.GetTableSchema, tableSchemaInput) diff --git a/flow/workflows/snapshot_flow.go b/flow/workflows/snapshot_flow.go index 8c88440035..f1c8c7d7f3 100644 --- a/flow/workflows/snapshot_flow.go +++ b/flow/workflows/snapshot_flow.go @@ -3,7 +3,6 @@ package peerflow import ( "fmt" "log/slog" - "regexp" "slices" "strings" "time" @@ -117,8 +116,7 @@ func (s *SnapshotFlowExecution) cloneTable( originalRunID := workflow.GetInfo(ctx).OriginalRunID childWorkflowID := fmt.Sprintf("clone_%s_%s_%s", flowName, dstName, originalRunID) - reg := regexp.MustCompile("[^a-zA-Z0-9_]+") - childWorkflowID = reg.ReplaceAllString(childWorkflowID, "_") + childWorkflowID = shared.ReplaceIllegalCharactersWithUnderscores(childWorkflowID) s.logger.Info(fmt.Sprintf("Obtained child id %s for source table %s and destination table %s", childWorkflowID, srcName, dstName), cloneLog) @@ -190,6 +188,7 @@ func (s *SnapshotFlowExecution) cloneTable( WriteMode: &protos.QRepWriteMode{ WriteType: protos.QRepWriteType_QREP_WRITE_MODE_APPEND, }, + System: s.config.System, } state := NewQRepFlowState() diff --git a/flow/workflows/sync_flow.go b/flow/workflows/sync_flow.go index 61fc94753d..e421759909 100644 --- a/flow/workflows/sync_flow.go +++ b/flow/workflows/sync_flow.go @@ -113,7 +113,12 @@ func SyncFlowWorkflow( WaitForCancellation: true, }) - syncFlowFuture := workflow.ExecuteActivity(syncFlowCtx, flowable.SyncFlow, config, options, sessionID) + var syncFlowFuture workflow.Future + if config.System == protos.TypeSystem_Q { + syncFlowFuture = workflow.ExecuteActivity(syncFlowCtx, flowable.SyncRecords, config, options, sessionID) + } else { + syncFlowFuture = workflow.ExecuteActivity(syncFlowCtx, flowable.SyncPg, config, options, sessionID) + } selector.AddFuture(syncFlowFuture, func(f workflow.Future) { syncDone = true @@ -155,6 +160,7 @@ func SyncFlowWorkflow( PeerConnectionConfig: config.Source, TableIdentifiers: modifiedSrcTables, FlowName: config.FlowJobName, + System: config.System, }) var getModifiedSchemaRes *protos.GetTableSchemaBatchOutput diff --git a/nexus/Cargo.lock b/nexus/Cargo.lock index 1201854e15..e53188476f 100644 --- a/nexus/Cargo.lock +++ b/nexus/Cargo.lock @@ -127,9 +127,9 @@ dependencies = [ [[package]] name = "anyhow" -version = "1.0.81" +version = "1.0.82" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0952808a6c2afd1aa8947271f3a60f1a6763c7b912d210184c5149b5cf147247" +checksum = "f538837af36e6f6a9be0faa67f9a314f8119e4e4b5867c6ab40ed60360142519" [[package]] name = "ar" @@ -170,7 +170,7 @@ checksum = "30c5ef0ede93efbf733c1a727f3b6b5a1060bbedd5600183e66f6e4be4af0ec5" dependencies = [ "proc-macro2", "quote", - "syn 2.0.58", + "syn 2.0.60", ] [[package]] @@ -192,18 +192,18 @@ checksum = "16e62a023e7c117e27523144c5d2459f4397fcc3cab0085af8e2224f643a0193" dependencies = [ "proc-macro2", "quote", - "syn 2.0.58", + "syn 2.0.60", ] [[package]] name = "async-trait" -version = "0.1.79" +version = "0.1.80" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a507401cad91ec6a857ed5513a2073c82a9b9048762b885bb98655b306964681" +checksum = "c6fa2087f2753a7da8cc1c0dbfcf89579dd57458e36769de5ac750b4671737ca" dependencies = [ "proc-macro2", "quote", - "syn 2.0.58", + "syn 2.0.60", ] [[package]] @@ -362,15 +362,15 @@ dependencies = [ "proc-macro-crate", "proc-macro2", "quote", - "syn 2.0.58", + "syn 2.0.60", "syn_derive", ] [[package]] name = "bumpalo" -version = "3.15.4" +version = "3.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7ff69b9dd49fd426c69a0db9fc04dd934cdb6645ff000864d98f7e2af8830eaa" +checksum = "79296716171880943b8470b5f8d03aa55eb2e645a4874bdbb28adb49162e012c" [[package]] name = "bytecheck" @@ -474,9 +474,9 @@ dependencies = [ [[package]] name = "cc" -version = "1.0.91" +version = "1.0.95" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1fd97381a8cc6493395a5afc4c691c1084b3768db713b73aa215217aa245d153" +checksum = "d32a725bc159af97c3e629873bb9f88fb8cf8a4867175f76dc987815ea07c83b" [[package]] name = "cfg-if" @@ -492,9 +492,9 @@ checksum = "fd16c4719339c4530435d38e511904438d07cce7950afa3718a84ac36c10e89e" [[package]] name = "chrono" -version = "0.4.37" +version = "0.4.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8a0d04d43504c61aa6c7531f1871dd0d418d91130162063b789da00fd7057a5e" +checksum = "a21f936df1771bf62b77f047b726c4625ff2e8aa607c01ec06e5a05bd8463401" dependencies = [ "android-tzdata", "iana-time-zone", @@ -502,7 +502,7 @@ dependencies = [ "num-traits", "serde", "wasm-bindgen", - "windows-targets 0.52.4", + "windows-targets 0.52.5", ] [[package]] @@ -546,7 +546,7 @@ dependencies = [ "heck 0.5.0", "proc-macro2", "quote", - "syn 2.0.58", + "syn 2.0.60", ] [[package]] @@ -697,9 +697,9 @@ dependencies = [ [[package]] name = "deadpool" -version = "0.11.1" +version = "0.11.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "144f5e4b9ce67c972acc225e71aefe6b21241276f94005024562874611064d30" +checksum = "ff0fc28638c21092aba483136debc6e177fff3dace8c835d715866923b03323e" dependencies = [ "deadpool-runtime", "num_cpus", @@ -756,7 +756,7 @@ checksum = "d150dea618e920167e5973d70ae6ece4385b7164e0d799fe7c122dd0a5d912ad" dependencies = [ "proc-macro2", "quote", - "syn 2.0.58", + "syn 2.0.60", ] [[package]] @@ -785,15 +785,15 @@ checksum = "0d6ef0072f8a535281e4876be788938b528e9a1d43900b82c2569af7da799125" [[package]] name = "either" -version = "1.10.0" +version = "1.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "11157ac094ffbdde99aa67b23417ebdd801842852b500e395a45a9c0aac03e4a" +checksum = "a47c1c47d2f5964e29c61246e81db715514cd532db6b5116a25ea3c03d6780a2" [[package]] name = "encoding_rs" -version = "0.8.33" +version = "0.8.34" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7268b386296a025e474d5140678f75d6de9493ae55a5d709eeb9dd08149945e1" +checksum = "b45de904aa0b010bce2ab45264d0631681847fa7b6f2eaa7dab7619943bc4f59" dependencies = [ "cfg-if", ] @@ -971,7 +971,7 @@ checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" dependencies = [ "proc-macro2", "quote", - "syn 2.0.58", + "syn 2.0.60", ] [[package]] @@ -1048,9 +1048,9 @@ dependencies = [ [[package]] name = "getrandom" -version = "0.2.13" +version = "0.2.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a06fddc2749e0528d2813f95e050e87e52c8cbbae56223b9babf73b3e53b0cc6" +checksum = "94b22e06ecb0110981051723910cbf0b5f5e09a2062dd7663334ee79a9d1286c" dependencies = [ "cfg-if", "js-sys", @@ -1251,9 +1251,9 @@ dependencies = [ [[package]] name = "hyper" -version = "1.2.0" +version = "1.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "186548d73ac615b32a73aafe38fb4f56c0d340e110e5a200bcadbaf2e199263a" +checksum = "fe575dd17d0862a9a33781c8c4696a55c320909004a67a00fb286ba8b1bc496d" dependencies = [ "bytes", "futures-channel", @@ -1278,7 +1278,7 @@ dependencies = [ "http 0.2.12", "hyper 0.14.28", "log", - "rustls 0.22.3", + "rustls 0.22.4", "rustls-native-certs", "rustls-pki-types", "tokio", @@ -1293,9 +1293,9 @@ checksum = "a0bea761b46ae2b24eb4aef630d8d1c398157b6fc29e6350ecf090a0b70c952c" dependencies = [ "futures-util", "http 1.1.0", - "hyper 1.2.0", + "hyper 1.3.1", "hyper-util", - "rustls 0.22.3", + "rustls 0.22.4", "rustls-pki-types", "tokio", "tokio-rustls 0.25.0", @@ -1325,7 +1325,7 @@ dependencies = [ "futures-util", "http 1.1.0", "http-body 1.0.0", - "hyper 1.2.0", + "hyper 1.3.1", "pin-project-lite", "socket2", "tokio", @@ -1945,11 +1945,11 @@ dependencies = [ [[package]] name = "pem" -version = "3.0.3" +version = "3.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1b8fcc794035347fb64beda2d3b462595dd2753e3f268d89c5aae77e8cf2c310" +checksum = "8e459365e590736a54c3fa561947c84837534b8e9af6fc5bf781307e82658fae" dependencies = [ - "base64 0.21.7", + "base64 0.22.0", "serde", ] @@ -2040,7 +2040,7 @@ checksum = "2f38a4412a78282e09a2cf38d195ea5420d15ba0602cb375210efbc877243965" dependencies = [ "proc-macro2", "quote", - "syn 2.0.58", + "syn 2.0.60", ] [[package]] @@ -2119,7 +2119,7 @@ version = "0.1.0" dependencies = [ "anyhow", "pt", - "rustls 0.23.4", + "rustls 0.23.5", "tokio", "tokio-postgres", "tokio-postgres-rustls", @@ -2185,12 +2185,12 @@ checksum = "5b40af805b3121feab8a3c29f04d8ad262fa8e0561883e7653e024ae4479e6de" [[package]] name = "prettyplease" -version = "0.2.17" +version = "0.2.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8d3928fb5db768cb86f891ff014f0144589297e3c6a1aba6ed7cecfdace270c7" +checksum = "5ac2cf0f2e4f42b49f5ffd07dae8d746508ef7526c13940e5f524012ae6c6550" dependencies = [ "proc-macro2", - "syn 2.0.58", + "syn 2.0.60", ] [[package]] @@ -2227,9 +2227,9 @@ dependencies = [ [[package]] name = "proc-macro2" -version = "1.0.79" +version = "1.0.81" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e835ff2298f5721608eb1a980ecaee1aef2c132bf95ecc026a11b7bf3c01c02e" +checksum = "3d1597b0c024618f09a9c3b8655b7e430397a36d23fdafec26d6965e9eec3eba" dependencies = [ "unicode-ident", ] @@ -2261,7 +2261,7 @@ dependencies = [ "prost", "prost-types", "regex", - "syn 2.0.58", + "syn 2.0.60", "tempfile", ] @@ -2275,7 +2275,7 @@ dependencies = [ "itertools 0.12.1", "proc-macro2", "quote", - "syn 2.0.58", + "syn 2.0.60", ] [[package]] @@ -2331,9 +2331,9 @@ checksum = "a993555f31e5a609f617c12db6250dedcac1b0a85076912c436e6fc9b2c8e6a3" [[package]] name = "quote" -version = "1.0.35" +version = "1.0.36" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "291ec9ab5efd934aaf503a6466c5d5251535d108ee747472c3977cc5acc868ef" +checksum = "0fa76aaf39101c457836aec0ce2316dbdc3ab723cdda1c6bd4e6ad4208acaca7" dependencies = [ "proc-macro2", ] @@ -2443,7 +2443,7 @@ dependencies = [ "quote", "refinery-core", "regex", - "syn 2.0.58", + "syn 2.0.60", ] [[package]] @@ -2501,9 +2501,9 @@ dependencies = [ [[package]] name = "reqwest" -version = "0.12.3" +version = "0.12.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3e6cc1e89e689536eb5aeede61520e874df5a4707df811cd5da4aa5fbb2aae19" +checksum = "566cafdd92868e0939d3fb961bd0dc25fcfaaed179291093b3d43e6b3150ea10" dependencies = [ "async-compression", "base64 0.22.0", @@ -2513,7 +2513,7 @@ dependencies = [ "http 1.1.0", "http-body 1.0.0", "http-body-util", - "hyper 1.2.0", + "hyper 1.3.1", "hyper-rustls 0.26.0", "hyper-util", "ipnet", @@ -2523,7 +2523,7 @@ dependencies = [ "once_cell", "percent-encoding", "pin-project-lite", - "rustls 0.22.3", + "rustls 0.22.4", "rustls-pemfile 2.1.2", "rustls-pki-types", "serde", @@ -2631,9 +2631,9 @@ checksum = "d626bb9dae77e28219937af045c257c28bfd3f69333c512553507f5f9798cb76" [[package]] name = "rustix" -version = "0.38.32" +version = "0.38.33" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "65e04861e65f21776e67888bfbea442b3642beaa0138fdb1dd7a84a52dffdb89" +checksum = "e3cc72858054fcff6d7dea32df2aeaee6a7c24227366d7ea429aada2f26b16ad" dependencies = [ "bitflags 2.5.0", "errno", @@ -2644,9 +2644,9 @@ dependencies = [ [[package]] name = "rustls" -version = "0.22.3" +version = "0.22.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "99008d7ad0bbbea527ec27bddbc0e432c5b87d8175178cee68d2eec9c4a1813c" +checksum = "bf4ef73721ac7bcd79b2b315da7779d8fc09718c6b3d2d1b2d94850eb8c18432" dependencies = [ "log", "ring", @@ -2658,9 +2658,9 @@ dependencies = [ [[package]] name = "rustls" -version = "0.23.4" +version = "0.23.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8c4d6d8ad9f2492485e13453acbb291dd08f64441b6609c491f1c2cd2c6b4fe1" +checksum = "afabcee0551bd1aa3e18e5adbf2c0544722014b899adb31bd186ec638d3da97e" dependencies = [ "once_cell", "ring", @@ -2815,9 +2815,9 @@ dependencies = [ [[package]] name = "serde" -version = "1.0.197" +version = "1.0.198" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3fb1c873e1b9b056a4dc4c0c198b24c3ffa059243875552b2bd0933b1aee4ce2" +checksum = "9846a40c979031340571da2545a4e5b7c4163bdae79b301d5f86d03979451fcc" dependencies = [ "serde_derive", ] @@ -2833,20 +2833,20 @@ dependencies = [ [[package]] name = "serde_derive" -version = "1.0.197" +version = "1.0.198" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7eb0b34b42edc17f6b7cac84a52a1c5f0e1bb2227e997ca9011ea3dd34e8610b" +checksum = "e88edab869b01783ba905e7d0153f9fc1a6505a96e4ad3018011eedb838566d9" dependencies = [ "proc-macro2", "quote", - "syn 2.0.58", + "syn 2.0.60", ] [[package]] name = "serde_json" -version = "1.0.115" +version = "1.0.116" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "12dc5c46daa8e9fdf4f5e71b6cf9a53f2487da0e86e55808e2d35539666497dd" +checksum = "3e17db7126d17feb94eb3fad46bf1a96b034e8aacbc2e775fe81505f8b0b2813" dependencies = [ "itoa", "ryu", @@ -2909,9 +2909,9 @@ dependencies = [ [[package]] name = "signal-hook-registry" -version = "1.4.1" +version = "1.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d8229b473baa5980ac72ef434c4415e70c4b5e71b423043adb4ba059f89c99a1" +checksum = "a9e9e0b4211b72e7b8b6e85c807d36c212bdb33ea8587f7569562a84df5465b1" dependencies = [ "libc", ] @@ -3012,7 +3012,7 @@ dependencies = [ [[package]] name = "sqlparser" version = "0.41.0" -source = "git+https://github.com/peerdb-io/sqlparser-rs.git#178a84c3c48123a2818298d87748033581886eae" +source = "git+https://github.com/peerdb-io/sqlparser-rs.git?branch=ps-es-parsing#e86bcd9ed3dd9ac7be94569f0ae1fc20c440dafa" dependencies = [ "log", "sqlparser_derive", @@ -3021,11 +3021,11 @@ dependencies = [ [[package]] name = "sqlparser_derive" version = "0.2.1" -source = "git+https://github.com/peerdb-io/sqlparser-rs.git#178a84c3c48123a2818298d87748033581886eae" +source = "git+https://github.com/peerdb-io/sqlparser-rs.git?branch=ps-es-parsing#e86bcd9ed3dd9ac7be94569f0ae1fc20c440dafa" dependencies = [ "proc-macro2", "quote", - "syn 2.0.58", + "syn 2.0.60", ] [[package]] @@ -3064,9 +3064,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.58" +version = "2.0.60" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "44cfb93f38070beee36b3fef7d4f5a16f27751d94b187b666a5cc5e9b0d30687" +checksum = "909518bc7b1c9b779f1bbf07f2929d35af9f0f37e47c6e9ef7f9dddc1e1821f3" dependencies = [ "proc-macro2", "quote", @@ -3082,7 +3082,7 @@ dependencies = [ "proc-macro-error", "proc-macro2", "quote", - "syn 2.0.58", + "syn 2.0.60", ] [[package]] @@ -3121,22 +3121,22 @@ dependencies = [ [[package]] name = "thiserror" -version = "1.0.58" +version = "1.0.59" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "03468839009160513471e86a034bb2c5c0e4baae3b43f79ffc55c4a5427b3297" +checksum = "f0126ad08bff79f29fc3ae6a55cc72352056dfff61e3ff8bb7129476d44b23aa" dependencies = [ "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "1.0.58" +version = "1.0.59" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c61f3ba182994efc43764a46c018c347bc492c79f024e705f46567b418f6d4f7" +checksum = "d1cd413b5d558b4c5bf3680e324a6fa5014e7b7c067a51e69dbdf47eb7148b66" dependencies = [ "proc-macro2", "quote", - "syn 2.0.58", + "syn 2.0.60", ] [[package]] @@ -3151,9 +3151,9 @@ dependencies = [ [[package]] name = "time" -version = "0.3.34" +version = "0.3.36" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c8248b6521bb14bc45b4067159b9b6ad792e2d6d754d6c41fb50e29fefe38749" +checksum = "5dfd88e563464686c916c7e46e623e520ddc6d79fa6641390f2e3fa86e83e885" dependencies = [ "deranged", "itoa", @@ -3174,9 +3174,9 @@ checksum = "ef927ca75afb808a4d64dd374f00a2adf8d0fcff8e7b184af886c3c87ec4a3f3" [[package]] name = "time-macros" -version = "0.2.17" +version = "0.2.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7ba3a3ef41e6672a2f0f001392bb5dcd3ff0a9992d618ca761a11c3121547774" +checksum = "3f252a68540fde3a3877aeea552b832b40ab9a69e318efd078774a01ddee1ccf" dependencies = [ "num-conv", "time-core", @@ -3235,7 +3235,7 @@ checksum = "5b8a1e28f2deaa14e508979454cb3a223b10b938b45af148bc0986de36f1923b" dependencies = [ "proc-macro2", "quote", - "syn 2.0.58", + "syn 2.0.60", ] [[package]] @@ -3271,7 +3271,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "04fb792ccd6bbcd4bba408eb8a292f70fc4a3589e5d793626f45190e6454b6ab" dependencies = [ "ring", - "rustls 0.23.4", + "rustls 0.23.5", "tokio", "tokio-postgres", "tokio-rustls 0.26.0", @@ -3284,7 +3284,7 @@ version = "0.25.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "775e0c0f0adb3a2f22a00c4745d728b479985fc15ee7ca6a2608388c5569860f" dependencies = [ - "rustls 0.22.3", + "rustls 0.22.4", "rustls-pki-types", "tokio", ] @@ -3295,7 +3295,7 @@ version = "0.26.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0c7bc40d0e5a97695bb96e27995cd3a08538541b0a846f65bba7a359f36700d4" dependencies = [ - "rustls 0.23.4", + "rustls 0.23.5", "rustls-pki-types", "tokio", ] @@ -3334,7 +3334,7 @@ dependencies = [ "serde", "serde_spanned", "toml_datetime", - "toml_edit 0.22.9", + "toml_edit 0.22.12", ] [[package]] @@ -3359,15 +3359,15 @@ dependencies = [ [[package]] name = "toml_edit" -version = "0.22.9" +version = "0.22.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8e40bb779c5187258fd7aad0eb68cb8706a0a81fa712fbea808ab43c4b8374c4" +checksum = "d3328d4f68a705b2a4498da1d580585d39a6510f98318a2cec3018a7ec61ddef" dependencies = [ "indexmap 2.2.6", "serde", "serde_spanned", "toml_datetime", - "winnow 0.6.5", + "winnow 0.6.6", ] [[package]] @@ -3514,7 +3514,7 @@ checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.58", + "syn 2.0.60", ] [[package]] @@ -3618,7 +3618,7 @@ dependencies = [ "flate2", "log", "once_cell", - "rustls 0.22.3", + "rustls 0.22.4", "rustls-pki-types", "rustls-webpki", "serde", @@ -3742,7 +3742,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.58", + "syn 2.0.60", "wasm-bindgen-shared", ] @@ -3776,7 +3776,7 @@ checksum = "e94f17b526d0a461a191c78ea52bbce64071ed5c04c9ffe424dcb38f74171bb7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.58", + "syn 2.0.60", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -3854,7 +3854,7 @@ version = "0.52.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "33ab640c8d7e35bf8ba19b884ba838ceb4fba93a4e8c65a9059d08afcfc683d9" dependencies = [ - "windows-targets 0.52.4", + "windows-targets 0.52.5", ] [[package]] @@ -3872,7 +3872,7 @@ version = "0.52.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "282be5f36a8ce781fad8c8ae18fa3f9beff57ec1b52cb3de0789201425d9a33d" dependencies = [ - "windows-targets 0.52.4", + "windows-targets 0.52.5", ] [[package]] @@ -3892,17 +3892,18 @@ dependencies = [ [[package]] name = "windows-targets" -version = "0.52.4" +version = "0.52.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7dd37b7e5ab9018759f893a1952c9420d060016fc19a472b4bb20d1bdd694d1b" +checksum = "6f0713a46559409d202e70e28227288446bf7841d3211583a4b53e3f6d96e7eb" dependencies = [ - "windows_aarch64_gnullvm 0.52.4", - "windows_aarch64_msvc 0.52.4", - "windows_i686_gnu 0.52.4", - "windows_i686_msvc 0.52.4", - "windows_x86_64_gnu 0.52.4", - "windows_x86_64_gnullvm 0.52.4", - "windows_x86_64_msvc 0.52.4", + "windows_aarch64_gnullvm 0.52.5", + "windows_aarch64_msvc 0.52.5", + "windows_i686_gnu 0.52.5", + "windows_i686_gnullvm", + "windows_i686_msvc 0.52.5", + "windows_x86_64_gnu 0.52.5", + "windows_x86_64_gnullvm 0.52.5", + "windows_x86_64_msvc 0.52.5", ] [[package]] @@ -3913,9 +3914,9 @@ checksum = "2b38e32f0abccf9987a4e3079dfb67dcd799fb61361e53e2882c3cbaf0d905d8" [[package]] name = "windows_aarch64_gnullvm" -version = "0.52.4" +version = "0.52.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bcf46cf4c365c6f2d1cc93ce535f2c8b244591df96ceee75d8e83deb70a9cac9" +checksum = "7088eed71e8b8dda258ecc8bac5fb1153c5cffaf2578fc8ff5d61e23578d3263" [[package]] name = "windows_aarch64_msvc" @@ -3925,9 +3926,9 @@ checksum = "dc35310971f3b2dbbf3f0690a219f40e2d9afcf64f9ab7cc1be722937c26b4bc" [[package]] name = "windows_aarch64_msvc" -version = "0.52.4" +version = "0.52.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "da9f259dd3bcf6990b55bffd094c4f7235817ba4ceebde8e6d11cd0c5633b675" +checksum = "9985fd1504e250c615ca5f281c3f7a6da76213ebd5ccc9561496568a2752afb6" [[package]] name = "windows_i686_gnu" @@ -3937,9 +3938,15 @@ checksum = "a75915e7def60c94dcef72200b9a8e58e5091744960da64ec734a6c6e9b3743e" [[package]] name = "windows_i686_gnu" -version = "0.52.4" +version = "0.52.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b474d8268f99e0995f25b9f095bc7434632601028cf86590aea5c8a5cb7801d3" +checksum = "88ba073cf16d5372720ec942a8ccbf61626074c6d4dd2e745299726ce8b89670" + +[[package]] +name = "windows_i686_gnullvm" +version = "0.52.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "87f4261229030a858f36b459e748ae97545d6f1ec60e5e0d6a3d32e0dc232ee9" [[package]] name = "windows_i686_msvc" @@ -3949,9 +3956,9 @@ checksum = "8f55c233f70c4b27f66c523580f78f1004e8b5a8b659e05a4eb49d4166cca406" [[package]] name = "windows_i686_msvc" -version = "0.52.4" +version = "0.52.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1515e9a29e5bed743cb4415a9ecf5dfca648ce85ee42e15873c3cd8610ff8e02" +checksum = "db3c2bf3d13d5b658be73463284eaf12830ac9a26a90c717b7f771dfe97487bf" [[package]] name = "windows_x86_64_gnu" @@ -3961,9 +3968,9 @@ checksum = "53d40abd2583d23e4718fddf1ebec84dbff8381c07cae67ff7768bbf19c6718e" [[package]] name = "windows_x86_64_gnu" -version = "0.52.4" +version = "0.52.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5eee091590e89cc02ad514ffe3ead9eb6b660aedca2183455434b93546371a03" +checksum = "4e4246f76bdeff09eb48875a0fd3e2af6aada79d409d33011886d3e1581517d9" [[package]] name = "windows_x86_64_gnullvm" @@ -3973,9 +3980,9 @@ checksum = "0b7b52767868a23d5bab768e390dc5f5c55825b6d30b86c844ff2dc7414044cc" [[package]] name = "windows_x86_64_gnullvm" -version = "0.52.4" +version = "0.52.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "77ca79f2451b49fa9e2af39f0747fe999fcda4f5e241b2898624dca97a1f2177" +checksum = "852298e482cd67c356ddd9570386e2862b5673c85bd5f88df9ab6802b334c596" [[package]] name = "windows_x86_64_msvc" @@ -3985,9 +3992,9 @@ checksum = "ed94fce61571a4006852b7389a063ab983c02eb1bb37b47f8272ce92d06d9538" [[package]] name = "windows_x86_64_msvc" -version = "0.52.4" +version = "0.52.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "32b752e52a2da0ddfbdbcc6fceadfeede4c939ed16d13e648833a61dfb611ed8" +checksum = "bec47e5bfd1bff0eeaf6d8b485cc1074891a197ab4225d504cb7a1ab88b02bf0" [[package]] name = "winnow" @@ -4000,9 +4007,9 @@ dependencies = [ [[package]] name = "winnow" -version = "0.6.5" +version = "0.6.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dffa400e67ed5a4dd237983829e66475f0a4a26938c4b04c21baede6262215b8" +checksum = "f0c976aaaa0e1f90dbb21e9587cdaf1d9679a1cde8875c0d6bd83ab96a208352" dependencies = [ "memchr", ] @@ -4070,7 +4077,7 @@ dependencies = [ "itertools 0.12.1", "log", "percent-encoding", - "rustls 0.22.3", + "rustls 0.22.4", "rustls-pemfile 1.0.4", "seahash", "serde", @@ -4098,7 +4105,7 @@ checksum = "ce36e65b0d2999d2aafac989fb249189a141aee1f53c612c1f37d72631959f69" dependencies = [ "proc-macro2", "quote", - "syn 2.0.58", + "syn 2.0.60", ] [[package]] diff --git a/nexus/Cargo.toml b/nexus/Cargo.toml index 2f031ff918..3e4a6a062d 100644 --- a/nexus/Cargo.toml +++ b/nexus/Cargo.toml @@ -19,5 +19,5 @@ resolver = "2" [workspace.dependencies] chrono = { version = "0.4", default-features = false, features = ["serde", "std"] } -sqlparser = { git = "https://github.com/peerdb-io/sqlparser-rs.git" } +sqlparser = { git = "https://github.com/peerdb-io/sqlparser-rs.git", branch = "ps-es-parsing" } pgwire = "0.19" diff --git a/nexus/analyzer/src/lib.rs b/nexus/analyzer/src/lib.rs index 1feae83ed4..3fb7861e03 100644 --- a/nexus/analyzer/src/lib.rs +++ b/nexus/analyzer/src/lib.rs @@ -104,11 +104,11 @@ pub enum PeerDDL { }, CreateMirrorForCDC { if_not_exists: bool, - flow_job: FlowJob, + flow_job: Box, }, CreateMirrorForSelect { if_not_exists: bool, - qrep_flow_job: QRepFlowJob, + qrep_flow_job: Box, }, ExecuteMirrorForSelect { flow_job_name: String, @@ -309,6 +309,11 @@ impl StatementAnalyzer for PeerDDLAnalyzer { _ => String::new(), }; + let system = match raw_options.remove("system") { + Some(sqlparser::ast::Value::SingleQuotedString(s)) => s.clone(), + _ => "Q".to_string(), + }; + let flow_job = FlowJob { name: cdc.mirror_name.to_string().to_lowercase(), source_peer: cdc.source_peer.to_string().to_lowercase(), @@ -332,6 +337,7 @@ impl StatementAnalyzer for PeerDDLAnalyzer { synced_at_col_name, initial_snapshot_only: initial_copy_only, script, + system, }; if initial_copy_only && !do_initial_copy { @@ -340,7 +346,7 @@ impl StatementAnalyzer for PeerDDLAnalyzer { Ok(Some(PeerDDL::CreateMirrorForCDC { if_not_exists: *if_not_exists, - flow_job, + flow_job: Box::new(flow_job), })) } Select(select) => { @@ -372,7 +378,7 @@ impl StatementAnalyzer for PeerDDLAnalyzer { Ok(Some(PeerDDL::CreateMirrorForSelect { if_not_exists: *if_not_exists, - qrep_flow_job, + qrep_flow_job: Box::new(qrep_flow_job), })) } } @@ -848,8 +854,53 @@ fn parse_db_options(db_type: DbType, with_options: &[SqlOption]) -> anyhow::Resu unnest_columns, }; - println!("eventhub_group_config: {:?}", eventhub_group_config); Config::EventhubGroupConfig(eventhub_group_config) } + DbType::Elasticsearch => { + let addresses = opts + .get("addresses") + .map(|columns| { + columns + .split(',') + .map(|column| column.trim().to_string()) + .collect::>() + }) + .ok_or_else(|| anyhow::anyhow!("missing connection addresses for Elasticsearch"))?; + + // either basic auth or API key auth, not both + let api_key = opts.get("api_key").map(|s| s.to_string()); + let username = opts.get("username").map(|s| s.to_string()); + let password = opts.get("password").map(|s| s.to_string()); + if api_key.is_some() { + if username.is_some() || password.is_some() { + return Err(anyhow::anyhow!( + "both API key auth and basic auth specified" + )); + } + Config::ElasticsearchConfig(pt::peerdb_peers::ElasticsearchConfig { + addresses, + auth_type: pt::peerdb_peers::ElasticsearchAuthType::Apikey.into(), + username: None, + password: None, + api_key, + }) + } else if username.is_some() && password.is_some() { + Config::ElasticsearchConfig(pt::peerdb_peers::ElasticsearchConfig { + addresses, + auth_type: pt::peerdb_peers::ElasticsearchAuthType::Basic.into(), + username, + password, + api_key: None, + }) + } else { + Config::ElasticsearchConfig(pt::peerdb_peers::ElasticsearchConfig { + addresses, + auth_type: pt::peerdb_peers::ElasticsearchAuthType::None.into(), + username: None, + password: None, + api_key: None, + }) + } + } })) } diff --git a/nexus/catalog/src/lib.rs b/nexus/catalog/src/lib.rs index 3e01159f9d..2c53000759 100644 --- a/nexus/catalog/src/lib.rs +++ b/nexus/catalog/src/lib.rs @@ -102,6 +102,9 @@ impl Catalog { Config::ClickhouseConfig(clickhouse_config) => clickhouse_config.encode_to_vec(), Config::KafkaConfig(kafka_config) => kafka_config.encode_to_vec(), Config::PubsubConfig(pubsub_config) => pubsub_config.encode_to_vec(), + Config::ElasticsearchConfig(elasticsearch_config) => { + elasticsearch_config.encode_to_vec() + } } }; @@ -313,6 +316,11 @@ impl Catalog { pt::peerdb_peers::PubSubConfig::decode(options).with_context(err)?; Config::PubsubConfig(pubsub_config) } + DbType::Elasticsearch => { + let elasticsearch_config = + pt::peerdb_peers::ElasticsearchConfig::decode(options).with_context(err)?; + Config::ElasticsearchConfig(elasticsearch_config) + } }) } else { None diff --git a/nexus/flow-rs/src/grpc.rs b/nexus/flow-rs/src/grpc.rs index 3425a1b29f..35382ca287 100644 --- a/nexus/flow-rs/src/grpc.rs +++ b/nexus/flow-rs/src/grpc.rs @@ -1,7 +1,7 @@ use catalog::WorkflowDetails; use pt::{ flow_model::{FlowJob, QRepFlowJob}, - peerdb_flow::{QRepWriteMode, QRepWriteType}, + peerdb_flow::{QRepWriteMode, QRepWriteType, TypeSystem}, peerdb_route, tonic, }; use serde_json::Value; @@ -134,15 +134,16 @@ impl FlowGrpcClient { src: pt::peerdb_peers::Peer, dst: pt::peerdb_peers::Peer, ) -> anyhow::Result { - let mut table_mappings: Vec = vec![]; - job.table_mappings.iter().for_each(|mapping| { - table_mappings.push(pt::peerdb_flow::TableMapping { + let table_mappings: Vec = job + .table_mappings + .iter() + .map(|mapping| pt::peerdb_flow::TableMapping { source_table_identifier: mapping.source_table_identifier.clone(), destination_table_identifier: mapping.destination_table_identifier.clone(), partition_key: mapping.partition_key.clone().unwrap_or_default(), exclude: mapping.exclude.clone(), - }); - }); + }) + .collect::>(); let do_initial_snapshot = job.do_initial_copy; let publication_name = job.publication_name.clone(); @@ -150,6 +151,9 @@ impl FlowGrpcClient { let snapshot_num_rows_per_partition = job.snapshot_num_rows_per_partition; let snapshot_max_parallel_workers = job.snapshot_max_parallel_workers; let snapshot_num_tables_in_parallel = job.snapshot_num_tables_in_parallel; + let Some(system) = TypeSystem::from_str_name(&job.system) else { + return anyhow::Result::Err(anyhow::anyhow!("invalid system {}", job.system)); + }; let flow_conn_cfg = pt::peerdb_flow::FlowConnectionConfigs { source: Some(src), @@ -171,6 +175,7 @@ impl FlowGrpcClient { synced_at_col_name: job.synced_at_col_name.clone().unwrap_or_default(), initial_snapshot_only: job.initial_snapshot_only, script: job.script.clone(), + system: system as i32, ..Default::default() }; diff --git a/nexus/pt/src/flow_model.rs b/nexus/pt/src/flow_model.rs index 2946dd1d30..8dd40edc42 100644 --- a/nexus/pt/src/flow_model.rs +++ b/nexus/pt/src/flow_model.rs @@ -35,6 +35,7 @@ pub struct FlowJob { pub synced_at_col_name: Option, pub initial_snapshot_only: bool, pub script: String, + pub system: String, } #[derive(Debug, PartialEq, Eq, Serialize, Deserialize, Clone)] diff --git a/nexus/pt/src/lib.rs b/nexus/pt/src/lib.rs index d04c615aee..23487c80b3 100644 --- a/nexus/pt/src/lib.rs +++ b/nexus/pt/src/lib.rs @@ -28,6 +28,8 @@ impl From for DbType { PeerType::SQLServer => DbType::Sqlserver, PeerType::Kafka => DbType::Kafka, PeerType::Eventhubs => DbType::Eventhubs, + PeerType::PubSub => DbType::Pubsub, + PeerType::Elasticsearch => DbType::Elasticsearch, } } } diff --git a/protos/flow.proto b/protos/flow.proto index 502520e1ea..46516fea29 100644 --- a/protos/flow.proto +++ b/protos/flow.proto @@ -71,6 +71,8 @@ message FlowConnectionConfigs { string synced_at_col_name = 19; string script = 20; + + TypeSystem system = 21; } message RenameTableOption { @@ -161,6 +163,7 @@ message TableSchema { string table_identifier = 1; repeated string primary_key_columns = 2; bool is_replica_identity_full = 3; + TypeSystem system = 4; repeated FieldDescription columns = 6; } @@ -174,18 +177,13 @@ message GetTableSchemaBatchInput { peerdb_peers.Peer peer_connection_config = 1; repeated string table_identifiers = 2; string flow_name = 3; + TypeSystem system = 4; } message GetTableSchemaBatchOutput { map table_name_schema_mapping = 1; } -message SetupNormalizedTableInput { - peerdb_peers.Peer peer_connection_config = 1; - string table_identifier = 2; - TableSchema source_table_schema = 3; -} - message SetupNormalizedTableBatchInput { peerdb_peers.Peer peer_connection_config = 1; map table_name_schema_mapping = 2; @@ -248,6 +246,11 @@ message QRepWriteMode { repeated string upsert_key_columns = 2; } +enum TypeSystem { + Q = 0; + PG = 1; +} + message QRepConfig { string flow_job_name = 1; @@ -292,6 +295,8 @@ message QRepConfig { string synced_at_col_name = 16; string soft_delete_col_name = 17; + + TypeSystem system = 18; } message QRepPartition { @@ -313,15 +318,11 @@ message DropFlowInput { string flow_name = 1; } -message DeltaAddedColumn { - string column_name = 1; - string column_type = 2; -} - message TableSchemaDelta { string src_table_name = 1; string dst_table_name = 2; - repeated DeltaAddedColumn added_columns = 3; + repeated FieldDescription added_columns = 3; + TypeSystem system = 4; } message QRepFlowState { diff --git a/protos/peers.proto b/protos/peers.proto index 360e08f311..805c765099 100644 --- a/protos/peers.proto +++ b/protos/peers.proto @@ -134,6 +134,22 @@ message KafkaConfig { string partitioner = 6; } +enum ElasticsearchAuthType { + UNKNOWN = 0; + NONE = 1; + BASIC = 2; + APIKEY = 3; +} + +message ElasticsearchConfig { + // decide if this is something actually used or single address is enough + repeated string addresses = 1; + ElasticsearchAuthType auth_type = 2; + optional string username = 3; + optional string password = 4; + optional string api_key = 5; +} + enum DBType { BIGQUERY = 0; SNOWFLAKE = 1; @@ -145,6 +161,7 @@ enum DBType { KAFKA = 9; PUBSUB = 10; EVENTHUBS = 11; + ELASTICSEARCH = 12; } message Peer { @@ -162,5 +179,6 @@ message Peer { ClickhouseConfig clickhouse_config = 11; KafkaConfig kafka_config = 12; PubSubConfig pubsub_config = 13; + ElasticsearchConfig elasticsearch_config = 14; } } diff --git a/ui/app/api/peers/getTruePeer.ts b/ui/app/api/peers/getTruePeer.ts index 8644ce6662..e21323a07b 100644 --- a/ui/app/api/peers/getTruePeer.ts +++ b/ui/app/api/peers/getTruePeer.ts @@ -2,6 +2,7 @@ import { CatalogPeer } from '@/app/dto/PeersDTO'; import { BigqueryConfig, ClickhouseConfig, + ElasticsearchConfig, EventHubConfig, EventHubGroupConfig, KafkaConfig, @@ -29,7 +30,8 @@ export const getTruePeer = (peer: CatalogPeer) => { | PubSubConfig | S3Config | SnowflakeConfig - | SqlServerConfig; + | SqlServerConfig + | ElasticsearchConfig; switch (peer.type) { case 0: config = BigqueryConfig.decode(options); @@ -75,6 +77,10 @@ export const getTruePeer = (peer: CatalogPeer) => { config = EventHubGroupConfig.decode(options); newPeer.eventhubGroupConfig = config; break; + case 12: + config = ElasticsearchConfig.decode(options); + newPeer.elasticsearchConfig = config; + break; default: return newPeer; } diff --git a/ui/app/api/peers/route.ts b/ui/app/api/peers/route.ts index 1ca8f21825..4f468d8fb2 100644 --- a/ui/app/api/peers/route.ts +++ b/ui/app/api/peers/route.ts @@ -10,6 +10,7 @@ import { BigqueryConfig, ClickhouseConfig, DBType, + ElasticsearchConfig, EventHubGroupConfig, KafkaConfig, Peer, @@ -84,6 +85,12 @@ const constructPeer = ( type: DBType.EVENTHUBS, eventhubGroupConfig: config as EventHubGroupConfig, }; + case 'ELASTICSEARCH': + return { + name, + type: DBType.ELASTICSEARCH, + elasticsearchConfig: config as ElasticsearchConfig, + }; default: return; } diff --git a/ui/app/dto/PeersDTO.ts b/ui/app/dto/PeersDTO.ts index 0e05e7e2c9..abc3bc80fb 100644 --- a/ui/app/dto/PeersDTO.ts +++ b/ui/app/dto/PeersDTO.ts @@ -1,6 +1,7 @@ import { BigqueryConfig, ClickhouseConfig, + ElasticsearchConfig, EventHubConfig, EventHubGroupConfig, KafkaConfig, @@ -51,7 +52,8 @@ export type PeerConfig = | KafkaConfig | PubSubConfig | EventHubConfig - | EventHubGroupConfig; + | EventHubGroupConfig + | ElasticsearchConfig; export type CatalogPeer = { id: number; name: string; diff --git a/ui/app/mirrors/[mirrorId]/cdcDetails.tsx b/ui/app/mirrors/[mirrorId]/cdcDetails.tsx index 7b0cfe9a74..456bacb0c7 100644 --- a/ui/app/mirrors/[mirrorId]/cdcDetails.tsx +++ b/ui/app/mirrors/[mirrorId]/cdcDetails.tsx @@ -5,13 +5,12 @@ import PeerButton from '@/components/PeerComponent'; import TimeLabel from '@/components/TimeComponent'; import { FlowConnectionConfigs, FlowStatus } from '@/grpc_generated/flow'; import { dBTypeFromJSON } from '@/grpc_generated/peers'; -import { FlowStateChangeRequest } from '@/grpc_generated/route'; -import { Button } from '@/lib/Button'; -import { Icon } from '@/lib/Icon'; import { Label } from '@/lib/Label'; -import moment from 'moment'; +import { ProgressCircle } from '@/lib/ProgressCircle'; import Link from 'next/link'; +import { useEffect, useState } from 'react'; import MirrorValues from './configValues'; +import { getCurrentIdleTimeout } from './handlers'; import { RowDataFormatter } from './rowsDisplay'; import TablePairs from './tablePairs'; @@ -22,13 +21,8 @@ type props = { mirrorStatus: FlowStatus; }; function CdcDetails({ syncs, createdAt, mirrorConfig, mirrorStatus }: props) { - let lastSyncedAt = moment( - syncs.length > 1 - ? syncs[1]?.endTime - : syncs.length - ? syncs[0]?.startTime - : new Date() - ).fromNow(); + const [syncInterval, getSyncInterval] = useState(); + let rowsSynced = syncs.reduce((acc, sync) => { if (sync.endTime !== null) { return acc + sync.numRows; @@ -37,6 +31,11 @@ function CdcDetails({ syncs, createdAt, mirrorConfig, mirrorStatus }: props) { }, 0); const tablesSynced = mirrorConfig.tableMappings; + useEffect(() => { + getCurrentIdleTimeout(mirrorConfig.flowJobName).then((res) => { + getSyncInterval(res); + }); + }, [mirrorConfig.flowJobName]); return ( <>
@@ -53,15 +52,11 @@ function CdcDetails({ syncs, createdAt, mirrorConfig, mirrorStatus }: props) { width: 'fit-content', borderRadius: '1rem', border: '1px solid rgba(0,0,0,0.1)', - cursor: 'pointer', - display: 'flex', - alignItems: 'center', }} > - {statusChangeHandle(mirrorConfig, mirrorStatus)}
@@ -105,11 +100,11 @@ function CdcDetails({ syncs, createdAt, mirrorConfig, mirrorStatus }: props) {
- +
@@ -144,71 +139,6 @@ function CdcDetails({ syncs, createdAt, mirrorConfig, mirrorStatus }: props) { ); } -function statusChangeHandle( - mirrorConfig: FlowConnectionConfigs, - mirrorStatus: FlowStatus -) { - // hopefully there's a better way to do this cast - if (mirrorStatus.toString() === FlowStatus[FlowStatus.STATUS_RUNNING]) { - return ( - - ); - } else if (mirrorStatus.toString() === FlowStatus[FlowStatus.STATUS_PAUSED]) { - return ( - - ); - } else { - return ( - - ); - } -} - function formatStatus(mirrorStatus: FlowStatus) { const mirrorStatusLower = mirrorStatus .toString() @@ -220,4 +150,25 @@ function formatStatus(mirrorStatus: FlowStatus) { ); } +const SyncIntervalLabel: React.FC<{ syncInterval?: number }> = ({ + syncInterval, +}) => { + let formattedInterval: string; + + if (!syncInterval) { + return ; + } + if (syncInterval >= 3600) { + const hours = Math.floor(syncInterval / 3600); + formattedInterval = `${hours} hour${hours !== 1 ? 's' : ''}`; + } else if (syncInterval >= 60) { + const minutes = Math.floor(syncInterval / 60); + formattedInterval = `${minutes} minute${minutes !== 1 ? 's' : ''}`; + } else { + formattedInterval = `${syncInterval} second${syncInterval !== 1 ? 's' : ''}`; + } + + return ; +}; + export default CdcDetails; diff --git a/ui/app/mirrors/[mirrorId]/edit/page.tsx b/ui/app/mirrors/[mirrorId]/edit/page.tsx index 9f65a72d8b..9c738fb712 100644 --- a/ui/app/mirrors/[mirrorId]/edit/page.tsx +++ b/ui/app/mirrors/[mirrorId]/edit/page.tsx @@ -21,6 +21,7 @@ import TableMapping from '../../create/cdc/tablemapping'; import { reformattedTableMapping } from '../../create/handlers'; import { blankCDCSetting } from '../../create/helpers/common'; import * as styles from '../../create/styles'; +import { getMirrorState } from '../handlers'; type EditMirrorProps = { params: { mirrorId: string }; @@ -41,26 +42,19 @@ const EditMirror = ({ params: { mirrorId } }: EditMirrorProps) => { const { push } = useRouter(); const fetchStateAndUpdateDeps = useCallback(async () => { - await fetch('/api/mirrors/state', { - method: 'POST', - body: JSON.stringify({ - flowJobName: mirrorId, - }), - }) - .then((res) => res.json()) - .then((res) => { - setMirrorState(res); - - setConfig({ - batchSize: - (res as MirrorStatusResponse).cdcStatus?.config?.maxBatchSize || - defaultBatchSize, - idleTimeout: - (res as MirrorStatusResponse).cdcStatus?.config - ?.idleTimeoutSeconds || defaultIdleTimeout, - additionalTables: [], - }); + await getMirrorState(mirrorId).then((res) => { + setMirrorState(res); + + setConfig({ + batchSize: + (res as MirrorStatusResponse).cdcStatus?.config?.maxBatchSize || + defaultBatchSize, + idleTimeout: + (res as MirrorStatusResponse).cdcStatus?.config?.idleTimeoutSeconds || + defaultIdleTimeout, + additionalTables: [], }); + }); }, [mirrorId, defaultBatchSize, defaultIdleTimeout]); useEffect(() => { @@ -183,6 +177,10 @@ const EditMirror = ({ params: { mirrorId } }: EditMirrorProps) => { been completed.

The replication slot will grow during this period. +

+ For custom publications, ensure that the tables are part of the + publication you provided. This can be done with ALTER PUBLICATION + pubname ADD TABLE table1, table2; )} diff --git a/ui/app/mirrors/[mirrorId]/handlers.ts b/ui/app/mirrors/[mirrorId]/handlers.ts new file mode 100644 index 0000000000..bd6e0d3e2f --- /dev/null +++ b/ui/app/mirrors/[mirrorId]/handlers.ts @@ -0,0 +1,38 @@ +import { FlowConnectionConfigs, FlowStatus } from '@/grpc_generated/flow'; +import { + FlowStateChangeRequest, + MirrorStatusResponse, +} from '@/grpc_generated/route'; + +export const getMirrorState = async (mirrorId: string) => { + return await fetch('/api/mirrors/state', { + method: 'POST', + body: JSON.stringify({ + flowJobName: mirrorId, + }), + }).then((res) => res.json()); +}; + +export const getCurrentIdleTimeout = async (mirrorId: string) => { + return await getMirrorState(mirrorId).then((res) => { + return (res as MirrorStatusResponse).cdcStatus?.config?.idleTimeoutSeconds; + }); +}; + +export const changeFlowState = async ( + mirrorConfig: FlowConnectionConfigs, + flowState: FlowStatus +) => { + const req: FlowStateChangeRequest = { + flowJobName: mirrorConfig.flowJobName, + sourcePeer: mirrorConfig.source, + destinationPeer: mirrorConfig.destination, + requestedFlowState: flowState, + }; + await fetch(`/api/mirrors/state_change`, { + method: 'POST', + body: JSON.stringify(req), + cache: 'no-store', + }); + window.location.reload(); +}; diff --git a/ui/app/mirrors/[mirrorId]/snapshot.tsx b/ui/app/mirrors/[mirrorId]/snapshot.tsx index d0a3e69e1a..f60c3241d9 100644 --- a/ui/app/mirrors/[mirrorId]/snapshot.tsx +++ b/ui/app/mirrors/[mirrorId]/snapshot.tsx @@ -1,18 +1,7 @@ 'use client'; -import SelectTheme from '@/app/styles/select'; -import TimeLabel from '@/components/TimeComponent'; import { CloneTableSummary, SnapshotStatus } from '@/grpc_generated/route'; -import { Badge } from '@/lib/Badge/Badge'; -import { Button } from '@/lib/Button'; -import { Icon } from '@/lib/Icon'; -import { Label } from '@/lib/Label'; -import { ProgressBar } from '@/lib/ProgressBar'; -import { SearchField } from '@/lib/SearchField'; -import { Table, TableCell, TableRow } from '@/lib/Table'; import moment, { Duration, Moment } from 'moment'; -import Link from 'next/link'; -import { useMemo, useState } from 'react'; -import ReactSelect from 'react-select'; +import SnapshotTable from './snapshotTable'; export class TableCloneSummary { cloneStartTime: Moment | null = null; @@ -53,233 +42,34 @@ type SnapshotStatusProps = { status: SnapshotStatus; }; -const ROWS_PER_PAGE = 5; export const SnapshotStatusTable = ({ status }: SnapshotStatusProps) => { - const [sortField, setSortField] = useState< - 'cloneStartTime' | 'avgTimePerPartition' - >('cloneStartTime'); - const allRows = status.clones.map(summarizeTableClone); - const [currentPage, setCurrentPage] = useState(1); - const totalPages = Math.ceil(allRows.length / ROWS_PER_PAGE); - const [searchQuery, setSearchQuery] = useState(''); - const [sortDir, setSortDir] = useState<'asc' | 'dsc'>('dsc'); - const displayedRows = useMemo(() => { - const shownRows = allRows.filter((row: TableCloneSummary) => - row.cloneTableSummary.tableName - ?.toLowerCase() - .includes(searchQuery.toLowerCase()) - ); - shownRows.sort((a, b) => { - const aValue = a[sortField]; - const bValue = b[sortField]; - if (aValue === null || bValue === null) { - return 0; - } - - if (aValue < bValue) { - return sortDir === 'dsc' ? 1 : -1; - } else if (aValue > bValue) { - return sortDir === 'dsc' ? -1 : 1; - } else { - return 0; - } - }); - - const startRow = (currentPage - 1) * ROWS_PER_PAGE; - const endRow = startRow + ROWS_PER_PAGE; - return shownRows.length > ROWS_PER_PAGE - ? shownRows.slice(startRow, endRow) - : shownRows; - }, [allRows, currentPage, searchQuery, sortField, sortDir]); - - const handlePrevPage = () => { - if (currentPage > 1) { - setCurrentPage(currentPage - 1); - } - }; - - const handleNextPage = () => { - if (currentPage < totalPages) { - setCurrentPage(currentPage + 1); - } - }; - - const getStatus = (clone: CloneTableSummary) => { - if (clone.consolidateCompleted) { - return ( - - -
Done
-
- ); - } - if (!clone.fetchCompleted) { - return ( - - -
Fetching
-
- ); - } - - if (clone.numPartitionsCompleted == clone.numPartitionsTotal) { - return ( - - -
Consolidating
-
- ); - } - - return ( - - -
Syncing
-
- ); - }; + const allTableLoads = status.clones.map(summarizeTableClone); + const completedTableLoads = allTableLoads.filter( + (row) => row.cloneTableSummary.consolidateCompleted === true + ); + const inProgressTableLoads = allTableLoads.filter( + (row) => !row.cloneTableSummary.consolidateCompleted + ); - const sortOptions = [ - { value: 'cloneStartTime', label: 'Start Time' }, - { value: 'avgTimePerPartition', label: 'Time Per Partition' }, - ]; return (
- Initial Copy} - toolbar={{ - left: ( -
- - - - -
- { - const sortVal = - (val?.value as - | 'cloneStartTime' - | 'avgTimePerPartition') ?? 'cloneStartTime'; - setSortField(sortVal); - }} - value={{ - value: sortField, - label: sortOptions.find((opt) => opt.value === sortField) - ?.label, - }} - defaultValue={{ - value: 'cloneStartTime', - label: 'Start Time', - }} - theme={SelectTheme} - /> -
- - -
- ), - right: ( - ) => - setSearchQuery(e.target.value) - } - /> - ), - }} - header={ - - Table Identifier - Status - Sync Start - Progress Partitions - Num Rows Processed - Avg Time Per Partition - - } - > - {displayedRows.map((clone, index) => ( - - - - - {getStatus(clone.cloneTableSummary)} - - {clone.cloneStartTime ? ( - - ) : ( - 'N/A' - )} - - {clone.cloneTableSummary.fetchCompleted ? ( - - - {clone.cloneTableSummary.numPartitionsCompleted} /{' '} - {clone.cloneTableSummary.numPartitionsTotal} - - ) : ( - N/A - )} - - {clone.cloneTableSummary.fetchCompleted - ? clone.cloneTableSummary.numRowsSynced - : 0} - - {clone.cloneTableSummary.fetchCompleted ? ( - - - - ) : ( - N/A - )} - - ))} -
+ {[ + { data: inProgressTableLoads, title: 'In progress' }, + { data: completedTableLoads, title: 'Completed tables' }, + ].map((tableLoads, index) => ( + + ))}
); }; diff --git a/ui/app/mirrors/[mirrorId]/snapshotTable.tsx b/ui/app/mirrors/[mirrorId]/snapshotTable.tsx new file mode 100644 index 0000000000..65776d5dd6 --- /dev/null +++ b/ui/app/mirrors/[mirrorId]/snapshotTable.tsx @@ -0,0 +1,249 @@ +'use client'; +import SelectTheme from '@/app/styles/select'; +import TimeLabel from '@/components/TimeComponent'; +import { CloneTableSummary } from '@/grpc_generated/route'; +import { Badge } from '@/lib/Badge/Badge'; +import { Button } from '@/lib/Button'; +import { Icon } from '@/lib/Icon'; +import { Label } from '@/lib/Label'; +import { ProgressBar } from '@/lib/ProgressBar'; +import { SearchField } from '@/lib/SearchField'; +import { Table, TableCell, TableRow } from '@/lib/Table'; +import Link from 'next/link'; +import { useMemo, useState } from 'react'; +import ReactSelect from 'react-select'; +import { TableCloneSummary } from './snapshot'; + +const ROWS_PER_PAGE = 5; + +const getStatus = (clone: CloneTableSummary) => { + if (clone.consolidateCompleted) { + return ( + + +
Done
+
+ ); + } + if (!clone.fetchCompleted) { + return ( + + +
Fetching
+
+ ); + } + + if (clone.numPartitionsCompleted == clone.numPartitionsTotal) { + return ( + + +
Consolidating
+
+ ); + } + + return ( + + +
Syncing
+
+ ); +}; + +const SnapshotTable = ({ + tableLoads, + title, +}: { + tableLoads: TableCloneSummary[]; + title: string; +}) => { + const [sortField, setSortField] = useState< + 'cloneStartTime' | 'avgTimePerPartition' + >('cloneStartTime'); + + const [currentPage, setCurrentPage] = useState(1); + const totalPages = Math.ceil(tableLoads.length / ROWS_PER_PAGE); + const [searchQuery, setSearchQuery] = useState(''); + const [sortDir, setSortDir] = useState<'asc' | 'dsc'>('dsc'); + const displayedLoads = useMemo(() => { + const shownRows = tableLoads.filter((row: TableCloneSummary) => + row.cloneTableSummary.tableName + ?.toLowerCase() + .includes(searchQuery.toLowerCase()) + ); + shownRows.sort((a, b) => { + const aValue = a[sortField]; + const bValue = b[sortField]; + if (aValue === null || bValue === null) { + return 0; + } + + if (aValue < bValue) { + return sortDir === 'dsc' ? 1 : -1; + } else if (aValue > bValue) { + return sortDir === 'dsc' ? -1 : 1; + } else { + return 0; + } + }); + + const startRow = (currentPage - 1) * ROWS_PER_PAGE; + const endRow = startRow + ROWS_PER_PAGE; + return shownRows.length > ROWS_PER_PAGE + ? shownRows.slice(startRow, endRow) + : shownRows; + }, [tableLoads, currentPage, searchQuery, sortField, sortDir]); + + const handlePrevPage = () => { + if (currentPage > 1) { + setCurrentPage(currentPage - 1); + } + }; + + const handleNextPage = () => { + if (currentPage < totalPages) { + setCurrentPage(currentPage + 1); + } + }; + + const sortOptions = [ + { value: 'cloneStartTime', label: 'Start Time' }, + { value: 'avgTimePerPartition', label: 'Time Per Partition' }, + ]; + return ( + {title}} + toolbar={{ + left: ( +
+ + + + +
+ { + const sortVal = + (val?.value as 'cloneStartTime' | 'avgTimePerPartition') ?? + 'cloneStartTime'; + setSortField(sortVal); + }} + value={{ + value: sortField, + label: sortOptions.find((opt) => opt.value === sortField) + ?.label, + }} + defaultValue={{ + value: 'cloneStartTime', + label: 'Start Time', + }} + theme={SelectTheme} + /> +
+ + +
+ ), + right: ( + ) => + setSearchQuery(e.target.value) + } + /> + ), + }} + header={ + + {[ + 'Table Identifier', + 'Status', + 'Sync Start', + 'Progress Partitions', + 'Num Rows Processed', + 'Avg Time Per Partition', + ].map((header) => ( + + {header} + + ))} + + } + > + {displayedLoads.map((clone, index) => ( + + + + + {getStatus(clone.cloneTableSummary)} + + {clone.cloneStartTime ? ( + + ) : ( + 'N/A' + )} + + {clone.cloneTableSummary.fetchCompleted ? ( + + + {clone.cloneTableSummary.numPartitionsCompleted} /{' '} + {clone.cloneTableSummary.numPartitionsTotal} + + ) : ( + N/A + )} + + {clone.cloneTableSummary.fetchCompleted + ? clone.cloneTableSummary.numRowsSynced + : 0} + + {clone.cloneTableSummary.fetchCompleted ? ( + + + + ) : ( + N/A + )} + + ))} +
+ ); +}; + +export default SnapshotTable; diff --git a/ui/app/mirrors/create/cdc/cdc.tsx b/ui/app/mirrors/create/cdc/cdc.tsx index 589382aedb..dc34c1dfd9 100644 --- a/ui/app/mirrors/create/cdc/cdc.tsx +++ b/ui/app/mirrors/create/cdc/cdc.tsx @@ -67,7 +67,10 @@ export default function CDCConfigForm({ (label.includes('initial copy') || label.includes('initial load') || label.includes('soft delete') || - label.includes('snapshot'))) + label.includes('snapshot'))) || + ((mirrorConfig.source?.type !== DBType.POSTGRES || + mirrorConfig.destination?.type !== DBType.POSTGRES) && + label.includes('type system')) ) { return false; } @@ -97,7 +100,7 @@ export default function CDCConfigForm({ ? publications : undefined } - publicationsLoading={pubLoading} + optionsLoading={pubLoading} /> ) ); diff --git a/ui/app/mirrors/create/cdc/fields.tsx b/ui/app/mirrors/create/cdc/fields.tsx index 7d8842e89a..3a1f8064b2 100644 --- a/ui/app/mirrors/create/cdc/fields.tsx +++ b/ui/app/mirrors/create/cdc/fields.tsx @@ -13,14 +13,14 @@ interface FieldProps { setting: MirrorSetting; handleChange: (val: string | boolean, setting: MirrorSetting) => void; options?: string[]; - publicationsLoading?: boolean; + optionsLoading?: boolean; } const CDCField = ({ setting, handleChange, options, - publicationsLoading, + optionsLoading, }: FieldProps) => { return setting.type === 'switch' ? (
val && handleChange(val.option, setting) } @@ -78,7 +77,7 @@ const CDCField = ({ getOptionLabel={(option) => option.label} getOptionValue={(option) => option.option} theme={SelectTheme} - isLoading={publicationsLoading} + isLoading={optionsLoading} />
{setting.tips && ( diff --git a/ui/app/mirrors/create/cdc/tablemapping.tsx b/ui/app/mirrors/create/cdc/tablemapping.tsx index 7b0566cd30..d94358edc4 100644 --- a/ui/app/mirrors/create/cdc/tablemapping.tsx +++ b/ui/app/mirrors/create/cdc/tablemapping.tsx @@ -54,7 +54,7 @@ const TableMapping = ({ style={{ color: 'teal' }} target='_blank' href={ - 'https://docs.peerdb.io/connect/rds_postgres#creating-peerdb-user-and-granting-permissions' + 'https://docs.peerdb.io/connect/postgres/rds_postgres#creating-peerdb-user-and-granting-permissions' } > these permissions diff --git a/ui/app/mirrors/create/handlers.ts b/ui/app/mirrors/create/handlers.ts index 49fc31f0e4..2b705ea42c 100644 --- a/ui/app/mirrors/create/handlers.ts +++ b/ui/app/mirrors/create/handlers.ts @@ -7,6 +7,7 @@ import { UTablesResponse, } from '@/app/dto/PeersDTO'; import { notifyErr } from '@/app/utils/notify'; +import { DBTypeToGoodText } from '@/components/PeerTypeComponent'; import { FlowConnectionConfigs, QRepConfig, @@ -197,6 +198,7 @@ export const handleCreateQRep = async ( route: RouteCallback, xmin?: boolean ) => { + console.log(config); const flowNameValid = flowNameSchema.safeParse(flowJobName); if (!flowNameValid.success) { const flowNameErr = flowNameValid.error.issues[0].message; @@ -229,6 +231,31 @@ export const handleCreateQRep = async ( config.flowJobName = flowJobName; config.query = query; + const isSchemaLessPeer = + config.destinationPeer?.type === DBType.BIGQUERY || + config.destinationPeer?.type === DBType.CLICKHOUSE; + if (config.destinationPeer?.type !== DBType.ELASTICSEARCH) { + if (isSchemaLessPeer && config.destinationTableIdentifier?.includes('.')) { + notifyErr( + 'Destination table should not be schema qualified for ' + + DBTypeToGoodText(config.destinationPeer?.type) + + ' targets' + ); + return; + } + if ( + !isSchemaLessPeer && + !config.destinationTableIdentifier?.includes('.') + ) { + notifyErr( + 'Destination table should be schema qualified for ' + + DBTypeToGoodText(config.destinationPeer?.type) + + ' targets' + ); + return; + } + } + setLoading(true); const statusMessage: UCreateMirrorResponse = await fetch( '/api/mirrors/qrep', diff --git a/ui/app/mirrors/create/helpers/cdc.ts b/ui/app/mirrors/create/helpers/cdc.ts index 5d63cdd69a..c18f483f30 100644 --- a/ui/app/mirrors/create/helpers/cdc.ts +++ b/ui/app/mirrors/create/helpers/cdc.ts @@ -1,3 +1,4 @@ +import { TypeSystem } from '@/grpc_generated/flow'; import { CDCConfig } from '../../../dto/MirrorsDTO'; import { MirrorSetting } from './common'; export const cdcSettings: MirrorSetting[] = [ @@ -149,4 +150,16 @@ export const cdcSettings: MirrorSetting[] = [ tips: 'Associate PeerDB script with this mirror.', advanced: true, }, + { + label: 'Use Postgres type system', + stateHandler: (value, setter) => + setter((curr: CDCConfig) => ({ + ...curr, + system: value === true ? TypeSystem.PG : TypeSystem.Q, + })), + type: 'switch', + default: false, + tips: 'Decide if PeerDB should use native Postgres types directly', + advanced: true, + }, ]; diff --git a/ui/app/mirrors/create/helpers/common.ts b/ui/app/mirrors/create/helpers/common.ts index 63dea81815..4237bc2176 100644 --- a/ui/app/mirrors/create/helpers/common.ts +++ b/ui/app/mirrors/create/helpers/common.ts @@ -1,12 +1,8 @@ -import { FlowConnectionConfigs, QRepWriteType } from '@/grpc_generated/flow'; -import { Peer } from '@/grpc_generated/peers'; +import { FlowConnectionConfigs, TypeSystem } from '@/grpc_generated/flow'; export interface MirrorSetting { label: string; - stateHandler: ( - value: string | string[] | Peer | boolean | QRepWriteType, - setter: any - ) => void; + stateHandler: (value: any, setter: any) => void; type?: string; required?: boolean; tips?: string; @@ -37,6 +33,7 @@ export const blankCDCSetting: FlowConnectionConfigs = { initialSnapshotOnly: false, idleTimeoutSeconds: 60, script: '', + system: TypeSystem.Q, }; export const blankQRepSetting = { diff --git a/ui/app/mirrors/create/schema.ts b/ui/app/mirrors/create/schema.ts index 7b615e7c34..fc4ea12eaa 100644 --- a/ui/app/mirrors/create/schema.ts +++ b/ui/app/mirrors/create/schema.ts @@ -6,7 +6,7 @@ export const flowNameSchema = z required_error: 'Mirror name is required.', }) .min(1, { message: 'Mirror name cannot be empty.' }) - .regex(/^[a-z0-9_]*$/, { + .regex(/^[a-z_][a-z0-9_]*$/, { message: 'Mirror name must contain only lowercase letters, numbers and underscores', }); diff --git a/ui/app/peers/[peerName]/lagGraph.tsx b/ui/app/peers/[peerName]/lagGraph.tsx index 12b0c2b49b..05da92832a 100644 --- a/ui/app/peers/[peerName]/lagGraph.tsx +++ b/ui/app/peers/[peerName]/lagGraph.tsx @@ -98,7 +98,7 @@ function LagGraph({ slotNames }: { slotNames: string[] }) { !x.value.endsWith('min'))} defaultValue={{ label: 'hour', value: 'hour' }} onChange={(val, _) => val && setTimeSince(val.value)} theme={SelectTheme} diff --git a/ui/app/peers/create/[peerType]/handlers.ts b/ui/app/peers/create/[peerType]/handlers.ts index 7b36b7acb6..4778182445 100644 --- a/ui/app/peers/create/[peerType]/handlers.ts +++ b/ui/app/peers/create/[peerType]/handlers.ts @@ -9,6 +9,7 @@ import { bqSchema, chSchema, ehGroupSchema, + esSchema, kaSchema, peerNameSchema, pgSchema, @@ -73,6 +74,13 @@ const validateFields = ( if (!ehGroupConfig.success) validationErr = ehGroupConfig.error.issues[0].message; break; + case 'ELASTICSEARCH': + const esConfig = esSchema.safeParse(config); + if (!esConfig.success) { + console.log(esConfig.error); + validationErr = esConfig.error.issues[0].message; + } + break; default: validationErr = 'Unsupported peer type ' + type; } @@ -94,6 +102,7 @@ export const handleValidate = async ( const isValid = validateFields(type, config, notify, name); if (!isValid) return; setLoading(true); + const valid: UValidatePeerResponse = await fetch('/api/peers/', { method: 'POST', body: JSON.stringify({ diff --git a/ui/app/peers/create/[peerType]/helpers/common.ts b/ui/app/peers/create/[peerType]/helpers/common.ts index f0b03c22f5..4aa1423417 100644 --- a/ui/app/peers/create/[peerType]/helpers/common.ts +++ b/ui/app/peers/create/[peerType]/helpers/common.ts @@ -2,6 +2,7 @@ import { PeerConfig, PeerSetter } from '@/app/dto/PeersDTO'; import { blankBigquerySetting } from './bq'; import { blankClickhouseSetting } from './ch'; import { blankEventHubGroupSetting } from './eh'; +import { blankElasticsearchSetting } from './es'; import { blankKafkaSetting } from './ka'; import { blankPostgresSetting } from './pg'; import { blankPubSubSetting } from './ps'; @@ -38,6 +39,8 @@ export const getBlankSetting = (dbType: string): PeerConfig => { return blankS3Setting; case 'EVENTHUBS': return blankEventHubGroupSetting; + case 'ELASTICSEARCH': + return blankElasticsearchSetting; default: return blankPostgresSetting; } diff --git a/ui/app/peers/create/[peerType]/helpers/es.ts b/ui/app/peers/create/[peerType]/helpers/es.ts new file mode 100644 index 0000000000..4da8b58f74 --- /dev/null +++ b/ui/app/peers/create/[peerType]/helpers/es.ts @@ -0,0 +1,58 @@ +import { + ElasticsearchAuthType, + elasticsearchAuthTypeFromJSON, + ElasticsearchConfig, +} from '@/grpc_generated/peers'; +import { PeerSetting } from './common'; + +export const esSetting: PeerSetting[] = [ + { + label: 'Addresses', + stateHandler: (value, setter) => + setter((curr) => ({ ...curr, addresses: (value as string).split(',') })), + }, + { + label: 'Authentication type', + stateHandler: (value, setter) => + setter((curr) => { + let esConfig = curr as ElasticsearchConfig; + return { + ...esConfig, + authType: elasticsearchAuthTypeFromJSON(value), + }; + }), + type: 'select', + placeholder: 'Select a mechanism', + options: [ + { value: 'NONE', label: 'None' }, + { value: 'BASIC', label: 'Basic' }, + { value: 'APIKEY', label: 'API Key' }, + ], + }, + // remaining fields are optional but displayed conditionally so not optional style wise + { + label: 'Username', + stateHandler: (value, setter) => + setter((curr) => ({ ...curr, username: value as string })), + }, + { + label: 'Password', + type: 'password', + stateHandler: (value, setter) => + setter((curr) => ({ ...curr, password: value as string })), + }, + { + label: 'API Key', + type: 'password', + stateHandler: (value, setter) => + setter((curr) => ({ ...curr, apiKey: value as string })), + }, +]; + +export const blankElasticsearchSetting: ElasticsearchConfig = { + addresses: [], + authType: ElasticsearchAuthType.UNKNOWN, + username: '', + password: '', + apiKey: '', +}; diff --git a/ui/app/peers/create/[peerType]/page.tsx b/ui/app/peers/create/[peerType]/page.tsx index 8325571fab..0e46f5f4b7 100644 --- a/ui/app/peers/create/[peerType]/page.tsx +++ b/ui/app/peers/create/[peerType]/page.tsx @@ -11,8 +11,12 @@ import SnowflakeForm from '@/components/PeerForms/SnowflakeForm'; import { notifyErr } from '@/app/utils/notify'; import TitleCase from '@/app/utils/titlecase'; +import ElasticsearchConfigForm from '@/components/PeerForms/ElasticsearchConfigForm'; import EventhubsForm from '@/components/PeerForms/Eventhubs/EventhubGroupConfig'; -import { EventHubGroupConfig } from '@/grpc_generated/peers'; +import { + ElasticsearchConfig, + EventHubGroupConfig, +} from '@/grpc_generated/peers'; import { Button } from '@/lib/Button'; import { ButtonGroup } from '@/lib/ButtonGroup'; import { Label } from '@/lib/Label'; @@ -82,6 +86,13 @@ export default function CreateConfig({ setter={setConfig} /> ); + case 'ELASTICSEARCH': + return ( + + ); default: return <>; } diff --git a/ui/app/peers/create/[peerType]/schema.ts b/ui/app/peers/create/[peerType]/schema.ts index 8868272f7e..87381862fd 100644 --- a/ui/app/peers/create/[peerType]/schema.ts +++ b/ui/app/peers/create/[peerType]/schema.ts @@ -1,4 +1,5 @@ import { ehSchema } from '@/components/PeerForms/Eventhubs/schema'; +import { ElasticsearchAuthType } from '@/grpc_generated/peers'; import * as z from 'zod'; export const peerNameSchema = z @@ -7,7 +8,7 @@ export const peerNameSchema = z required_error: 'Peer name is required.', }) .min(1, { message: 'Peer name cannot be empty.' }) - .regex(/^[a-z0-9_]*$/, { + .regex(/^[a-z_][a-z0-9_]*$/, { message: 'Peer name must contain only lowercase letters, numbers and underscores', }); @@ -429,3 +430,62 @@ export const ehGroupSchema = z.object({ message: 'At least 1 Event Hub is required', }), }); + +// slightly cursed, check for non-empty and non-whitespace string +const isString = (i: string | undefined): boolean => { + return !!i && !!i.trim(); +}; + +export const esSchema = z + .object({ + addresses: z.array( + z.string().url({ + message: 'Addresses must be a comma-seperated list of URLs', + }) + ), + authType: z.nativeEnum(ElasticsearchAuthType, { + required_error: 'Auth type cannot be empty', + invalid_type_error: 'Auth type must be one of [none,basic,apikey]', + }), + username: z + .string({ + invalid_type_error: 'Username must be a string', + }) + .optional(), + password: z + .string({ + invalid_type_error: 'Password must be a string', + }) + .optional(), + apiKey: z + .string({ + invalid_type_error: 'API key must be a string', + }) + .optional(), + }) + .refine( + (esSchema) => { + if (esSchema.authType === ElasticsearchAuthType.BASIC) { + return ( + isString(esSchema.username) && + isString(esSchema.password) && + !isString(esSchema.apiKey) + ); + } else if (esSchema.authType === ElasticsearchAuthType.APIKEY) { + return ( + !isString(esSchema.username) && + !isString(esSchema.password) && + isString(esSchema.apiKey) + ); + } else if (esSchema.authType === ElasticsearchAuthType.NONE) { + return ( + !isString(esSchema.username) && + !isString(esSchema.password) && + !isString(esSchema.apiKey) + ); + } + }, + { + message: 'Authentication info not valid', + } + ); diff --git a/ui/app/peers/create/page.tsx b/ui/app/peers/create/page.tsx index 0fd16869ae..f5c0578193 100644 --- a/ui/app/peers/create/page.tsx +++ b/ui/app/peers/create/page.tsx @@ -1,16 +1,10 @@ 'use client'; import SelectSource from '@/components/SelectSource'; import { Action } from '@/lib/Action'; -import { Button } from '@/lib/Button'; -import { ButtonGroup } from '@/lib/ButtonGroup'; import { Icon } from '@/lib/Icon'; import { Label } from '@/lib/Label'; -import { RowWithSelect } from '@/lib/Layout'; -import Link from 'next/link'; -import { useState } from 'react'; export default function CreatePeer() { - const [peerType, setPeerType] = useState(''); return (
- - Data source - - } - action={ - - } - /> - - - - - - - +
); diff --git a/ui/app/utils/titlecase.ts b/ui/app/utils/titlecase.ts index 5e87b2c169..dfc70e99d0 100644 --- a/ui/app/utils/titlecase.ts +++ b/ui/app/utils/titlecase.ts @@ -1,4 +1,5 @@ function TitleCase(input: string): string { + if (input == 'BIGQUERY') return 'BigQuery'; return input .toLowerCase() .replace(/\b\w/g, function (char) { diff --git a/ui/components/EditButton.tsx b/ui/components/EditButton.tsx index 5e87cbfaa8..965b5a1855 100644 --- a/ui/components/EditButton.tsx +++ b/ui/components/EditButton.tsx @@ -2,6 +2,7 @@ import { Button } from '@/lib/Button'; import { Label } from '@/lib/Label'; import { ProgressCircle } from '@/lib/ProgressCircle'; +import { Tooltip } from '@/lib/Tooltip'; import { useRouter } from 'next/navigation'; import { useState } from 'react'; @@ -20,22 +21,32 @@ const EditButton = ({ router.push(toLink); }; return ( - + + ); }; diff --git a/ui/components/MirrorActionsDropdown.tsx b/ui/components/MirrorActionsDropdown.tsx index c11e68e019..dd1eafe787 100644 --- a/ui/components/MirrorActionsDropdown.tsx +++ b/ui/components/MirrorActionsDropdown.tsx @@ -1,12 +1,12 @@ 'use client'; +import { getMirrorState } from '@/app/mirrors/[mirrorId]/handlers'; import EditButton from '@/components/EditButton'; import { ResyncDialog } from '@/components/ResyncDialog'; -import { FlowConnectionConfigs } from '@/grpc_generated/flow'; -import { Button } from '@/lib/Button/Button'; -import { Icon } from '@/lib/Icon'; -import { Label } from '@/lib/Label/Label'; -import * as DropdownMenu from '@radix-ui/react-dropdown-menu'; +import { FlowConnectionConfigs, FlowStatus } from '@/grpc_generated/flow'; +import { MirrorStatusResponse } from '@/grpc_generated/route'; +import { Select, SelectItem } from '@tremor/react'; import { useEffect, useState } from 'react'; +import PauseOrResumeButton from './PauseOrResumeButton'; const MirrorActions = ({ mirrorConfig, @@ -21,50 +21,48 @@ const MirrorActions = ({ canResync: boolean; isNotPaused: boolean; }) => { + const [mirrorStatus, setMirrorStatus] = useState(); const [mounted, setMounted] = useState(false); - const [open, setOpen] = useState(false); - const handleToggle = () => { - setOpen((prevOpen) => !prevOpen); - }; - useEffect(() => setMounted(true), []); + + useEffect(() => { + getMirrorState(mirrorConfig.flowJobName).then( + (res: MirrorStatusResponse) => { + setMirrorStatus(res.currentFlowState); + } + ); + setMounted(true); + }, [mirrorConfig.flowJobName]); + if (mounted) return ( - - - - - - - +
+ +
); return <>; }; diff --git a/ui/components/PauseOrResumeButton.tsx b/ui/components/PauseOrResumeButton.tsx new file mode 100644 index 0000000000..cdf3b600ef --- /dev/null +++ b/ui/components/PauseOrResumeButton.tsx @@ -0,0 +1,48 @@ +'use client'; +import { changeFlowState } from '@/app/mirrors/[mirrorId]/handlers'; +import { FlowConnectionConfigs, FlowStatus } from '@/grpc_generated/flow'; +import { Button } from '@/lib/Button'; +import { Label } from '@/lib/Label/Label'; + +function PauseOrResumeButton({ + mirrorConfig, + mirrorStatus, +}: { + mirrorConfig: FlowConnectionConfigs; + mirrorStatus: FlowStatus; +}) { + if (mirrorStatus.toString() === FlowStatus[FlowStatus.STATUS_RUNNING]) { + return ( + + ); + } else if (mirrorStatus.toString() === FlowStatus[FlowStatus.STATUS_PAUSED]) { + return ( + + ); + } else { + return ( + + ); + } +} + +export default PauseOrResumeButton; diff --git a/ui/components/PeerComponent.tsx b/ui/components/PeerComponent.tsx index 55659080b8..6527a20247 100644 --- a/ui/components/PeerComponent.tsx +++ b/ui/components/PeerComponent.tsx @@ -43,6 +43,9 @@ export const DBTypeToImageMapping = (peerType: DBType | string) => { return '/svgs/pubsub.svg'; case 'EVENTHUBS': return '/svgs/ms.svg'; + case DBType.ELASTICSEARCH: + case 'ELASTICSEARCH': + return '/svgs/elasticsearch.svg'; default: return '/svgs/pg.svg'; } diff --git a/ui/components/PeerForms/ElasticsearchConfigForm.tsx b/ui/components/PeerForms/ElasticsearchConfigForm.tsx new file mode 100644 index 0000000000..e8bc3c6f3d --- /dev/null +++ b/ui/components/PeerForms/ElasticsearchConfigForm.tsx @@ -0,0 +1,96 @@ +import { PeerSetter } from '@/app/dto/PeersDTO'; +import { esSetting } from '@/app/peers/create/[peerType]/helpers/es'; +import SelectTheme from '@/app/styles/select'; +import { + ElasticsearchAuthType, + ElasticsearchConfig, +} from '@/grpc_generated/peers'; +import { Label } from '@/lib/Label'; +import { RowWithSelect, RowWithTextField } from '@/lib/Layout'; +import { TextField } from '@/lib/TextField'; +import { Tooltip } from '@/lib/Tooltip'; +import ReactSelect from 'react-select'; +import { InfoPopover } from '../InfoPopover'; + +interface ElasticsearchProps { + config: ElasticsearchConfig; + setter: PeerSetter; +} + +const ElasticsearchConfigForm = ({ config, setter }: ElasticsearchProps) => { + return ( +
+ {esSetting.map((setting, index) => { + return setting.type === 'select' ? ( + {setting.label}} + action={ + + val && setting.stateHandler(val.value, setter) + } + options={setting.options} + theme={SelectTheme} + /> + } + /> + ) : (setting.label === 'API Key' && + config.authType === ElasticsearchAuthType.APIKEY) || + (setting.label !== 'API Key' && + config.authType === ElasticsearchAuthType.BASIC) || + setting.label === 'Addresses' ? ( + + {setting.label}{' '} + {!setting.optional && ( + + + + )} + + } + action={ +
+ ) => + setting.stateHandler(e.target.value, setter) + } + /> + {setting.tips && ( + + )} +
+ } + /> + ) : ( + <> + ); + })} +
+ ); +}; + +export default ElasticsearchConfigForm; diff --git a/ui/components/PeerTypeComponent.tsx b/ui/components/PeerTypeComponent.tsx index b96abdbc6c..fbcc565e38 100644 --- a/ui/components/PeerTypeComponent.tsx +++ b/ui/components/PeerTypeComponent.tsx @@ -4,7 +4,7 @@ import { Label } from '@/lib/Label'; import Image from 'next/image'; import { DBTypeToImageMapping } from './PeerComponent'; -export const DBTypeToGoodText = (ptype: DBType) => { +export const DBTypeToGoodText = (ptype?: DBType) => { switch (ptype) { case DBType.POSTGRES: return 'PostgreSQL'; @@ -26,6 +26,8 @@ export const DBTypeToGoodText = (ptype: DBType) => { return 'Kafka'; case DBType.PUBSUB: return 'PubSub'; + case DBType.ELASTICSEARCH: + return 'Elasticsearch'; default: return 'Unrecognised'; } diff --git a/ui/components/ResyncDialog.tsx b/ui/components/ResyncDialog.tsx index 6caeea3821..14d6c13cfb 100644 --- a/ui/components/ResyncDialog.tsx +++ b/ui/components/ResyncDialog.tsx @@ -62,8 +62,8 @@ export const ResyncDialog = ({ noInteract={true} size='xLarge' triggerButton={ - } > diff --git a/ui/components/SelectSource.tsx b/ui/components/SelectSource.tsx index 668fed3c10..cdb788d858 100644 --- a/ui/components/SelectSource.tsx +++ b/ui/components/SelectSource.tsx @@ -1,64 +1,81 @@ 'use client'; -import SelectTheme from '@/app/styles/select'; import TitleCase from '@/app/utils/titlecase'; -import { DBType } from '@/grpc_generated/peers'; +import { Button } from '@/lib/Button/Button'; import Image from 'next/image'; -import { Dispatch, SetStateAction } from 'react'; -import ReactSelect from 'react-select'; +import Link from 'next/link'; import { DBTypeToImageMapping } from './PeerComponent'; -interface SelectSourceProps { - peerType: string; - setPeerType: Dispatch>; -} - -function SourceLabel({ value, label }: { value: string; label: string }) { +function SourceLabel({ label }: { label: string }) { const peerLogo = DBTypeToImageMapping(label); return ( -
- peer -
{TitleCase(label)}
-
+ ); } -export default function SelectSource({ - peerType, - setPeerType, -}: SelectSourceProps) { - const dbTypes = Object.values(DBType) - .filter( - (value): value is string => - typeof value === 'string' && - (value === 'POSTGRESQL' || - value === 'SNOWFLAKE' || - value === 'BIGQUERY' || - value === 'S3' || - value === 'CLICKHOUSE' || - value === 'KAFKA' || - value === 'EVENTHUBS' || - value === 'PUBSUB') - ) - .map((value) => ({ label: value, value })); +const dbTypes = [ + [ + 'Postgres', + 'POSTGRESQL', + 'RDS POSTGRESQL', + 'GOOGLE CLOUD POSTGRESQL', + 'AZURE FLEXIBLE POSTGRESQL', + 'TEMBO', + 'CRUNCHY POSTGRES', + ], + ['Warehouses', 'SNOWFLAKE', 'BIGQUERY', 'S3', 'CLICKHOUSE', 'ELASTICSEARCH'], + ['Queues', 'KAFKA', 'EVENTHUBS', 'PUBSUB'], +]; - dbTypes.push( - { value: 'POSTGRESQL', label: 'POSTGRESQL' }, - { value: 'POSTGRESQL', label: 'RDS POSTGRESQL' }, - { value: 'POSTGRESQL', label: 'GOOGLE CLOUD POSTGRESQL' }, - { value: 'POSTGRESQL', label: 'AZURE FLEXIBLE POSTGRESQL' }, - { value: 'POSTGRESQL', label: 'TEMBO' }, - { value: 'POSTGRESQL', label: 'CRUNCHY POSTGRES' } - ); - return ( - opt.label === peerType)} - onChange={(val, _) => val && setPeerType(val.label)} - formatOptionLabel={SourceLabel} - theme={SelectTheme} - getOptionValue={(option) => option.label} - /> - ); +const gridContainerStyle = { + display: 'flex', + gap: '20px', + flexWrap: 'wrap', + border: 'solid #18794e', + borderRadius: '20px', + position: 'relative', + padding: '20px', + marginTop: '20px', +} as const; +const gridHeaderStyle = { + position: 'absolute', + top: '-15px', + height: '30px', + display: 'flex', + alignItems: 'center', + color: '#fff', + backgroundColor: '#18794e', + borderRadius: '15px', + marginLeft: '10px', + paddingLeft: '10px', + paddingRight: '10px', +} as const; + +export default function SelectSource() { + return dbTypes.map(([category, ...items]) => ( +
+
{category}
+ {items.map((item) => ( + + ))} +
+ )); } diff --git a/ui/package-lock.json b/ui/package-lock.json index bef033b6b4..67f78371b9 100644 --- a/ui/package-lock.json +++ b/ui/package-lock.json @@ -26,17 +26,17 @@ "@radix-ui/react-toggle": "^1.0.3", "@radix-ui/react-toggle-group": "^1.0.4", "@radix-ui/react-tooltip": "^1.0.7", - "@tremor/react": "^3.15.0", - "@types/node": "^20.12.5", - "@types/react": "^18.2.74", - "@types/react-dom": "^18.2.24", + "@tremor/react": "^3.16.1", + "@types/node": "^20.12.7", + "@types/react": "^18.2.79", + "@types/react-dom": "^18.2.25", "classnames": "^2.5.1", "long": "^5.2.3", - "lucide-react": "^0.365.0", - "material-symbols": "^0.17.1", + "lucide-react": "^0.372.0", + "material-symbols": "^0.17.2", "moment": "^2.30.1", "moment-timezone": "^0.5.45", - "next": "^14.1.4", + "next": "^14.2.2", "next-auth": "^4.24.7", "prop-types": "^15.8.1", "protobufjs": "^7.2.6", @@ -48,13 +48,13 @@ "styled-components": "^6.1.8", "swr": "^2.2.5", "usehooks-ts": "^3.1.0", - "zod": "^3.22.4" + "zod": "^3.23.0" }, "devDependencies": { "autoprefixer": "^10.4.19", "copy-webpack-plugin": "^12.0.2", "eslint": "^8.57.0", - "eslint-config-next": "^14.1.4", + "eslint-config-next": "^14.2.2", "eslint-config-prettier": "^9.1.0", "gh-pages": "^6.1.1", "less": "^4.2.0", @@ -65,7 +65,7 @@ "string-width": "^7.1.0", "tailwindcss": "^3.4.3", "tailwindcss-animate": "^1.0.7", - "typescript": "^5.4.4", + "typescript": "^5.4.5", "webpack": "^5.91.0" } }, @@ -495,9 +495,9 @@ } }, "node_modules/@headlessui/react": { - "version": "1.7.18", - "resolved": "https://registry.npmjs.org/@headlessui/react/-/react-1.7.18.tgz", - "integrity": "sha512-4i5DOrzwN4qSgNsL4Si61VMkUcWbcSKueUV7sFhpHzQcSShdlHENE5+QBntMSRvHt8NyoFO2AGG8si9lq+w4zQ==", + "version": "1.7.19", + "resolved": "https://registry.npmjs.org/@headlessui/react/-/react-1.7.19.tgz", + "integrity": "sha512-Ll+8q3OlMJfJbAKM/+/Y2q6PPYbryqNTXDbryx7SXLIDamkF6iQFbriYHga0dY44PvDhvvBWCx1Xj4U5+G4hOw==", "dependencies": { "@tanstack/react-virtual": "^3.0.0-beta.60", "client-only": "^0.0.1" @@ -698,23 +698,23 @@ } }, "node_modules/@next/env": { - "version": "14.1.4", - "resolved": "https://registry.npmjs.org/@next/env/-/env-14.1.4.tgz", - "integrity": "sha512-e7X7bbn3Z6DWnDi75UWn+REgAbLEqxI8Tq2pkFOFAMpWAWApz/YCUhtWMWn410h8Q2fYiYL7Yg5OlxMOCfFjJQ==" + "version": "14.2.2", + "resolved": "https://registry.npmjs.org/@next/env/-/env-14.2.2.tgz", + "integrity": "sha512-sk72qRfM1Q90XZWYRoJKu/UWlTgihrASiYw/scb15u+tyzcze3bOuJ/UV6TBOQEeUaxOkRqGeuGUdiiuxc5oqw==" }, "node_modules/@next/eslint-plugin-next": { - "version": "14.1.4", - "resolved": "https://registry.npmjs.org/@next/eslint-plugin-next/-/eslint-plugin-next-14.1.4.tgz", - "integrity": "sha512-n4zYNLSyCo0Ln5b7qxqQeQ34OZKXwgbdcx6kmkQbywr+0k6M3Vinft0T72R6CDAcDrne2IAgSud4uWCzFgc5HA==", + "version": "14.2.2", + "resolved": "https://registry.npmjs.org/@next/eslint-plugin-next/-/eslint-plugin-next-14.2.2.tgz", + "integrity": "sha512-q+Ec2648JtBpKiu/FSJm8HAsFXlNvioHeBCbTP12T1SGcHYwhqHULSfQgFkPgHDu3kzNp2Kem4J54bK4rPQ5SQ==", "dev": true, "dependencies": { "glob": "10.3.10" } }, "node_modules/@next/swc-darwin-arm64": { - "version": "14.1.4", - "resolved": "https://registry.npmjs.org/@next/swc-darwin-arm64/-/swc-darwin-arm64-14.1.4.tgz", - "integrity": "sha512-ubmUkbmW65nIAOmoxT1IROZdmmJMmdYvXIe8211send9ZYJu+SqxSnJM4TrPj9wmL6g9Atvj0S/2cFmMSS99jg==", + "version": "14.2.2", + "resolved": "https://registry.npmjs.org/@next/swc-darwin-arm64/-/swc-darwin-arm64-14.2.2.tgz", + "integrity": "sha512-3iPgMhzbalizGwHNFUcGnDhFPSgVBHQ8aqSTAMxB5BvJG0oYrDf1WOJZlbXBgunOEj/8KMVbejEur/FpvFsgFQ==", "cpu": [ "arm64" ], @@ -727,9 +727,9 @@ } }, "node_modules/@next/swc-darwin-x64": { - "version": "14.1.4", - "resolved": "https://registry.npmjs.org/@next/swc-darwin-x64/-/swc-darwin-x64-14.1.4.tgz", - "integrity": "sha512-b0Xo1ELj3u7IkZWAKcJPJEhBop117U78l70nfoQGo4xUSvv0PJSTaV4U9xQBLvZlnjsYkc8RwQN1HoH/oQmLlQ==", + "version": "14.2.2", + "resolved": "https://registry.npmjs.org/@next/swc-darwin-x64/-/swc-darwin-x64-14.2.2.tgz", + "integrity": "sha512-x7Afi/jt0ZBRUZHTi49yyej4o8znfIMHO4RvThuoc0P+uli8Jd99y5GKjxoYunPKsXL09xBXEM1+OQy2xEL0Ag==", "cpu": [ "x64" ], @@ -742,9 +742,9 @@ } }, "node_modules/@next/swc-linux-arm64-gnu": { - "version": "14.1.4", - "resolved": "https://registry.npmjs.org/@next/swc-linux-arm64-gnu/-/swc-linux-arm64-gnu-14.1.4.tgz", - "integrity": "sha512-457G0hcLrdYA/u1O2XkRMsDKId5VKe3uKPvrKVOyuARa6nXrdhJOOYU9hkKKyQTMru1B8qEP78IAhf/1XnVqKA==", + "version": "14.2.2", + "resolved": "https://registry.npmjs.org/@next/swc-linux-arm64-gnu/-/swc-linux-arm64-gnu-14.2.2.tgz", + "integrity": "sha512-zbfPtkk7L41ODMJwSp5VbmPozPmMMQrzAc0HAUomVeVIIwlDGs/UCqLJvLNDt4jpWgc21SjjyIn762lNGrMaUA==", "cpu": [ "arm64" ], @@ -757,9 +757,9 @@ } }, "node_modules/@next/swc-linux-arm64-musl": { - "version": "14.1.4", - "resolved": "https://registry.npmjs.org/@next/swc-linux-arm64-musl/-/swc-linux-arm64-musl-14.1.4.tgz", - "integrity": "sha512-l/kMG+z6MB+fKA9KdtyprkTQ1ihlJcBh66cf0HvqGP+rXBbOXX0dpJatjZbHeunvEHoBBS69GYQG5ry78JMy3g==", + "version": "14.2.2", + "resolved": "https://registry.npmjs.org/@next/swc-linux-arm64-musl/-/swc-linux-arm64-musl-14.2.2.tgz", + "integrity": "sha512-wPbS3pI/JU16rm3XdLvvTmlsmm1nd+sBa2ohXgBZcShX4TgOjD4R+RqHKlI1cjo/jDZKXt6OxmcU0Iys0OC/yg==", "cpu": [ "arm64" ], @@ -772,9 +772,9 @@ } }, "node_modules/@next/swc-linux-x64-gnu": { - "version": "14.1.4", - "resolved": "https://registry.npmjs.org/@next/swc-linux-x64-gnu/-/swc-linux-x64-gnu-14.1.4.tgz", - "integrity": "sha512-BapIFZ3ZRnvQ1uWbmqEGJuPT9cgLwvKtxhK/L2t4QYO7l+/DxXuIGjvp1x8rvfa/x1FFSsipERZK70pewbtJtw==", + "version": "14.2.2", + "resolved": "https://registry.npmjs.org/@next/swc-linux-x64-gnu/-/swc-linux-x64-gnu-14.2.2.tgz", + "integrity": "sha512-NqWOHqqq8iC9tuHvZxjQ2tX+jWy2X9y8NX2mcB4sj2bIccuCxbIZrU/ThFPZZPauygajZuVQ6zediejQHwZHwQ==", "cpu": [ "x64" ], @@ -787,9 +787,9 @@ } }, "node_modules/@next/swc-linux-x64-musl": { - "version": "14.1.4", - "resolved": "https://registry.npmjs.org/@next/swc-linux-x64-musl/-/swc-linux-x64-musl-14.1.4.tgz", - "integrity": "sha512-mqVxTwk4XuBl49qn2A5UmzFImoL1iLm0KQQwtdRJRKl21ylQwwGCxJtIYo2rbfkZHoSKlh/YgztY0qH3wG1xIg==", + "version": "14.2.2", + "resolved": "https://registry.npmjs.org/@next/swc-linux-x64-musl/-/swc-linux-x64-musl-14.2.2.tgz", + "integrity": "sha512-lGepHhwb9sGhCcU7999+iK1ZZT+6rrIoVg40MP7DZski9GIZP80wORSbt5kJzh9v2x2ev2lxC6VgwMQT0PcgTA==", "cpu": [ "x64" ], @@ -802,9 +802,9 @@ } }, "node_modules/@next/swc-win32-arm64-msvc": { - "version": "14.1.4", - "resolved": "https://registry.npmjs.org/@next/swc-win32-arm64-msvc/-/swc-win32-arm64-msvc-14.1.4.tgz", - "integrity": "sha512-xzxF4ErcumXjO2Pvg/wVGrtr9QQJLk3IyQX1ddAC/fi6/5jZCZ9xpuL9Tzc4KPWMFq8GGWFVDMshZOdHGdkvag==", + "version": "14.2.2", + "resolved": "https://registry.npmjs.org/@next/swc-win32-arm64-msvc/-/swc-win32-arm64-msvc-14.2.2.tgz", + "integrity": "sha512-TZSh/48SfcLEQ4rD25VVn2kdIgUWmMflRX3OiyPwGNXn3NiyPqhqei/BaqCYXViIQ+6QsG9R0C8LftMqy8JPMA==", "cpu": [ "arm64" ], @@ -817,9 +817,9 @@ } }, "node_modules/@next/swc-win32-ia32-msvc": { - "version": "14.1.4", - "resolved": "https://registry.npmjs.org/@next/swc-win32-ia32-msvc/-/swc-win32-ia32-msvc-14.1.4.tgz", - "integrity": "sha512-WZiz8OdbkpRw6/IU/lredZWKKZopUMhcI2F+XiMAcPja0uZYdMTZQRoQ0WZcvinn9xZAidimE7tN9W5v9Yyfyw==", + "version": "14.2.2", + "resolved": "https://registry.npmjs.org/@next/swc-win32-ia32-msvc/-/swc-win32-ia32-msvc-14.2.2.tgz", + "integrity": "sha512-M0tBVNMEBJN2ZNQWlcekMn6pvLria7Sa2Fai5znm7CCJz4pP3lrvlSxhKdkCerk0D9E0bqx5yAo3o2Q7RrD4gA==", "cpu": [ "ia32" ], @@ -832,9 +832,9 @@ } }, "node_modules/@next/swc-win32-x64-msvc": { - "version": "14.1.4", - "resolved": "https://registry.npmjs.org/@next/swc-win32-x64-msvc/-/swc-win32-x64-msvc-14.1.4.tgz", - "integrity": "sha512-4Rto21sPfw555sZ/XNLqfxDUNeLhNYGO2dlPqsnuCg8N8a2a9u1ltqBOPQ4vj1Gf7eJC0W2hHG2eYUHuiXgY2w==", + "version": "14.2.2", + "resolved": "https://registry.npmjs.org/@next/swc-win32-x64-msvc/-/swc-win32-x64-msvc-14.2.2.tgz", + "integrity": "sha512-a/20E/wtTJZ3Ykv3f/8F0l7TtgQa2LWHU2oNB9bsu0VjqGuGGHmm/q6waoUNQYTVPYrrlxxaHjJcDV6aiSTt/w==", "cpu": [ "x64" ], @@ -2010,9 +2010,9 @@ } }, "node_modules/@rushstack/eslint-patch": { - "version": "1.10.1", - "resolved": "https://registry.npmjs.org/@rushstack/eslint-patch/-/eslint-patch-1.10.1.tgz", - "integrity": "sha512-S3Kq8e7LqxkA9s7HKLqXGTGck1uwis5vAXan3FnU5yw1Ec5hsSGnq4s/UCaSqABPOnOTg7zASLyst7+ohgWexg==", + "version": "1.10.2", + "resolved": "https://registry.npmjs.org/@rushstack/eslint-patch/-/eslint-patch-1.10.2.tgz", + "integrity": "sha512-hw437iINopmQuxWPSUEvqE56NCPsiU8N4AYtfHmJFckclktzK9YQJieD3XkDCDH4OjL+C7zgPUh73R/nrcHrqw==", "dev": true }, "node_modules/@sindresorhus/merge-streams": { @@ -2027,20 +2027,26 @@ "url": "https://github.com/sponsors/sindresorhus" } }, + "node_modules/@swc/counter": { + "version": "0.1.3", + "resolved": "https://registry.npmjs.org/@swc/counter/-/counter-0.1.3.tgz", + "integrity": "sha512-e2BR4lsJkkRlKZ/qCHPw9ZaSxc0MVUd7gtbtaB7aMvHeJVYe8sOB8DBZkP2DtISHGSku9sCK6T6cnY0CtXrOCQ==" + }, "node_modules/@swc/helpers": { - "version": "0.5.2", - "resolved": "https://registry.npmjs.org/@swc/helpers/-/helpers-0.5.2.tgz", - "integrity": "sha512-E4KcWTpoLHqwPHLxidpOqQbcrZVgi0rsmmZXUle1jXmJfuIf/UWpczUJ7MZZ5tlxytgJXyp0w4PGkkeLiuIdZw==", + "version": "0.5.5", + "resolved": "https://registry.npmjs.org/@swc/helpers/-/helpers-0.5.5.tgz", + "integrity": "sha512-KGYxvIOXcceOAbEk4bi/dVLEK9z8sZ0uBB3Il5b1rhfClSpcX0yfRO0KmTkqR2cnQDymwLB+25ZyMzICg/cm/A==", "dependencies": { + "@swc/counter": "^0.1.3", "tslib": "^2.4.0" } }, "node_modules/@tanstack/react-virtual": { - "version": "3.2.0", - "resolved": "https://registry.npmjs.org/@tanstack/react-virtual/-/react-virtual-3.2.0.tgz", - "integrity": "sha512-OEdMByf2hEfDa6XDbGlZN8qO6bTjlNKqjM3im9JG+u3mCL8jALy0T/67oDI001raUUPh1Bdmfn4ZvPOV5knpcg==", + "version": "3.4.0", + "resolved": "https://registry.npmjs.org/@tanstack/react-virtual/-/react-virtual-3.4.0.tgz", + "integrity": "sha512-GZN4xn/Tg5w7gvYeVcMVCeL4pEyUhvg+Cp6KX2Z01C4FRNxIWMgIQ9ibgMarNQfo+gt0PVLcEER4A9sNv/jlow==", "dependencies": { - "@tanstack/virtual-core": "3.2.0" + "@tanstack/virtual-core": "3.4.0" }, "funding": { "type": "github", @@ -2052,18 +2058,18 @@ } }, "node_modules/@tanstack/virtual-core": { - "version": "3.2.0", - "resolved": "https://registry.npmjs.org/@tanstack/virtual-core/-/virtual-core-3.2.0.tgz", - "integrity": "sha512-P5XgYoAw/vfW65byBbJQCw+cagdXDT/qH6wmABiLt4v4YBT2q2vqCOhihe+D1Nt325F/S/0Tkv6C5z0Lv+VBQQ==", + "version": "3.4.0", + "resolved": "https://registry.npmjs.org/@tanstack/virtual-core/-/virtual-core-3.4.0.tgz", + "integrity": "sha512-75jXqXxqq5M5Veb9KP1STi8kA5u408uOOAefk2ftHDGCpUk3RP6zX++QqfbmHJTBiU72NQ+ghgCZVts/Wocz8Q==", "funding": { "type": "github", "url": "https://github.com/sponsors/tannerlinsley" } }, "node_modules/@tremor/react": { - "version": "3.15.0", - "resolved": "https://registry.npmjs.org/@tremor/react/-/react-3.15.0.tgz", - "integrity": "sha512-TNhLzlGV9ph0Bzd7uTFHaE7Je9diIDuqt7Khnm5XkqO7y0jTE+9aGT/gcV6qTZV/9dfkcv8Aw1J/NpxH+hynGQ==", + "version": "3.16.1", + "resolved": "https://registry.npmjs.org/@tremor/react/-/react-3.16.1.tgz", + "integrity": "sha512-ablAsFL7twiUYf57gMAS2ar6YTajsQ7fs63vqlC2i9DpLuhFSrwEp69y1a6bxXP+KH0etYRMj6xxCvdCdiy8mQ==", "dependencies": { "@floating-ui/react": "^0.19.2", "@headlessui/react": "^1.7.18", @@ -2134,9 +2140,9 @@ "integrity": "sha512-Ps3T8E8dZDam6fUyNiMkekK3XUsaUEik+idO9/YjPtfj2qruF8tFBXS7XhtE4iIXBLxhmLjP3SXpLhVf21I9Lw==" }, "node_modules/@types/eslint": { - "version": "8.56.7", - "resolved": "https://registry.npmjs.org/@types/eslint/-/eslint-8.56.7.tgz", - "integrity": "sha512-SjDvI/x3zsZnOkYZ3lCt9lOZWZLB2jIlNKz+LBgCtDurK0JZcwucxYHn1w2BJkD34dgX9Tjnak0txtq4WTggEA==", + "version": "8.56.10", + "resolved": "https://registry.npmjs.org/@types/eslint/-/eslint-8.56.10.tgz", + "integrity": "sha512-Shavhk87gCtY2fhXDctcfS3e6FdxWkCx1iUZ9eEUbh7rTqlZT0/IzOkCOVt0fCjcFuZ9FPYfuezTBImfHCDBGQ==", "dev": true, "dependencies": { "@types/estree": "*", @@ -2172,9 +2178,9 @@ "dev": true }, "node_modules/@types/node": { - "version": "20.12.5", - "resolved": "https://registry.npmjs.org/@types/node/-/node-20.12.5.tgz", - "integrity": "sha512-BD+BjQ9LS/D8ST9p5uqBxghlN+S42iuNxjsUGjeZobe/ciXzk2qb1B6IXc6AnRLS+yFJRpN2IPEHMzwspfDJNw==", + "version": "20.12.7", + "resolved": "https://registry.npmjs.org/@types/node/-/node-20.12.7.tgz", + "integrity": "sha512-wq0cICSkRLVaf3UGLMGItu/PtdY7oaXaI/RVU+xliKVOtRna3PRY57ZDfztpDL0n11vfymMUnXv8QwYCO7L1wg==", "dependencies": { "undici-types": "~5.26.4" } @@ -2190,18 +2196,18 @@ "integrity": "sha512-5zvhXYtRNRluoE/jAp4GVsSduVUzNWKkOZrCDBWYtE7biZywwdC2AcEzg+cSMLFRfVgeAFqpfNabiPjxFddV1Q==" }, "node_modules/@types/react": { - "version": "18.2.74", - "resolved": "https://registry.npmjs.org/@types/react/-/react-18.2.74.tgz", - "integrity": "sha512-9AEqNZZyBx8OdZpxzQlaFEVCSFUM2YXJH46yPOiOpm078k6ZLOCcuAzGum/zK8YBwY+dbahVNbHrbgrAwIRlqw==", + "version": "18.2.79", + "resolved": "https://registry.npmjs.org/@types/react/-/react-18.2.79.tgz", + "integrity": "sha512-RwGAGXPl9kSXwdNTafkOEuFrTBD5SA2B3iEB96xi8+xu5ddUa/cpvyVCSNn+asgLCTHkb5ZxN8gbuibYJi4s1w==", "dependencies": { "@types/prop-types": "*", "csstype": "^3.0.2" } }, "node_modules/@types/react-dom": { - "version": "18.2.24", - "resolved": "https://registry.npmjs.org/@types/react-dom/-/react-dom-18.2.24.tgz", - "integrity": "sha512-cN6upcKd8zkGy4HU9F1+/s98Hrp6D4MOcippK4PoE8OZRngohHZpbJn1GsaDLz87MqvHNoT13nHvNqM9ocRHZg==", + "version": "18.2.25", + "resolved": "https://registry.npmjs.org/@types/react-dom/-/react-dom-18.2.25.tgz", + "integrity": "sha512-o/V48vf4MQh7juIKZU2QGDfli6p1+OOi5oXx36Hffpc9adsHeXjVp8rHuPkjd8VT8sOJ2Zp05HR7CdpGTIUFUA==", "dependencies": { "@types/react": "*" } @@ -2219,14 +2225,42 @@ "resolved": "https://registry.npmjs.org/@types/stylis/-/stylis-4.2.0.tgz", "integrity": "sha512-n4sx2bqL0mW1tvDf/loQ+aMX7GQD3lc3fkCMC55VFNDu/vBOabO+LTIeXKM14xK0ppk5TUGcWRjiSpIlUpghKw==" }, + "node_modules/@typescript-eslint/parser": { + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/parser/-/parser-7.2.0.tgz", + "integrity": "sha512-5FKsVcHTk6TafQKQbuIVkXq58Fnbkd2wDL4LB7AURN7RUOu1utVP+G8+6u3ZhEroW3DF6hyo3ZEXxgKgp4KeCg==", + "dev": true, + "dependencies": { + "@typescript-eslint/scope-manager": "7.2.0", + "@typescript-eslint/types": "7.2.0", + "@typescript-eslint/typescript-estree": "7.2.0", + "@typescript-eslint/visitor-keys": "7.2.0", + "debug": "^4.3.4" + }, + "engines": { + "node": "^16.0.0 || >=18.0.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + }, + "peerDependencies": { + "eslint": "^8.56.0" + }, + "peerDependenciesMeta": { + "typescript": { + "optional": true + } + } + }, "node_modules/@typescript-eslint/scope-manager": { - "version": "6.21.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/scope-manager/-/scope-manager-6.21.0.tgz", - "integrity": "sha512-OwLUIWZJry80O99zvqXVEioyniJMa+d2GrqpUTqi5/v5D5rOrppJVBPa0yKCblcigC0/aYAzxxqQ1B+DS2RYsg==", + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/scope-manager/-/scope-manager-7.2.0.tgz", + "integrity": "sha512-Qh976RbQM/fYtjx9hs4XkayYujB/aPwglw2choHmf3zBjB4qOywWSdt9+KLRdHubGcoSwBnXUH2sR3hkyaERRg==", "dev": true, "dependencies": { - "@typescript-eslint/types": "6.21.0", - "@typescript-eslint/visitor-keys": "6.21.0" + "@typescript-eslint/types": "7.2.0", + "@typescript-eslint/visitor-keys": "7.2.0" }, "engines": { "node": "^16.0.0 || >=18.0.0" @@ -2237,9 +2271,9 @@ } }, "node_modules/@typescript-eslint/types": { - "version": "6.21.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/types/-/types-6.21.0.tgz", - "integrity": "sha512-1kFmZ1rOm5epu9NZEZm1kckCDGj5UJEf7P1kliH4LKu/RkwpsfqqGmY2OOcUs18lSlQBKLDYBOGxRVtrMN5lpg==", + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/types/-/types-7.2.0.tgz", + "integrity": "sha512-XFtUHPI/abFhm4cbCDc5Ykc8npOKBSJePY3a3s+lwumt7XWJuzP5cZcfZ610MIPHjQjNsOLlYK8ASPaNG8UiyA==", "dev": true, "engines": { "node": "^16.0.0 || >=18.0.0" @@ -2250,13 +2284,13 @@ } }, "node_modules/@typescript-eslint/typescript-estree": { - "version": "6.21.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/typescript-estree/-/typescript-estree-6.21.0.tgz", - "integrity": "sha512-6npJTkZcO+y2/kr+z0hc4HwNfrrP4kNYh57ek7yCNlrBjWQ1Y0OS7jiZTkgumrvkX5HkEKXFZkkdFNkaW2wmUQ==", + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/typescript-estree/-/typescript-estree-7.2.0.tgz", + "integrity": "sha512-cyxS5WQQCoBwSakpMrvMXuMDEbhOo9bNHHrNcEWis6XHx6KF518tkF1wBvKIn/tpq5ZpUYK7Bdklu8qY0MsFIA==", "dev": true, "dependencies": { - "@typescript-eslint/types": "6.21.0", - "@typescript-eslint/visitor-keys": "6.21.0", + "@typescript-eslint/types": "7.2.0", + "@typescript-eslint/visitor-keys": "7.2.0", "debug": "^4.3.4", "globby": "^11.1.0", "is-glob": "^4.0.3", @@ -2331,12 +2365,12 @@ } }, "node_modules/@typescript-eslint/visitor-keys": { - "version": "6.21.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/visitor-keys/-/visitor-keys-6.21.0.tgz", - "integrity": "sha512-JJtkDduxLi9bivAB+cYOVMtbkqdPOhZ+ZI5LC47MIRrDV4Yn2o+ZnW10Nkmr28xRpSpdJ6Sm42Hjf2+REYXm0A==", + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/visitor-keys/-/visitor-keys-7.2.0.tgz", + "integrity": "sha512-c6EIQRHhcpl6+tO8EMR+kjkkV+ugUNXOmeASA1rlzkd8EPIriavpWoiEz1HR/VLhbVIdhqnV6E7JZm00cBDx2A==", "dev": true, "dependencies": { - "@typescript-eslint/types": "6.21.0", + "@typescript-eslint/types": "7.2.0", "eslint-visitor-keys": "^3.4.1" }, "engines": { @@ -3078,9 +3112,9 @@ } }, "node_modules/caniuse-lite": { - "version": "1.0.30001606", - "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001606.tgz", - "integrity": "sha512-LPbwnW4vfpJId225pwjZJOgX1m9sGfbw/RKJvw/t0QhYOOaTXHvkjVGFGPpvwEzufrjvTlsULnVTxdy4/6cqkg==", + "version": "1.0.30001612", + "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001612.tgz", + "integrity": "sha512-lFgnZ07UhaCcsSZgWW0K5j4e69dK1u/ltrL9lTUiFOwNHs12S3UMIEYgBV0Z6C6hRDev7iRnMzzYmKabYdXF9g==", "funding": [ { "type": "opencollective", @@ -3666,15 +3700,15 @@ "integrity": "sha512-+HlytyjlPKnIG8XuRG8WvmBP8xs8P71y+SKKS6ZXWoEgLuePxtDoUEiH7WkdePWrQ5JBpE6aoVqfZfJUQkjXwA==" }, "node_modules/doctrine": { - "version": "2.1.0", - "resolved": "https://registry.npmjs.org/doctrine/-/doctrine-2.1.0.tgz", - "integrity": "sha512-35mSku4ZXK0vfCuHEDAwt55dg2jNajHZ1odvF+8SSr82EsZY4QmXfuWso8oEd8zRhVObSN18aM0CjSdoBX7zIw==", + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/doctrine/-/doctrine-3.0.0.tgz", + "integrity": "sha512-yS+Q5i3hBf7GBkd4KG8a7eBNNWNGLTaEwwYWUijIYM7zrlYDM0BFXHjjPWlWZ1Rg7UaddZeIDmi9jF3HmqiQ2w==", "dev": true, "dependencies": { "esutils": "^2.0.2" }, "engines": { - "node": ">=0.10.0" + "node": ">=6.0.0" } }, "node_modules/dom-helpers": { @@ -3692,9 +3726,9 @@ "integrity": "sha512-I88TYZWc9XiYHRQ4/3c5rjjfgkjhLyW2luGIheGERbNQ6OY7yTybanSpDXZa8y7VUP9YmDcYa+eyq4ca7iLqWA==" }, "node_modules/electron-to-chromium": { - "version": "1.4.729", - "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.729.tgz", - "integrity": "sha512-bx7+5Saea/qu14kmPTDHQxkp2UnziG3iajUQu3BxFvCOnpAJdDbMV4rSl+EqFDkkpNNVUFlR1kDfpL59xfy1HA==", + "version": "1.4.745", + "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.745.tgz", + "integrity": "sha512-tRbzkaRI5gbUn5DEvF0dV4TQbMZ5CLkWeTAXmpC9IrYT+GE+x76i9p+o3RJ5l9XmdQlI1pPhVtE9uNcJJ0G0EA==", "dev": true }, "node_modules/email-addresses": { @@ -3981,14 +4015,14 @@ } }, "node_modules/eslint-config-next": { - "version": "14.1.4", - "resolved": "https://registry.npmjs.org/eslint-config-next/-/eslint-config-next-14.1.4.tgz", - "integrity": "sha512-cihIahbhYAWwXJwZkAaRPpUi5t9aOi/HdfWXOjZeUOqNWXHD8X22kd1KG58Dc3MVaRx3HoR/oMGk2ltcrqDn8g==", + "version": "14.2.2", + "resolved": "https://registry.npmjs.org/eslint-config-next/-/eslint-config-next-14.2.2.tgz", + "integrity": "sha512-12/uFc0KX+wUs7EDpOUGKMXBXZJiBVGdK5/m/QgXOCg2mQ0bQWoKSWNrCeOg7Vum6Kw1d1TW453W6xh+GbHquw==", "dev": true, "dependencies": { - "@next/eslint-plugin-next": "14.1.4", + "@next/eslint-plugin-next": "14.2.2", "@rushstack/eslint-patch": "^1.3.3", - "@typescript-eslint/parser": "^5.4.2 || ^6.0.0", + "@typescript-eslint/parser": "^5.4.2 || ^6.0.0 || 7.0.0 - 7.2.0", "eslint-import-resolver-node": "^0.3.6", "eslint-import-resolver-typescript": "^3.5.2", "eslint-plugin-import": "^2.28.1", @@ -4006,35 +4040,39 @@ } } }, - "node_modules/eslint-config-next/node_modules/@typescript-eslint/parser": { - "version": "6.21.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/parser/-/parser-6.21.0.tgz", - "integrity": "sha512-tbsV1jPne5CkFQCgPBcDOt30ItF7aJoZL997JSF7MhGQqOeT3svWRYxiqlfA5RUdlHN6Fi+EI9bxqbdyAUZjYQ==", + "node_modules/eslint-config-prettier": { + "version": "9.1.0", + "resolved": "https://registry.npmjs.org/eslint-config-prettier/-/eslint-config-prettier-9.1.0.tgz", + "integrity": "sha512-NSWl5BFQWEPi1j4TjVNItzYV7dZXZ+wP6I6ZhrBGpChQhZRUaElihE9uRRkcbRnNb76UMKDF3r+WTmNcGPKsqw==", "dev": true, - "dependencies": { - "@typescript-eslint/scope-manager": "6.21.0", - "@typescript-eslint/types": "6.21.0", - "@typescript-eslint/typescript-estree": "6.21.0", - "@typescript-eslint/visitor-keys": "6.21.0", - "debug": "^4.3.4" - }, - "engines": { - "node": "^16.0.0 || >=18.0.0" - }, - "funding": { - "type": "opencollective", - "url": "https://opencollective.com/typescript-eslint" + "bin": { + "eslint-config-prettier": "bin/cli.js" }, "peerDependencies": { - "eslint": "^7.0.0 || ^8.0.0" - }, - "peerDependenciesMeta": { - "typescript": { - "optional": true - } + "eslint": ">=7.0.0" + } + }, + "node_modules/eslint-import-resolver-node": { + "version": "0.3.9", + "resolved": "https://registry.npmjs.org/eslint-import-resolver-node/-/eslint-import-resolver-node-0.3.9.tgz", + "integrity": "sha512-WFj2isz22JahUv+B788TlO3N6zL3nNJGU8CcZbPZvVEkBPaJdCV4vy5wyghty5ROFbCRnm132v8BScu5/1BQ8g==", + "dev": true, + "dependencies": { + "debug": "^3.2.7", + "is-core-module": "^2.13.0", + "resolve": "^1.22.4" } }, - "node_modules/eslint-config-next/node_modules/eslint-import-resolver-typescript": { + "node_modules/eslint-import-resolver-node/node_modules/debug": { + "version": "3.2.7", + "resolved": "https://registry.npmjs.org/debug/-/debug-3.2.7.tgz", + "integrity": "sha512-CFjzYYAi4ThfiQvizrFQevTTXHtnCqWfe7x1AhgEscTz6ZbLbfoLRLPugTQyBth6f8ZERVUSyWHFD/7Wu4t1XQ==", + "dev": true, + "dependencies": { + "ms": "^2.1.1" + } + }, + "node_modules/eslint-import-resolver-typescript": { "version": "3.6.1", "resolved": "https://registry.npmjs.org/eslint-import-resolver-typescript/-/eslint-import-resolver-typescript-3.6.1.tgz", "integrity": "sha512-xgdptdoi5W3niYeuQxKmzVDTATvLYqhpwmykwsh7f6HIOStGWEIL9iqZgQDF9u9OEzrRwR8no5q2VT+bjAujTg==", @@ -4059,7 +4097,33 @@ "eslint-plugin-import": "*" } }, - "node_modules/eslint-config-next/node_modules/eslint-plugin-import": { + "node_modules/eslint-module-utils": { + "version": "2.8.1", + "resolved": "https://registry.npmjs.org/eslint-module-utils/-/eslint-module-utils-2.8.1.tgz", + "integrity": "sha512-rXDXR3h7cs7dy9RNpUlQf80nX31XWJEyGq1tRMo+6GsO5VmTe4UTwtmonAD4ZkAsrfMVDA2wlGJ3790Ys+D49Q==", + "dev": true, + "dependencies": { + "debug": "^3.2.7" + }, + "engines": { + "node": ">=4" + }, + "peerDependenciesMeta": { + "eslint": { + "optional": true + } + } + }, + "node_modules/eslint-module-utils/node_modules/debug": { + "version": "3.2.7", + "resolved": "https://registry.npmjs.org/debug/-/debug-3.2.7.tgz", + "integrity": "sha512-CFjzYYAi4ThfiQvizrFQevTTXHtnCqWfe7x1AhgEscTz6ZbLbfoLRLPugTQyBth6f8ZERVUSyWHFD/7Wu4t1XQ==", + "dev": true, + "dependencies": { + "ms": "^2.1.1" + } + }, + "node_modules/eslint-plugin-import": { "version": "2.29.1", "resolved": "https://registry.npmjs.org/eslint-plugin-import/-/eslint-plugin-import-2.29.1.tgz", "integrity": "sha512-BbPC0cuExzhiMo4Ff1BTVwHpjjv28C5R+btTOGaCRC7UEz801up0JadwkeSk5Ued6TG34uaczuVuH6qyy5YUxw==", @@ -4090,7 +4154,7 @@ "eslint": "^2 || ^3 || ^4 || ^5 || ^6 || ^7.2.0 || ^8" } }, - "node_modules/eslint-config-next/node_modules/eslint-plugin-import/node_modules/debug": { + "node_modules/eslint-plugin-import/node_modules/debug": { "version": "3.2.7", "resolved": "https://registry.npmjs.org/debug/-/debug-3.2.7.tgz", "integrity": "sha512-CFjzYYAi4ThfiQvizrFQevTTXHtnCqWfe7x1AhgEscTz6ZbLbfoLRLPugTQyBth6f8ZERVUSyWHFD/7Wu4t1XQ==", @@ -4099,7 +4163,28 @@ "ms": "^2.1.1" } }, - "node_modules/eslint-config-next/node_modules/eslint-plugin-jsx-a11y": { + "node_modules/eslint-plugin-import/node_modules/doctrine": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/doctrine/-/doctrine-2.1.0.tgz", + "integrity": "sha512-35mSku4ZXK0vfCuHEDAwt55dg2jNajHZ1odvF+8SSr82EsZY4QmXfuWso8oEd8zRhVObSN18aM0CjSdoBX7zIw==", + "dev": true, + "dependencies": { + "esutils": "^2.0.2" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/eslint-plugin-import/node_modules/semver": { + "version": "6.3.1", + "resolved": "https://registry.npmjs.org/semver/-/semver-6.3.1.tgz", + "integrity": "sha512-BR7VvDCVHO+q2xBEWskxS6DJE1qRnb7DxzUrogb71CWoSficBxYsiAGd+Kl0mmq/MprG9yArRkyrQxTO6XjMzA==", + "dev": true, + "bin": { + "semver": "bin/semver.js" + } + }, + "node_modules/eslint-plugin-jsx-a11y": { "version": "6.8.0", "resolved": "https://registry.npmjs.org/eslint-plugin-jsx-a11y/-/eslint-plugin-jsx-a11y-6.8.0.tgz", "integrity": "sha512-Hdh937BS3KdwwbBaKd5+PLCOmYY6U4f2h9Z2ktwtNKvIdIEu137rjYbcb9ApSbVJfWxANNuiKTD/9tOKjK9qOA==", @@ -4129,7 +4214,7 @@ "eslint": "^3 || ^4 || ^5 || ^6 || ^7 || ^8" } }, - "node_modules/eslint-config-next/node_modules/eslint-plugin-react": { + "node_modules/eslint-plugin-react": { "version": "7.34.1", "resolved": "https://registry.npmjs.org/eslint-plugin-react/-/eslint-plugin-react-7.34.1.tgz", "integrity": "sha512-N97CxlouPT1AHt8Jn0mhhN2RrADlUAsk1/atcT2KyA/l9Q/E6ll7OIGwNumFmWfZ9skV3XXccYS19h80rHtgkw==", @@ -4161,7 +4246,7 @@ "eslint": "^3 || ^4 || ^5 || ^6 || ^7 || ^8" } }, - "node_modules/eslint-config-next/node_modules/eslint-plugin-react-hooks": { + "node_modules/eslint-plugin-react-hooks": { "version": "4.6.0", "resolved": "https://registry.npmjs.org/eslint-plugin-react-hooks/-/eslint-plugin-react-hooks-4.6.0.tgz", "integrity": "sha512-oFc7Itz9Qxh2x4gNHStv3BqJq54ExXmfC+a1NjAta66IAN87Wu0R/QArgIS9qKzX3dXKPI9H5crl9QchNMY9+g==", @@ -4173,7 +4258,19 @@ "eslint": "^3.0.0 || ^4.0.0 || ^5.0.0 || ^6.0.0 || ^7.0.0 || ^8.0.0-0" } }, - "node_modules/eslint-config-next/node_modules/resolve": { + "node_modules/eslint-plugin-react/node_modules/doctrine": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/doctrine/-/doctrine-2.1.0.tgz", + "integrity": "sha512-35mSku4ZXK0vfCuHEDAwt55dg2jNajHZ1odvF+8SSr82EsZY4QmXfuWso8oEd8zRhVObSN18aM0CjSdoBX7zIw==", + "dev": true, + "dependencies": { + "esutils": "^2.0.2" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/eslint-plugin-react/node_modules/resolve": { "version": "2.0.0-next.5", "resolved": "https://registry.npmjs.org/resolve/-/resolve-2.0.0-next.5.tgz", "integrity": "sha512-U7WjGVG9sH8tvjW5SmGbQuui75FiyjAX72HX15DwBBwF9dNiQZRQAg9nnPhYy+TUnE0+VcrttuvNI8oSxZcocA==", @@ -4190,7 +4287,7 @@ "url": "https://github.com/sponsors/ljharb" } }, - "node_modules/eslint-config-next/node_modules/semver": { + "node_modules/eslint-plugin-react/node_modules/semver": { "version": "6.3.1", "resolved": "https://registry.npmjs.org/semver/-/semver-6.3.1.tgz", "integrity": "sha512-BR7VvDCVHO+q2xBEWskxS6DJE1qRnb7DxzUrogb71CWoSficBxYsiAGd+Kl0mmq/MprG9yArRkyrQxTO6XjMzA==", @@ -4199,64 +4296,6 @@ "semver": "bin/semver.js" } }, - "node_modules/eslint-config-prettier": { - "version": "9.1.0", - "resolved": "https://registry.npmjs.org/eslint-config-prettier/-/eslint-config-prettier-9.1.0.tgz", - "integrity": "sha512-NSWl5BFQWEPi1j4TjVNItzYV7dZXZ+wP6I6ZhrBGpChQhZRUaElihE9uRRkcbRnNb76UMKDF3r+WTmNcGPKsqw==", - "dev": true, - "bin": { - "eslint-config-prettier": "bin/cli.js" - }, - "peerDependencies": { - "eslint": ">=7.0.0" - } - }, - "node_modules/eslint-import-resolver-node": { - "version": "0.3.9", - "resolved": "https://registry.npmjs.org/eslint-import-resolver-node/-/eslint-import-resolver-node-0.3.9.tgz", - "integrity": "sha512-WFj2isz22JahUv+B788TlO3N6zL3nNJGU8CcZbPZvVEkBPaJdCV4vy5wyghty5ROFbCRnm132v8BScu5/1BQ8g==", - "dev": true, - "dependencies": { - "debug": "^3.2.7", - "is-core-module": "^2.13.0", - "resolve": "^1.22.4" - } - }, - "node_modules/eslint-import-resolver-node/node_modules/debug": { - "version": "3.2.7", - "resolved": "https://registry.npmjs.org/debug/-/debug-3.2.7.tgz", - "integrity": "sha512-CFjzYYAi4ThfiQvizrFQevTTXHtnCqWfe7x1AhgEscTz6ZbLbfoLRLPugTQyBth6f8ZERVUSyWHFD/7Wu4t1XQ==", - "dev": true, - "dependencies": { - "ms": "^2.1.1" - } - }, - "node_modules/eslint-module-utils": { - "version": "2.8.1", - "resolved": "https://registry.npmjs.org/eslint-module-utils/-/eslint-module-utils-2.8.1.tgz", - "integrity": "sha512-rXDXR3h7cs7dy9RNpUlQf80nX31XWJEyGq1tRMo+6GsO5VmTe4UTwtmonAD4ZkAsrfMVDA2wlGJ3790Ys+D49Q==", - "dev": true, - "dependencies": { - "debug": "^3.2.7" - }, - "engines": { - "node": ">=4" - }, - "peerDependenciesMeta": { - "eslint": { - "optional": true - } - } - }, - "node_modules/eslint-module-utils/node_modules/debug": { - "version": "3.2.7", - "resolved": "https://registry.npmjs.org/debug/-/debug-3.2.7.tgz", - "integrity": "sha512-CFjzYYAi4ThfiQvizrFQevTTXHtnCqWfe7x1AhgEscTz6ZbLbfoLRLPugTQyBth6f8ZERVUSyWHFD/7Wu4t1XQ==", - "dev": true, - "dependencies": { - "ms": "^2.1.1" - } - }, "node_modules/eslint-scope": { "version": "7.2.2", "resolved": "https://registry.npmjs.org/eslint-scope/-/eslint-scope-7.2.2.tgz", @@ -4285,18 +4324,6 @@ "url": "https://opencollective.com/eslint" } }, - "node_modules/eslint/node_modules/doctrine": { - "version": "3.0.0", - "resolved": "https://registry.npmjs.org/doctrine/-/doctrine-3.0.0.tgz", - "integrity": "sha512-yS+Q5i3hBf7GBkd4KG8a7eBNNWNGLTaEwwYWUijIYM7zrlYDM0BFXHjjPWlWZ1Rg7UaddZeIDmi9jF3HmqiQ2w==", - "dev": true, - "dependencies": { - "esutils": "^2.0.2" - }, - "engines": { - "node": ">=6.0.0" - } - }, "node_modules/espree": { "version": "9.6.1", "resolved": "https://registry.npmjs.org/espree/-/espree-9.6.1.tgz", @@ -5837,9 +5864,9 @@ } }, "node_modules/lucide-react": { - "version": "0.365.0", - "resolved": "https://registry.npmjs.org/lucide-react/-/lucide-react-0.365.0.tgz", - "integrity": "sha512-sJYpPyyzGHI4B3pys+XSFnE4qtSWc68rFnDLxbNNKjkLST5XSx9DNn5+1Z3eFgFiw39PphNRiVBSVb+AL3oKwA==", + "version": "0.372.0", + "resolved": "https://registry.npmjs.org/lucide-react/-/lucide-react-0.372.0.tgz", + "integrity": "sha512-0cKdqmilHXWUwWAWnf6CrrjHD8YaqPMtLrmEHXolZusNTr9epULCsiJwIOHk2q1yFxdEwd96D4zShlAj67UJdA==", "peerDependencies": { "react": "^16.5.1 || ^17.0.0 || ^18.0.0" } @@ -5869,9 +5896,9 @@ } }, "node_modules/material-symbols": { - "version": "0.17.1", - "resolved": "https://registry.npmjs.org/material-symbols/-/material-symbols-0.17.1.tgz", - "integrity": "sha512-1kJan8t3U3Fmuu/YPu2MVsL/ODSja71o+J7ODROQfMaCzzal0izY4SATafEKgXUXU+jL0zIiBQdyzsno7vXBvA==" + "version": "0.17.2", + "resolved": "https://registry.npmjs.org/material-symbols/-/material-symbols-0.17.2.tgz", + "integrity": "sha512-PfquyvrUTbqUBa4FLqGLBM5eaAlnNMuhqB7rXu+OuGfmmvx+Ste8dmuNEDNC+GTcqtyvDmfCL/EWz3z0XXjncA==" }, "node_modules/memoize-one": { "version": "6.0.0", @@ -6055,12 +6082,12 @@ "dev": true }, "node_modules/next": { - "version": "14.1.4", - "resolved": "https://registry.npmjs.org/next/-/next-14.1.4.tgz", - "integrity": "sha512-1WTaXeSrUwlz/XcnhGTY7+8eiaFvdet5z9u3V2jb+Ek1vFo0VhHKSAIJvDWfQpttWjnyw14kBeq28TPq7bTeEQ==", + "version": "14.2.2", + "resolved": "https://registry.npmjs.org/next/-/next-14.2.2.tgz", + "integrity": "sha512-oGwUaa2bCs47FbuxWMpOoXtBMPYpvTPgdZr3UAo+pu7Ns00z9otmYpoeV1HEiYL06AlRQQIA/ypK526KjJfaxg==", "dependencies": { - "@next/env": "14.1.4", - "@swc/helpers": "0.5.2", + "@next/env": "14.2.2", + "@swc/helpers": "0.5.5", "busboy": "1.6.0", "caniuse-lite": "^1.0.30001579", "graceful-fs": "^4.2.11", @@ -6074,18 +6101,19 @@ "node": ">=18.17.0" }, "optionalDependencies": { - "@next/swc-darwin-arm64": "14.1.4", - "@next/swc-darwin-x64": "14.1.4", - "@next/swc-linux-arm64-gnu": "14.1.4", - "@next/swc-linux-arm64-musl": "14.1.4", - "@next/swc-linux-x64-gnu": "14.1.4", - "@next/swc-linux-x64-musl": "14.1.4", - "@next/swc-win32-arm64-msvc": "14.1.4", - "@next/swc-win32-ia32-msvc": "14.1.4", - "@next/swc-win32-x64-msvc": "14.1.4" + "@next/swc-darwin-arm64": "14.2.2", + "@next/swc-darwin-x64": "14.2.2", + "@next/swc-linux-arm64-gnu": "14.2.2", + "@next/swc-linux-arm64-musl": "14.2.2", + "@next/swc-linux-x64-gnu": "14.2.2", + "@next/swc-linux-x64-musl": "14.2.2", + "@next/swc-win32-arm64-msvc": "14.2.2", + "@next/swc-win32-ia32-msvc": "14.2.2", + "@next/swc-win32-x64-msvc": "14.2.2" }, "peerDependencies": { "@opentelemetry/api": "^1.1.0", + "@playwright/test": "^1.41.2", "react": "^18.2.0", "react-dom": "^18.2.0", "sass": "^1.3.0" @@ -6094,6 +6122,9 @@ "@opentelemetry/api": { "optional": true }, + "@playwright/test": { + "optional": true + }, "sass": { "optional": true } @@ -6782,9 +6813,9 @@ "integrity": "sha512-1NNCs6uurfkVbeXG4S8JFT9t19m45ICnif8zWLd5oPSZ50QnwMfK+H3jv408d4jw/7Bttv5axS5IiHoLaVNHeQ==" }, "node_modules/preact": { - "version": "10.20.1", - "resolved": "https://registry.npmjs.org/preact/-/preact-10.20.1.tgz", - "integrity": "sha512-JIFjgFg9B2qnOoGiYMVBtrcFxHqn+dNXbq76bVmcaHYJFYR4lW67AOcXgAYQQTDYXDOg/kTZrKPNCdRgJ2UJmw==", + "version": "10.20.2", + "resolved": "https://registry.npmjs.org/preact/-/preact-10.20.2.tgz", + "integrity": "sha512-S1d1ernz3KQ+Y2awUxKakpfOg2CEmJmwOP+6igPx6dgr6pgDvenqYviyokWso2rhHvGtTlWWnJDa7RaPbQerTg==", "funding": { "type": "opencollective", "url": "https://opencollective.com/preact" @@ -6955,9 +6986,9 @@ } }, "node_modules/react-day-picker": { - "version": "8.10.0", - "resolved": "https://registry.npmjs.org/react-day-picker/-/react-day-picker-8.10.0.tgz", - "integrity": "sha512-mz+qeyrOM7++1NCb1ARXmkjMkzWVh2GL9YiPbRjKe0zHccvekk4HE+0MPOZOrosn8r8zTHIIeOUXTmXRqmkRmg==", + "version": "8.10.1", + "resolved": "https://registry.npmjs.org/react-day-picker/-/react-day-picker-8.10.1.tgz", + "integrity": "sha512-TMx7fNbhLk15eqcMt+7Z7S2KF7mfTId/XJDjKE8f+IUcFn0l08/kI4FiYTL/0yuOLmEcbR4Fwe3GJf/NiiMnPA==", "funding": { "type": "individual", "url": "https://github.com/sponsors/gpbl" @@ -7150,9 +7181,9 @@ } }, "node_modules/recharts": { - "version": "2.12.4", - "resolved": "https://registry.npmjs.org/recharts/-/recharts-2.12.4.tgz", - "integrity": "sha512-dM4skmk4fDKEDjL9MNunxv6zcTxePGVEzRnLDXALRpfJ85JoQ0P0APJ/CoJlmnQI0gPjBlOkjzrwrfQrRST3KA==", + "version": "2.12.6", + "resolved": "https://registry.npmjs.org/recharts/-/recharts-2.12.6.tgz", + "integrity": "sha512-D+7j9WI+D0NHauah3fKHuNNcRK8bOypPW7os1DERinogGBGaHI7i6tQKJ0aUF3JXyBZ63dyfKIW2WTOPJDxJ8w==", "dependencies": { "clsx": "^2.0.0", "eventemitter3": "^4.0.1", @@ -8081,9 +8112,9 @@ } }, "node_modules/terser": { - "version": "5.30.3", - "resolved": "https://registry.npmjs.org/terser/-/terser-5.30.3.tgz", - "integrity": "sha512-STdUgOUx8rLbMGO9IOwHLpCqolkDITFFQSMYYwKE1N2lY6MVSaeoi10z/EhWxRc6ybqoVmKSkhKYH/XUpl7vSA==", + "version": "5.30.4", + "resolved": "https://registry.npmjs.org/terser/-/terser-5.30.4.tgz", + "integrity": "sha512-xRdd0v64a8mFK9bnsKVdoNP9GQIKUAaJPTaqEQDL4w/J8WaW4sWXXoMZ+6SimPkfT5bElreXf8m9HnmPc3E1BQ==", "dev": true, "dependencies": { "@jridgewell/source-map": "^0.3.3", @@ -8358,9 +8389,9 @@ } }, "node_modules/typescript": { - "version": "5.4.4", - "resolved": "https://registry.npmjs.org/typescript/-/typescript-5.4.4.tgz", - "integrity": "sha512-dGE2Vv8cpVvw28v8HCPqyb08EzbBURxDpuhJvTrusShUfGnhHBafDsLdS1EhhxyL6BJQE+2cT3dDPAv+MQ6oLw==", + "version": "5.4.5", + "resolved": "https://registry.npmjs.org/typescript/-/typescript-5.4.5.tgz", + "integrity": "sha512-vcI4UpRgg81oIRUFwR0WSIHKt11nJ7SAVlYNIu+QpqeyXP+gpQJy/Z4+F0aGxSE4MqwjyXvW/TzgkLAx2AGHwQ==", "dev": true, "bin": { "tsc": "bin/tsc", @@ -8948,9 +8979,9 @@ } }, "node_modules/zod": { - "version": "3.22.4", - "resolved": "https://registry.npmjs.org/zod/-/zod-3.22.4.tgz", - "integrity": "sha512-iC+8Io04lddc+mVqQ9AZ7OQ2MrUKGN+oIQyq1vemgt46jwCwLfhq7/pwnBnNXXXZb8VTVLKwp9EDkx+ryxIWmg==", + "version": "3.23.0", + "resolved": "https://registry.npmjs.org/zod/-/zod-3.23.0.tgz", + "integrity": "sha512-OFLT+LTocvabn6q76BTwVB0hExEBS0IduTr3cqZyMqEDbOnYmcU+y0tUAYbND4uwclpBGi4I4UUBGzylWpjLGA==", "funding": { "url": "https://github.com/sponsors/colinhacks" } diff --git a/ui/package.json b/ui/package.json index 95d2a56aeb..cbdb2ec6fa 100644 --- a/ui/package.json +++ b/ui/package.json @@ -28,17 +28,17 @@ "@radix-ui/react-toggle": "^1.0.3", "@radix-ui/react-toggle-group": "^1.0.4", "@radix-ui/react-tooltip": "^1.0.7", - "@tremor/react": "^3.15.0", - "@types/node": "^20.12.5", - "@types/react": "^18.2.74", - "@types/react-dom": "^18.2.24", + "@tremor/react": "^3.16.1", + "@types/node": "^20.12.7", + "@types/react": "^18.2.79", + "@types/react-dom": "^18.2.25", "classnames": "^2.5.1", "long": "^5.2.3", - "lucide-react": "^0.365.0", - "material-symbols": "^0.17.1", + "lucide-react": "^0.372.0", + "material-symbols": "^0.17.2", "moment": "^2.30.1", "moment-timezone": "^0.5.45", - "next": "^14.1.4", + "next": "^14.2.2", "next-auth": "^4.24.7", "prop-types": "^15.8.1", "protobufjs": "^7.2.6", @@ -50,13 +50,13 @@ "styled-components": "^6.1.8", "swr": "^2.2.5", "usehooks-ts": "^3.1.0", - "zod": "^3.22.4" + "zod": "^3.23.0" }, "devDependencies": { "autoprefixer": "^10.4.19", "copy-webpack-plugin": "^12.0.2", "eslint": "^8.57.0", - "eslint-config-next": "^14.1.4", + "eslint-config-next": "^14.2.2", "eslint-config-prettier": "^9.1.0", "gh-pages": "^6.1.1", "less": "^4.2.0", @@ -67,7 +67,7 @@ "string-width": "^7.1.0", "tailwindcss": "^3.4.3", "tailwindcss-animate": "^1.0.7", - "typescript": "^5.4.4", + "typescript": "^5.4.5", "webpack": "^5.91.0" } } diff --git a/ui/public/svgs/elasticsearch.svg b/ui/public/svgs/elasticsearch.svg new file mode 100644 index 0000000000..b95507cd54 --- /dev/null +++ b/ui/public/svgs/elasticsearch.svg @@ -0,0 +1 @@ + \ No newline at end of file