From e3f6ef658be8073490ba29de1c2180dbbb1fd77a Mon Sep 17 00:00:00 2001 From: Kevin Biju <52661649+heavycrystal@users.noreply.github.com> Date: Tue, 30 Apr 2024 10:26:19 +0530 Subject: [PATCH 01/41] Elasticsearch connector for CDC (#1649) Current limitations: 1) soft-delete does not work, needs to be handled specifically from the ES side 2) relies on implicit index creation, no custom mapping or other index configuration is possible 3) TOAST columns aren't handled properly --- flow/activities/flowable_core.go | 15 +- .../connelasticsearch/elasticsearch.go | 250 ++++++++++++------ flow/connectors/connelasticsearch/qrep.go | 175 ++++++++++++ flow/connectors/core.go | 1 + flow/connectors/postgres/cdc.go | 29 +- flow/connectors/postgres/postgres.go | 2 +- flow/e2e/elasticsearch/elasticsearch.go | 6 +- flow/e2e/elasticsearch/peer_flow_es_test.go | 150 +++++++++++ flow/e2e/elasticsearch/qrep_flow_es_test.go | 76 +++++- flow/model/model.go | 28 +- flow/workflows/snapshot_flow.go | 18 +- 11 files changed, 623 insertions(+), 127 deletions(-) create mode 100644 flow/connectors/connelasticsearch/qrep.go create mode 100644 flow/e2e/elasticsearch/peer_flow_es_test.go diff --git a/flow/activities/flowable_core.go b/flow/activities/flowable_core.go index e9687a2240..319da34c49 100644 --- a/flow/activities/flowable_core.go +++ b/flow/activities/flowable_core.go @@ -187,13 +187,14 @@ func syncCore[TPull connectors.CDCPullConnectorCore, TSync connectors.CDCSyncCon 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, + SyncBatchID: syncBatchID, + Records: recordBatch, + ConsumedOffset: &consumedOffset, + FlowJobName: flowName, + TableMappings: options.TableMappings, + StagingPath: config.CdcStagingPath, + Script: config.Script, + TableNameSchemaMapping: options.TableNameSchemaMapping, }) if err != nil { a.Alerter.LogFlowError(ctx, flowName, err) diff --git a/flow/connectors/connelasticsearch/elasticsearch.go b/flow/connectors/connelasticsearch/elasticsearch.go index 0672e05c20..8d5f092790 100644 --- a/flow/connectors/connelasticsearch/elasticsearch.go +++ b/flow/connectors/connelasticsearch/elasticsearch.go @@ -4,26 +4,36 @@ import ( "bytes" "context" "crypto/tls" + "encoding/base64" "encoding/json" + "errors" "fmt" "log/slog" "net/http" "sync" + "sync/atomic" "time" "github.com/elastic/go-elasticsearch/v8" "github.com/elastic/go-elasticsearch/v8/esutil" - "github.com/google/uuid" "go.temporal.io/sdk/log" + "golang.org/x/exp/maps" metadataStore "github.com/PeerDB-io/peer-flow/connectors/external_metadata" + "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/logger" "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" ) +const ( + actionIndex = "index" + actionDelete = "delete" +) + type ElasticsearchConnector struct { *metadataStore.PostgresMetadata client *elasticsearch.Client @@ -78,93 +88,170 @@ func (esc *ElasticsearchConnector) Close() error { return nil } -func (esc *ElasticsearchConnector) SetupQRepMetadataTables(ctx context.Context, - config *protos.QRepConfig, +// ES is queue-like, no raw table staging needed +func (esc *ElasticsearchConnector) CreateRawTable(ctx context.Context, + req *protos.CreateRawTableInput, +) (*protos.CreateRawTableOutput, error) { + return &protos.CreateRawTableOutput{TableIdentifier: "n/a"}, nil +} + +// we handle schema changes by not handling them since no mapping is being enforced right now +func (esc *ElasticsearchConnector) ReplayTableSchemaDeltas(ctx context.Context, + flowJobName string, schemaDeltas []*protos.TableSchemaDelta, ) error { return nil } -func (esc *ElasticsearchConnector) SyncQRepRecords(ctx context.Context, config *protos.QRepConfig, - partition *protos.QRepPartition, stream *model.QRecordStream, -) (int, error) { - startTime := time.Now() +func recordItemsProcessor(items model.RecordItems) ([]byte, error) { + qRecordJsonMap := make(map[string]any) + + for key, val := range items.ColToVal { + if r, ok := val.(qvalue.QValueJSON); ok { // JSON is stored as a string, fix that + qRecordJsonMap[key] = json.RawMessage( + shared.UnsafeFastStringToReadOnlyBytes(r.Val)) + } else { + qRecordJsonMap[key] = val.Value() + } + } - schema := stream.Schema() + return json.Marshal(qRecordJsonMap) +} - var bulkIndexFatalError error - 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 +func (esc *ElasticsearchConnector) SyncRecords(ctx context.Context, + req *model.SyncRecordsRequest[model.RecordItems], +) (*model.SyncResponse, error) { + tableNameRowsMapping := utils.InitialiseTableRowsMap(req.TableMappings) + // atomics for counts will be unnecessary in other destinations, using a mutex instead + var recordCountsUpdateMutex sync.Mutex + // we're taking a mutex anyway, avoid atomic + var lastSeenLSN atomic.Int64 + var numRecords atomic.Int64 + + // no I don't like this either + esBulkIndexerCache := make(map[string]esutil.BulkIndexer) + bulkIndexersHaveShutdown := false + // true if we saw errors while closing + cacheCloser := func() bool { + closeHasErrors := false + if bulkIndexersHaveShutdown { + for _, esBulkIndexer := range maps.Values(esBulkIndexerCache) { + err := esBulkIndexer.Close(context.Background()) + if err != nil { + esc.logger.Error("[es] failed to close bulk indexer", slog.Any("error", err)) + closeHasErrors = true + } } + bulkIndexersHaveShutdown = true } + return closeHasErrors } + defer cacheCloser() - 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)) + flushLoopDone := make(chan struct{}) + // we only update lastSeenLSN in the OnSuccess call, so this should be safe even if race + // between loop breaking and closing flushLoopDone + go func() { + ticker := time.NewTicker(peerdbenv.PeerDBQueueFlushTimeoutSeconds()) + defer ticker.Stop() + + for { + select { + case <-ctx.Done(): + return + case <-flushLoopDone: + return + case <-ticker.C: + lastSeen := lastSeenLSN.Load() + if lastSeen > req.ConsumedOffset.Load() { + if err := esc.SetLastOffset(ctx, req.FlowJobName, lastSeen); err != nil { + esc.logger.Warn("[es] SetLastOffset error", slog.Any("error", err)) + } else { + shared.AtomicInt64Max(req.ConsumedOffset, lastSeen) + esc.logger.Info("processBatch", slog.Int64("updated last offset", lastSeen)) + } + } } } }() - for qRecord := range stream.Records { - qRecordJsonMap := make(map[string]any) + var docId string + var bulkIndexFatalError error + var bulkIndexErrors []error + var bulkIndexOnFailureMutex sync.Mutex - if upsertColIndex >= 0 { - docId = fmt.Sprintf("%v", qRecord[upsertColIndex].Value()) - } else { - docId = uuid.New().String() + for record := range req.Records.GetRecords() { + var bodyBytes []byte + var err error + action := actionIndex + + switch record.(type) { + case *model.InsertRecord[model.RecordItems], *model.UpdateRecord[model.RecordItems]: + bodyBytes, err = recordItemsProcessor(record.GetItems()) + if err != nil { + esc.logger.Error("[es] failed to json.Marshal record", slog.Any("error", err)) + return nil, fmt.Errorf("[es] failed to json.Marshal record: %w", err) + } + case *model.DeleteRecord[model.RecordItems]: + action = actionDelete + // no need to supply the document since we are deleting + bodyBytes = nil } - 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() + + bulkIndexer, ok := esBulkIndexerCache[record.GetDestinationTableName()] + if !ok { + bulkIndexer, err = esutil.NewBulkIndexer(esutil.BulkIndexerConfig{ + Index: record.GetDestinationTableName(), + Client: esc.client, + // can't really ascertain how many tables present to provide a reasonable value + NumWorkers: 1, + FlushInterval: 10 * time.Second, + }) + if err != nil { + esc.logger.Error("[es] failed to initialize bulk indexer", slog.Any("error", err)) + return nil, fmt.Errorf("[es] failed to initialize bulk indexer: %w", err) } + esBulkIndexerCache[record.GetDestinationTableName()] = bulkIndexer } - 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) + + if len(req.TableNameSchemaMapping[record.GetDestinationTableName()].PrimaryKeyColumns) == 1 { + qValue, err := record.GetItems().GetValueByColName( + req.TableNameSchemaMapping[record.GetDestinationTableName()].PrimaryKeyColumns[0]) + if err != nil { + esc.logger.Error("[es] failed to process record", slog.Any("error", err)) + return nil, fmt.Errorf("[es] failed to process record: %w", err) + } + docId = fmt.Sprint(qValue.Value()) + } else { + tablePkey, err := model.RecToTablePKey(req.TableNameSchemaMapping, record) + if err != nil { + esc.logger.Error("[es] failed to process record", slog.Any("error", err)) + return nil, fmt.Errorf("[es] failed to process record: %w", err) + } + docId = base64.RawURLEncoding.EncodeToString(tablePkey.PkeyColVal[:]) } - err = esBulkIndexer.Add(ctx, esutil.BulkIndexerItem{ - Action: "index", + err = bulkIndexer.Add(ctx, esutil.BulkIndexerItem{ + Action: action, DocumentID: docId, - Body: bytes.NewReader(qRecordJsonBytes), + Body: bytes.NewReader(bodyBytes), + OnSuccess: func(_ context.Context, _ esutil.BulkIndexerItem, _ esutil.BulkIndexerResponseItem) { + shared.AtomicInt64Max(&lastSeenLSN, record.GetCheckpointID()) + numRecords.Add(1) + recordCountsUpdateMutex.Lock() + defer recordCountsUpdateMutex.Unlock() + record.PopulateCountMap(tableNameRowsMapping) + }, // 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() + // attempt to delete a record that wasn't present, possible from no initial load + if item.Action == actionDelete && res.Status == 404 { + return + } + bulkIndexOnFailureMutex.Lock() + defer bulkIndexOnFailureMutex.Unlock() if err != nil { bulkIndexErrors = append(bulkIndexErrors, err) } else { @@ -172,7 +259,7 @@ func (esc *ElasticsearchConnector) SyncQRepRecords(ctx context.Context, config * 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) } - cbErr := fmt.Errorf("id:%s type:%s reason:%s %s", item.DocumentID, res.Error.Type, + cbErr := fmt.Errorf("id:%s action:%s type:%s reason:%s %s", item.DocumentID, item.Action, res.Error.Type, res.Error.Reason, causeString) bulkIndexErrors = append(bulkIndexErrors, cbErr) if res.Error.Type == "illegal_argument_exception" { @@ -183,36 +270,37 @@ func (esc *ElasticsearchConnector) SyncQRepRecords(ctx context.Context, config * }) 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) + return nil, fmt.Errorf("[es] failed to add record to bulk indexer: %w", err) } if bulkIndexFatalError != nil { esc.logger.Error("[es] fatal error while indexing record", slog.Any("error", bulkIndexFatalError)) - return 0, fmt.Errorf("[es] fatal error while indexing record: %w", bulkIndexFatalError) + return nil, fmt.Errorf("[es] fatal error while indexing record: %w", bulkIndexFatalError) } - - // update here instead of OnSuccess, if we close successfully it should match - numRecords++ } + // "Receive on a closed channel yields the zero value after all elements in the channel are received." + close(flushLoopDone) - 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 cacheCloser() { + esc.logger.Error("[es] failed to close bulk indexer(s)") + return nil, errors.New("[es] failed to close bulk indexer(s)") } - 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 + bulkIndexersHaveShutdown = 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) + lastCheckpoint := req.Records.GetLastCheckpoint() + if err := esc.FinishBatch(ctx, req.FlowJobName, req.SyncBatchID, lastCheckpoint); err != nil { + return nil, err } - return numRecords, nil + + return &model.SyncResponse{ + CurrentSyncBatchID: req.SyncBatchID, + LastSyncedCheckpointID: lastCheckpoint, + NumRecordsSynced: numRecords.Load(), + TableNameRowsMapping: tableNameRowsMapping, + TableSchemaDeltas: req.Records.SchemaDeltas, + }, nil } diff --git a/flow/connectors/connelasticsearch/qrep.go b/flow/connectors/connelasticsearch/qrep.go new file mode 100644 index 0000000000..142c6de363 --- /dev/null +++ b/flow/connectors/connelasticsearch/qrep.go @@ -0,0 +1,175 @@ +package connelasticsearch + +import ( + "bytes" + "context" + "crypto/sha256" + "encoding/base64" + "encoding/json" + "fmt" + "log/slog" + "slices" + "sync" + "time" + + "github.com/elastic/go-elasticsearch/v8/esutil" + + "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 (esc *ElasticsearchConnector) SetupQRepMetadataTables(ctx context.Context, + config *protos.QRepConfig, +) error { + return nil +} + +func upsertKeyColsHash(qRecord []qvalue.QValue, upsertColIndices []int) string { + hasher := sha256.New() + + for _, upsertColIndex := range upsertColIndices { + // cannot return an error + _, _ = fmt.Fprint(hasher, qRecord[upsertColIndex].Value()) + } + hashBytes := hasher.Sum(nil) + return base64.RawURLEncoding.EncodeToString(hashBytes) +} + +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 bulkIndexFatalError error + var bulkIndexErrors []error + var bulkIndexOnFailureMutex sync.Mutex + var docId string + numRecords := 0 + bulkIndexerHasShutdown := false + + // len == 0 means use UUID + // len == 1 means single column, use value directly + // len > 1 means SHA256 hash of upsert key columns + // ordered such that we preserve order of UpsertKeyColumns + var upsertKeyColIndices []int + if config.WriteMode.WriteType == protos.QRepWriteType_QREP_WRITE_MODE_UPSERT { + schemaColNames := schema.GetColumnNames() + for _, upsertCol := range config.WriteMode.UpsertKeyColumns { + idx := slices.Index(schemaColNames, upsertCol) + if idx != -1 { + upsertKeyColIndices = append(upsertKeyColIndices, idx) + } + } + } + + 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) + + switch len(upsertKeyColIndices) { + case 0: + // relying on autogeneration of document ID + case 1: + docId = fmt.Sprint(qRecord[upsertKeyColIndices[0]].Value()) + default: + docId = upsertKeyColsHash(qRecord, upsertKeyColIndices) + } + for i, field := range schema.Fields { + if r, ok := qRecord[i].(qvalue.QValueJSON); ok { // JSON is stored as a string, fix that + qRecordJsonMap[field.Name] = json.RawMessage( + shared.UnsafeFastStringToReadOnlyBytes(r.Val)) + } else { + qRecordJsonMap[field.Name] = qRecord[i].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: actionIndex, + 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, + ) { + bulkIndexOnFailureMutex.Lock() + defer bulkIndexOnFailureMutex.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) + } + cbErr := fmt.Errorf("id:%s type:%s reason:%s %s", item.DocumentID, res.Error.Type, + res.Error.Reason, causeString) + bulkIndexErrors = append(bulkIndexErrors, cbErr) + if res.Error.Type == "illegal_argument_exception" { + bulkIndexFatalError = cbErr + } + } + }, + }) + 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) + } + if bulkIndexFatalError != nil { + esc.logger.Error("[es] fatal error while indexing record", slog.Any("error", bulkIndexFatalError)) + return 0, fmt.Errorf("[es] fatal error while indexing record: %w", bulkIndexFatalError) + } + + // 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 1e3321fa72..545269c4d1 100644 --- a/flow/connectors/core.go +++ b/flow/connectors/core.go @@ -304,6 +304,7 @@ var ( _ CDCSyncConnector = &connpubsub.PubSubConnector{} _ CDCSyncConnector = &conns3.S3Connector{} _ CDCSyncConnector = &connclickhouse.ClickhouseConnector{} + _ CDCSyncConnector = &connelasticsearch.ElasticsearchConnector{} _ CDCSyncPgConnector = &connpostgres.PostgresConnector{} diff --git a/flow/connectors/postgres/cdc.go b/flow/connectors/postgres/cdc.go index 4adcc1d0d4..ae1e84292e 100644 --- a/flow/connectors/postgres/cdc.go +++ b/flow/connectors/postgres/cdc.go @@ -2,10 +2,8 @@ package connpostgres import ( "context" - "crypto/sha256" "fmt" "log/slog" - "slices" "time" "github.com/jackc/pglogrepl" @@ -488,7 +486,7 @@ func PullCdcRecords[Items model.Items]( return err } } else { - tablePkeyVal, err := recToTablePKey(req, rec) + tablePkeyVal, err := model.RecToTablePKey[Items](req.TableNameSchemaMapping, rec) if err != nil { return err } @@ -520,7 +518,7 @@ func PullCdcRecords[Items model.Items]( return err } } else { - tablePkeyVal, err := recToTablePKey(req, rec) + tablePkeyVal, err := model.RecToTablePKey[Items](req.TableNameSchemaMapping, rec) if err != nil { return err } @@ -538,7 +536,7 @@ func PullCdcRecords[Items model.Items]( return err } } else { - tablePkeyVal, err := recToTablePKey(req, rec) + tablePkeyVal, err := model.RecToTablePKey[Items](req.TableNameSchemaMapping, rec) if err != nil { return err } @@ -865,27 +863,6 @@ func processRelationMessage[Items model.Items]( return nil, nil } -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 { - pkeyColBytes, err := rec.GetItems().GetBytesByColName(pkeyCol) - if err != nil { - return model.TableWithPkey{}, fmt.Errorf("error getting pkey column value: %w", err) - } - pkeyColsMerged = append(pkeyColsMerged, pkeyColBytes) - } - - return model.TableWithPkey{ - TableName: tableName, - PkeyColVal: sha256.Sum256(slices.Concat(pkeyColsMerged...)), - }, nil -} - func (p *PostgresCDCSource) getParentRelIDIfPartitioned(relID uint32) uint32 { parentRelID, ok := p.childToParentRelIDMapping[relID] if ok { diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index e88a736428..a20291b04d 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -706,7 +706,7 @@ func (c *PostgresConnector) CreateRawTable(ctx context.Context, req *protos.Crea _, err = createRawTableTx.Exec(ctx, fmt.Sprintf(createRawTableDstTableIndexSQL, rawTableIdentifier, c.metadataSchema, rawTableIdentifier)) if err != nil { - return nil, fmt.Errorf("error creating destion table index on raw table: %w", err) + return nil, fmt.Errorf("error creating destination table index on raw table: %w", err) } err = createRawTableTx.Commit(ctx) diff --git a/flow/e2e/elasticsearch/elasticsearch.go b/flow/e2e/elasticsearch/elasticsearch.go index d0abea9413..ed3389adb7 100644 --- a/flow/e2e/elasticsearch/elasticsearch.go +++ b/flow/e2e/elasticsearch/elasticsearch.go @@ -80,8 +80,12 @@ func (s elasticsearchSuite) Peer() *protos.Peer { return s.peer } -func (s elasticsearchSuite) CountDocumentsInIndex(index string) int64 { +func (s elasticsearchSuite) countDocumentsInIndex(index string) int64 { res, err := s.esClient.Count().Index(index).Do(context.Background()) + // index may not exist yet, don't error out for that + if err != nil && strings.Contains(err.Error(), "index_not_found_exception") { + return 0 + } require.NoError(s.t, err, "failed to get count of documents in index") return res.Count } diff --git a/flow/e2e/elasticsearch/peer_flow_es_test.go b/flow/e2e/elasticsearch/peer_flow_es_test.go new file mode 100644 index 0000000000..04e1e6d6ab --- /dev/null +++ b/flow/e2e/elasticsearch/peer_flow_es_test.go @@ -0,0 +1,150 @@ +package e2e_elasticsearch + +import ( + "context" + "fmt" + "time" + + "github.com/stretchr/testify/require" + + "github.com/PeerDB-io/peer-flow/e2e" + peerflow "github.com/PeerDB-io/peer-flow/workflows" +) + +func (s elasticsearchSuite) Test_Simple_PKey_CDC_Mirror() { + srcTableName := e2e.AttachSchema(s, "es_simple_pkey_cdc") + + _, err := s.conn.Conn().Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id INT PRIMARY KEY GENERATED ALWAYS AS IDENTITY, + c1 INT, + val TEXT, + updated_at TIMESTAMP DEFAULT now() + ); + `, srcTableName)) + require.NoError(s.t, err, "failed creating table") + + tc := e2e.NewTemporalClient(s.t) + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: e2e.AddSuffix(s, "es_simple_pkey_cdc"), + TableNameMapping: map[string]string{srcTableName: srcTableName}, + Destination: s.peer, + } + flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() + flowConnConfig.MaxBatchSize = 100 + flowConnConfig.DoInitialSnapshot = true + + rowCount := 10 + for i := range rowCount { + _, err := s.conn.Conn().Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1,val) VALUES(%d,'val%d') + `, srcTableName, i, i)) + require.NoError(s.t, err, "failed to insert row") + } + + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + + for i := range rowCount { + _, err := s.conn.Conn().Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1,val) VALUES(%d,'val%d') + `, srcTableName, i, i)) + require.NoError(s.t, err, "failed to insert row") + } + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "wait for initial snapshot + inserted rows", func() bool { + return s.countDocumentsInIndex(srcTableName) == int64(2*rowCount) + }) + + _, err = s.conn.Conn().Exec(context.Background(), fmt.Sprintf(` + UPDATE %s SET c1=c1+2,updated_at=now() WHERE id%%2=0;`, srcTableName)) + require.NoError(s.t, err, "failed to update rows on source") + for i := range rowCount { + _, err := s.conn.Conn().Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1,val) VALUES(%d,'val%d') + `, srcTableName, i, i)) + require.NoError(s.t, err, "failed to insert row") + } + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "wait for updates + new inserts", func() bool { + return s.countDocumentsInIndex(srcTableName) == int64(3*rowCount) + }) + + _, err = s.conn.Conn().Exec(context.Background(), fmt.Sprintf(` + DELETE FROM %s WHERE id%%2=1;`, srcTableName)) + require.NoError(s.t, err, "failed to delete rows on source") + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "wait for deletes", func() bool { + return s.countDocumentsInIndex(srcTableName) == int64(3*rowCount/2) + }) + + env.Cancel() + e2e.RequireEnvCanceled(s.t, env) +} + +func (s elasticsearchSuite) Test_Composite_PKey_CDC_Mirror() { + srcTableName := e2e.AttachSchema(s, "es_composite_pkey_cdc") + + _, err := s.conn.Conn().Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id INT GENERATED ALWAYS AS IDENTITY, + c1 INT, + val TEXT, + updated_at TIMESTAMP DEFAULT now(), + PRIMARY KEY(id,val) + ); + `, srcTableName)) + require.NoError(s.t, err, "failed creating table") + + tc := e2e.NewTemporalClient(s.t) + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: e2e.AddSuffix(s, "es_composite_pkey_cdc"), + TableNameMapping: map[string]string{srcTableName: srcTableName}, + Destination: s.peer, + } + flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() + flowConnConfig.MaxBatchSize = 100 + flowConnConfig.DoInitialSnapshot = true + + rowCount := 10 + for i := range rowCount { + _, err := s.conn.Conn().Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1,val) VALUES(%d,'val%d') + `, srcTableName, i, i)) + require.NoError(s.t, err, "failed to insert row") + } + + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + + for i := range rowCount { + _, err := s.conn.Conn().Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1,val) VALUES(%d,'val%d') + `, srcTableName, i, i)) + require.NoError(s.t, err, "failed to insert row") + } + + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "wait for initial snapshot + inserted rows", func() bool { + return s.countDocumentsInIndex(srcTableName) == int64(2*rowCount) + }) + + _, err = s.conn.Conn().Exec(context.Background(), fmt.Sprintf(` + UPDATE %s SET c1=c1+2,updated_at=now() WHERE id%%2=0;`, srcTableName)) + require.NoError(s.t, err, "failed to update rows on source") + for i := range rowCount { + _, err := s.conn.Conn().Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1,val) VALUES(%d,'val%d') + `, srcTableName, i, i)) + require.NoError(s.t, err, "failed to insert row") + } + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "wait for updates + new inserts", func() bool { + return s.countDocumentsInIndex(srcTableName) == int64(3*rowCount) + }) + + _, err = s.conn.Conn().Exec(context.Background(), fmt.Sprintf(` + DELETE FROM %s WHERE id%%2=1;`, srcTableName)) + require.NoError(s.t, err, "failed to delete rows on source") + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "wait for deletes", func() bool { + return s.countDocumentsInIndex(srcTableName) == int64(3*rowCount/2) + }) + + env.Cancel() + e2e.RequireEnvCanceled(s.t, env) +} diff --git a/flow/e2e/elasticsearch/qrep_flow_es_test.go b/flow/e2e/elasticsearch/qrep_flow_es_test.go index dafa1b2d79..85ac99043e 100644 --- a/flow/e2e/elasticsearch/qrep_flow_es_test.go +++ b/flow/e2e/elasticsearch/qrep_flow_es_test.go @@ -10,18 +10,20 @@ import ( "github.com/PeerDB-io/peer-flow/e2e" "github.com/PeerDB-io/peer-flow/e2eshared" + "github.com/PeerDB-io/peer-flow/generated/protos" ) func Test_Elasticsearch(t *testing.T) { e2eshared.RunSuite(t, SetupSuite) } -func (s elasticsearchSuite) Test_Simple_Qrep() { - srcTableName := e2e.AttachSchema(s, "es_simple") +func (s elasticsearchSuite) Test_Simple_QRep_Append() { + srcTableName := e2e.AttachSchema(s, "es_simple_append") _, err := s.conn.Conn().Exec(context.Background(), fmt.Sprintf(` CREATE TABLE IF NOT EXISTS %s ( id INT PRIMARY KEY GENERATED ALWAYS AS IDENTITY, + c1 INT, val TEXT, updated_at TIMESTAMP DEFAULT now() ); @@ -31,8 +33,8 @@ func (s elasticsearchSuite) Test_Simple_Qrep() { rowCount := 10 for i := range rowCount { _, err := s.conn.Conn().Exec(context.Background(), fmt.Sprintf(` - INSERT INTO %s(val) VALUES('val%d') - `, srcTableName, i)) + INSERT INTO %s(c1,val) VALUES(%d,'val%d') + `, srcTableName, i, i)) require.NoError(s.t, err, "failed to insert row") } @@ -51,10 +53,72 @@ func (s elasticsearchSuite) Test_Simple_Qrep() { "", "", ) + qrepConfig.InitialCopyOnly = false + require.NoError(s.t, err) env := e2e.RunQRepFlowWorkflow(tc, qrepConfig) - e2e.EnvWaitForFinished(s.t, env, 3*time.Minute) + + e2e.EnvWaitFor(s.t, env, 10*time.Second, "waiting for ES to catch up", func() bool { + return s.countDocumentsInIndex(srcTableName) == int64(rowCount) + }) + _, err = s.conn.Conn().Exec(context.Background(), fmt.Sprintf(` + UPDATE %s SET c1=c1+2,updated_at=now() WHERE id%%2=0;`, srcTableName)) + require.NoError(s.t, err, "failed to update rows on source") + e2e.EnvWaitFor(s.t, env, 20*time.Second, "waiting for ES to catch up", func() bool { + return s.countDocumentsInIndex(srcTableName) == int64(3*rowCount/2) + }) + require.NoError(s.t, env.Error()) +} + +func (s elasticsearchSuite) Test_Simple_QRep_Upsert() { + srcTableName := e2e.AttachSchema(s, "es_simple_upsert") + + _, err := s.conn.Conn().Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id INT PRIMARY KEY GENERATED ALWAYS AS IDENTITY, + c1 INT, + val TEXT, + updated_at TIMESTAMP DEFAULT now() + ); + `, srcTableName)) + require.NoError(s.t, err, "failed creating table") + + rowCount := 10 + for i := range rowCount { + _, err := s.conn.Conn().Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1,val) VALUES(%d,'val%d') + `, srcTableName, i, i)) + require.NoError(s.t, err, "failed to insert row") + } + + tc := e2e.NewTemporalClient(s.t) + + query := fmt.Sprintf("SELECT * FROM %s WHERE updated_at BETWEEN {{.start}} AND {{.end}}", + srcTableName) - require.EqualValues(s.t, rowCount, s.CountDocumentsInIndex(srcTableName)) + qrepConfig, err := e2e.CreateQRepWorkflowConfig("test_es_simple_qrep", + srcTableName, + srcTableName, + query, + s.peer, + "", + false, + "", + "", + ) + qrepConfig.WriteMode = &protos.QRepWriteMode{ + WriteType: protos.QRepWriteType_QREP_WRITE_MODE_UPSERT, + UpsertKeyColumns: []string{"id"}, + } + qrepConfig.InitialCopyOnly = false + + require.NoError(s.t, err) + env := e2e.RunQRepFlowWorkflow(tc, qrepConfig) + + e2e.EnvWaitFor(s.t, env, 10*time.Second, "waiting for ES to catch up", func() bool { + return s.countDocumentsInIndex(srcTableName) == int64(rowCount) + }) + + require.NoError(s.t, env.Error()) } diff --git a/flow/model/model.go b/flow/model/model.go index 54c2ef41ce..acdcdc5176 100644 --- a/flow/model/model.go +++ b/flow/model/model.go @@ -1,6 +1,8 @@ package model import ( + "crypto/sha256" + "fmt" "sync/atomic" "time" @@ -75,12 +77,36 @@ type TableWithPkey struct { PkeyColVal [32]byte } +func RecToTablePKey[T Items]( + tableNameSchemaMapping map[string]*protos.TableSchema, + rec Record[T], +) (TableWithPkey, error) { + tableName := rec.GetDestinationTableName() + hasher := sha256.New() + + for _, pkeyCol := range tableNameSchemaMapping[tableName].PrimaryKeyColumns { + pkeyColBytes, err := rec.GetItems().GetBytesByColName(pkeyCol) + if err != nil { + return TableWithPkey{}, fmt.Errorf("error getting pkey column value: %w", err) + } + // cannot return an error + _, _ = hasher.Write(pkeyColBytes) + } + + return TableWithPkey{ + TableName: tableName, + PkeyColVal: [32]byte(hasher.Sum(nil)), + }, nil +} + 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 + // destination table name -> schema mapping + TableNameSchemaMapping map[string]*protos.TableSchema + FlowJobName string // Staging path for AVRO files in CDC StagingPath string // Lua script diff --git a/flow/workflows/snapshot_flow.go b/flow/workflows/snapshot_flow.go index f1c8c7d7f3..fdec9d15c6 100644 --- a/flow/workflows/snapshot_flow.go +++ b/flow/workflows/snapshot_flow.go @@ -171,6 +171,18 @@ func (s *SnapshotFlowExecution) cloneTable( numRowsPerPartition = s.config.SnapshotNumRowsPerPartition } + snapshotWriteMode := &protos.QRepWriteMode{ + WriteType: protos.QRepWriteType_QREP_WRITE_MODE_APPEND, + } + // ensure document IDs are synchronized across initial load and CDC + // for the same document + if s.config.Destination.Type == protos.DBType_ELASTICSEARCH { + snapshotWriteMode = &protos.QRepWriteMode{ + WriteType: protos.QRepWriteType_QREP_WRITE_MODE_UPSERT, + UpsertKeyColumns: s.tableNameSchemaMapping[mapping.DestinationTableIdentifier].PrimaryKeyColumns, + } + } + config := &protos.QRepConfig{ FlowJobName: childWorkflowID, SourcePeer: sourcePostgres, @@ -185,10 +197,8 @@ func (s *SnapshotFlowExecution) cloneTable( StagingPath: s.config.SnapshotStagingPath, SyncedAtColName: s.config.SyncedAtColName, SoftDeleteColName: s.config.SoftDeleteColName, - WriteMode: &protos.QRepWriteMode{ - WriteType: protos.QRepWriteType_QREP_WRITE_MODE_APPEND, - }, - System: s.config.System, + WriteMode: snapshotWriteMode, + System: s.config.System, } state := NewQRepFlowState() From 7866e7919cf717cad4fa85768000437adbb1925d Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Tue, 30 Apr 2024 10:33:50 +0530 Subject: [PATCH 02/41] Telemetry for peer: only televise non nil validate peer err (#1653) Log non flow error only if not nil for validate peer --- flow/cmd/validate_peer.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/flow/cmd/validate_peer.go b/flow/cmd/validate_peer.go index 647c86c07a..bae2a1f8b6 100644 --- a/flow/cmd/validate_peer.go +++ b/flow/cmd/validate_peer.go @@ -61,11 +61,11 @@ func (h *FlowRequestHandler) ValidatePeer( validationConn, ok := conn.(connectors.ValidationConnector) if ok { validErr := validationConn.ValidateCheck(ctx) - displayErr := fmt.Sprintf("failed to validate peer %s: %v", req.Peer.Name, validErr) - h.alerter.LogNonFlowWarning(ctx, telemetry.CreatePeer, req.Peer.Name, - displayErr, - ) if validErr != nil { + displayErr := fmt.Sprintf("failed to validate peer %s: %v", req.Peer.Name, validErr) + h.alerter.LogNonFlowWarning(ctx, telemetry.CreatePeer, req.Peer.Name, + displayErr, + ) return &protos.ValidatePeerResponse{ Status: protos.ValidatePeerStatus_INVALID, Message: displayErr, From 3dab1b34ea86ea02575cf07bc334285e679454e5 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Tue, 30 Apr 2024 13:37:08 +0530 Subject: [PATCH 03/41] UI: check for start and end existence for qrep (#1652) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit For append and upsert modes in qrep, we need start and end placeholders for partitioning. this PR adds validation and explanation text for this Screenshot 2024-04-29 at 4 32 30 PM --------- Co-authored-by: Kevin Biju <52661649+heavycrystal@users.noreply.github.com> --- ui/app/mirrors/create/handlers.ts | 11 +++++++++++ ui/app/mirrors/create/page.tsx | 5 +++++ 2 files changed, 16 insertions(+) diff --git a/ui/app/mirrors/create/handlers.ts b/ui/app/mirrors/create/handlers.ts index 1d76c3196e..6b808fedbc 100644 --- a/ui/app/mirrors/create/handlers.ts +++ b/ui/app/mirrors/create/handlers.ts @@ -211,6 +211,17 @@ export const handleCreateQRep = async ( return; } + if ( + !xmin && + config.writeMode?.writeType != QRepWriteType.QREP_WRITE_MODE_OVERWRITE && + !(query.includes('{{.start}}') && query.includes('{{.end}}')) + ) { + notifyErr( + 'Please include placeholders {{.start}} and {{.end}} in the query' + ); + return; + } + if (xmin == true) { config.watermarkColumn = 'xmin'; config.query = `SELECT * FROM ${quotedWatermarkTable( diff --git a/ui/app/mirrors/create/page.tsx b/ui/app/mirrors/create/page.tsx index ed70c33b1e..2b0f59d33e 100644 --- a/ui/app/mirrors/create/page.tsx +++ b/ui/app/mirrors/create/page.tsx @@ -191,6 +191,11 @@ export default function CreateMirrors() { Write a query whose results will be replicated to a target table.

+ For append and upsert modes, make sure the query{' '} + includes the start and end placeholders in the query. + PeerDB uses these placeholders for partitioning query results + for performance. +

In most cases, you will require a watermark table and a watermark column in that table.

From 13921fbc84db17e6dcd8c8c7ad1d7f73d897780c Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Tue, 30 Apr 2024 17:27:00 +0530 Subject: [PATCH 04/41] Fix Clickhouse QRep Append (#1656) Gets CH QRep append working by skipping the sign column for the final insert into select --- flow/connectors/clickhouse/qrep_avro_sync.go | 1 + 1 file changed, 1 insertion(+) diff --git a/flow/connectors/clickhouse/qrep_avro_sync.go b/flow/connectors/clickhouse/qrep_avro_sync.go index f7c6cd014d..a0a9fb9a71 100644 --- a/flow/connectors/clickhouse/qrep_avro_sync.go +++ b/flow/connectors/clickhouse/qrep_avro_sync.go @@ -128,6 +128,7 @@ func (s *ClickhouseAvroSyncMethod) SyncQRepRecords( for _, col := range dstTableSchema { colName := col.Name() if strings.EqualFold(colName, config.SoftDeleteColName) || + strings.EqualFold(colName, signColName) || strings.EqualFold(colName, config.SyncedAtColName) || strings.EqualFold(colName, versionColName) { continue From 069a02d7209e95b68a7e2ffc89419352f36e8600 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Tue, 30 Apr 2024 17:52:35 +0530 Subject: [PATCH 05/41] Add peerdb columns to ui, fix for BQ (#1654) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - Add fields for synced_at and soft delete column names in UI under advanced section. - These are visible for BQ and SF destinations for now - Fix default soft delete false for BQ CDC Screenshot 2024-04-29 at 10 01 13 PM Functionally tested --- flow/connectors/bigquery/bigquery.go | 7 ++++--- flow/e2e/bigquery/peer_flow_bq_test.go | 2 +- ui/app/mirrors/create/cdc/cdc.tsx | 5 ++++- ui/app/mirrors/create/helpers/cdc.ts | 20 ++++++++++++++++++++ 4 files changed, 29 insertions(+), 5 deletions(-) diff --git a/flow/connectors/bigquery/bigquery.go b/flow/connectors/bigquery/bigquery.go index 7c9d66ab93..40682a5f75 100644 --- a/flow/connectors/bigquery/bigquery.go +++ b/flow/connectors/bigquery/bigquery.go @@ -639,9 +639,10 @@ func (c *BigQueryConnector) SetupNormalizedTable( if softDeleteColName != "" { columns = append(columns, &bigquery.FieldSchema{ - Name: softDeleteColName, - Type: bigquery.BooleanFieldType, - Repeated: false, + Name: softDeleteColName, + Type: bigquery.BooleanFieldType, + Repeated: false, + DefaultValueExpression: "false", }) } diff --git a/flow/e2e/bigquery/peer_flow_bq_test.go b/flow/e2e/bigquery/peer_flow_bq_test.go index 206f71ea27..da4f036b0a 100644 --- a/flow/e2e/bigquery/peer_flow_bq_test.go +++ b/flow/e2e/bigquery/peer_flow_bq_test.go @@ -1231,7 +1231,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Soft_Delete_UD_Same_Batch() { s.bqHelper.Config.DatasetId, dstName) numNewRows, err := s.bqHelper.RunInt64Query(newerSyncedAtQuery) require.NoError(s.t, err) - require.Equal(s.t, int64(0), numNewRows) + require.Equal(s.t, int64(1), numNewRows) } func (s PeerFlowE2ETestSuiteBQ) Test_Soft_Delete_Insert_After_Delete() { diff --git a/ui/app/mirrors/create/cdc/cdc.tsx b/ui/app/mirrors/create/cdc/cdc.tsx index dc34c1dfd9..b4581b9cf0 100644 --- a/ui/app/mirrors/create/cdc/cdc.tsx +++ b/ui/app/mirrors/create/cdc/cdc.tsx @@ -70,7 +70,10 @@ export default function CDCConfigForm({ label.includes('snapshot'))) || ((mirrorConfig.source?.type !== DBType.POSTGRES || mirrorConfig.destination?.type !== DBType.POSTGRES) && - label.includes('type system')) + label.includes('type system')) || + (mirrorConfig.destination?.type !== DBType.BIGQUERY && + mirrorConfig.destination?.type !== DBType.SNOWFLAKE && + label.includes('column name')) ) { return false; } diff --git a/ui/app/mirrors/create/helpers/cdc.ts b/ui/app/mirrors/create/helpers/cdc.ts index c18f483f30..1d190f78a7 100644 --- a/ui/app/mirrors/create/helpers/cdc.ts +++ b/ui/app/mirrors/create/helpers/cdc.ts @@ -162,4 +162,24 @@ export const cdcSettings: MirrorSetting[] = [ tips: 'Decide if PeerDB should use native Postgres types directly', advanced: true, }, + { + label: 'Synced-At Column Name', + stateHandler: (value, setter) => + setter((curr: CDCConfig) => ({ + ...curr, + syncedAtColName: value as string | '', + })), + tips: 'A field to set the name of PeerDBs synced_at column. If not set, a default name will be set', + advanced: true, + }, + { + label: 'Soft Delete Column Name', + stateHandler: (value, setter) => + setter((curr: CDCConfig) => ({ + ...curr, + softDeleteColName: value as string | '', + })), + tips: 'A field to set the name of PeerDBs soft delete column.', + advanced: true, + }, ]; From aad03d4f09b3a5c8665bf22913d583f349a56a9a Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Tue, 30 Apr 2024 19:35:04 +0530 Subject: [PATCH 06/41] UI: Fix kafka zod (#1657) - Allows for empty partition field - Adds some error messaging for the zod schema --- ui/app/peers/create/[peerType]/schema.ts | 44 ++++++++++++++++-------- 1 file changed, 30 insertions(+), 14 deletions(-) diff --git a/ui/app/peers/create/[peerType]/schema.ts b/ui/app/peers/create/[peerType]/schema.ts index ab0e2e73af..498613edb4 100644 --- a/ui/app/peers/create/[peerType]/schema.ts +++ b/ui/app/peers/create/[peerType]/schema.ts @@ -288,26 +288,42 @@ export const chSchema = z.object({ export const kaSchema = z.object({ servers: z - .array(z.string({ required_error: 'Server address must not be empty' })) + .array( + z.string({ + invalid_type_error: 'Invalid server provided', + required_error: 'Server address must not be empty', + }) + ) .min(1, { message: 'At least 1 server required' }), username: z.string().optional(), password: z.string().optional(), sasl: z - .union([ - z.literal('PLAIN'), - z.literal('SCRAM-SHA-256'), - z.literal('SCRAM-SHA-512'), - ]) + .union( + [ + z.literal('PLAIN'), + z.literal('SCRAM-SHA-256'), + z.literal('SCRAM-SHA-512'), + ], + { errorMap: (issue, ctx) => ({ message: 'Invalid SASL mechanism' }) } + ) .optional(), partitioner: z - .union([ - z.literal('Default'), - z.literal('LeastBackup'), - z.literal('Manual'), - z.literal('RoundRobin'), - z.literal('StickyKey'), - z.literal('Sticky'), - ]) + .union( + [ + z.literal('Default'), + z.literal('LeastBackup'), + z.literal('Manual'), + z.literal('RoundRobin'), + z.literal('StickyKey'), + z.literal('Sticky'), + z.literal(''), + ], + { + errorMap: (issue, ctx) => ({ + message: 'Invalid partitioning mechanism', + }), + } + ) .optional(), disableTls: z.boolean().optional(), }); From 67d7718a497bfc82ac1ddd858b0ba14b602c1ae1 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Wed, 1 May 2024 01:33:15 +0530 Subject: [PATCH 07/41] Add heartbeat for gettableschema (#1658) GetTableSchema of SetupFlow can take a long time for a large number of tables, leading to activity heartbeat timeouts. This PR adds a heartbeat after fetching schema for each table --- flow/connectors/postgres/postgres.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index a20291b04d..2c5b506b6b 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -15,6 +15,7 @@ import ( "github.com/jackc/pgx/v5" "github.com/jackc/pgx/v5/pgtype" "github.com/jackc/pgx/v5/pgxpool" + "go.temporal.io/sdk/activity" "go.temporal.io/sdk/log" "go.temporal.io/sdk/temporal" @@ -723,6 +724,7 @@ func (c *PostgresConnector) GetTableSchema( ) (*protos.GetTableSchemaBatchOutput, error) { res := make(map[string]*protos.TableSchema) for _, tableName := range req.TableIdentifiers { + activity.RecordHeartbeat(ctx, fmt.Sprintf("fetching schema for table %s", tableName)) tableSchema, err := c.getTableSchemaForTable(ctx, tableName, req.System) if err != nil { return nil, err From f7c7ac2da15e9d191e84d9b89b937152a0a54fd0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Wed, 1 May 2024 13:40:18 +0000 Subject: [PATCH 08/41] Fix CI regression, #1658 merged too soon (#1659) Also need to use `activity.IsActivity` outside activity specific code --- flow/connectors/postgres/postgres.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index 2c5b506b6b..1f5a7545f4 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -724,7 +724,9 @@ func (c *PostgresConnector) GetTableSchema( ) (*protos.GetTableSchemaBatchOutput, error) { res := make(map[string]*protos.TableSchema) for _, tableName := range req.TableIdentifiers { - activity.RecordHeartbeat(ctx, fmt.Sprintf("fetching schema for table %s", tableName)) + if activity.IsActivity(ctx) { + activity.RecordHeartbeat(ctx, "fetching schema for table "+tableName) + } tableSchema, err := c.getTableSchemaForTable(ctx, tableName, req.System) if err != nil { return nil, err From e301f7880e38eaf966a109b0ff5e09a7ea57e16b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Wed, 1 May 2024 13:48:44 +0000 Subject: [PATCH 09/41] MySQL peer (#1637) Currently testing with [mariadb](https://hub.docker.com/_/mariadb) ``` CREATE PEER my FROM MYSQL WITH ( host = 'mysql', port = '3306', user = 'root', password = 'example', database = 'main', compression = 1, disable_tls = true ); ``` The scope of this PR is only peer creation & nexus proxy. No mirror support Co-authored-by: Kaushik Iska --- flow/connectors/core.go | 7 +- flow/connectors/mysql/mysql.go | 15 + nexus/Cargo.lock | 540 +++++++++++++++--- nexus/Cargo.toml | 18 +- nexus/analyzer/src/lib.rs | 167 ++++-- nexus/analyzer/src/qrep.rs | 12 +- nexus/catalog/src/lib.rs | 9 +- nexus/flow-rs/Cargo.toml | 2 +- nexus/parser/Cargo.toml | 2 +- nexus/parser/src/lib.rs | 42 +- nexus/peer-ast/Cargo.toml | 12 + nexus/peer-ast/src/lib.rs | 101 ++++ nexus/peer-bigquery/Cargo.toml | 9 +- nexus/peer-bigquery/src/ast.rs | 122 +--- nexus/peer-bigquery/src/lib.rs | 98 ++-- nexus/peer-bigquery/src/stream.rs | 2 +- nexus/peer-connections/Cargo.toml | 2 +- nexus/peer-cursor/Cargo.toml | 2 + nexus/peer-cursor/src/lib.rs | 9 + .../cursor.rs => peer-cursor/src/manager.rs} | 61 +- nexus/peer-mysql/Cargo.toml | 27 + nexus/peer-mysql/src/ast.rs | 63 ++ nexus/peer-mysql/src/client.rs | 53 ++ nexus/peer-mysql/src/lib.rs | 232 ++++++++ nexus/peer-mysql/src/stream.rs | 189 ++++++ nexus/peer-postgres/Cargo.toml | 4 +- nexus/peer-snowflake/Cargo.toml | 3 +- nexus/peer-snowflake/src/ast.rs | 3 +- nexus/peer-snowflake/src/cursor.rs | 124 ---- nexus/peer-snowflake/src/lib.rs | 98 ++-- nexus/postgres-connection/Cargo.toml | 2 +- nexus/pt/src/lib.rs | 1 + nexus/server/Cargo.toml | 7 +- nexus/server/src/main.rs | 221 +++---- .../server/tests/results/expected/bq.sql.out | 2 +- .../tests/results/expected/postgres.sql.out | 14 +- nexus/server/tests/server_test.rs | 71 +-- nexus/value/Cargo.toml | 2 +- protos/peers.proto | 14 +- ui/app/api/peers/getTruePeer.ts | 10 +- 40 files changed, 1651 insertions(+), 721 deletions(-) create mode 100644 flow/connectors/mysql/mysql.go create mode 100644 nexus/peer-ast/Cargo.toml create mode 100644 nexus/peer-ast/src/lib.rs rename nexus/{peer-bigquery/src/cursor.rs => peer-cursor/src/manager.rs} (56%) create mode 100644 nexus/peer-mysql/Cargo.toml create mode 100644 nexus/peer-mysql/src/ast.rs create mode 100644 nexus/peer-mysql/src/client.rs create mode 100644 nexus/peer-mysql/src/lib.rs create mode 100644 nexus/peer-mysql/src/stream.rs delete mode 100644 nexus/peer-snowflake/src/cursor.rs diff --git a/flow/connectors/core.go b/flow/connectors/core.go index 545269c4d1..9a9a42f1a3 100644 --- a/flow/connectors/core.go +++ b/flow/connectors/core.go @@ -13,6 +13,7 @@ import ( 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" + connmysql "github.com/PeerDB-io/peer-flow/connectors/mysql" connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" connpubsub "github.com/PeerDB-io/peer-flow/connectors/pubsub" conns3 "github.com/PeerDB-io/peer-flow/connectors/s3" @@ -224,14 +225,14 @@ func GetConnector(ctx context.Context, config *protos.Peer) (Connector, error) { return connbigquery.NewBigQueryConnector(ctx, inner.BigqueryConfig) case *protos.Peer_SnowflakeConfig: return connsnowflake.NewSnowflakeConnector(ctx, inner.SnowflakeConfig) - case *protos.Peer_EventhubConfig: - return nil, errors.New("use eventhub group config instead") case *protos.Peer_EventhubGroupConfig: return conneventhub.NewEventHubConnector(ctx, inner.EventhubGroupConfig) case *protos.Peer_S3Config: return conns3.NewS3Connector(ctx, inner.S3Config) case *protos.Peer_SqlserverConfig: return connsqlserver.NewSQLServerConnector(ctx, inner.SqlserverConfig) + case *protos.Peer_MysqlConfig: + return connmysql.MySqlConnector{}, nil case *protos.Peer_ClickhouseConfig: return connclickhouse.NewClickhouseConnector(ctx, inner.ClickhouseConfig) case *protos.Peer_KafkaConfig: @@ -341,4 +342,6 @@ var ( _ ValidationConnector = &connclickhouse.ClickhouseConnector{} _ ValidationConnector = &connbigquery.BigQueryConnector{} _ ValidationConnector = &conns3.S3Connector{} + + _ Connector = &connmysql.MySqlConnector{} ) diff --git a/flow/connectors/mysql/mysql.go b/flow/connectors/mysql/mysql.go new file mode 100644 index 0000000000..d898826e1f --- /dev/null +++ b/flow/connectors/mysql/mysql.go @@ -0,0 +1,15 @@ +// stub to bypass validation + +package mysql + +import "context" + +type MySqlConnector struct{} + +func (MySqlConnector) Close() error { + return nil +} + +func (MySqlConnector) ConnectionActive(context.Context) error { + return nil +} diff --git a/nexus/Cargo.lock b/nexus/Cargo.lock index e53188476f..7d9188ff6d 100644 --- a/nexus/Cargo.lock +++ b/nexus/Cargo.lock @@ -39,6 +39,18 @@ dependencies = [ "version_check", ] +[[package]] +name = "ahash" +version = "0.8.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e89da841a80418a9b391ebaea17f5c112ffaaa96f621d2c285b5174da76b9011" +dependencies = [ + "cfg-if", + "once_cell", + "version_check", + "zerocopy", +] + [[package]] name = "aho-corasick" version = "1.1.3" @@ -48,6 +60,12 @@ dependencies = [ "memchr", ] +[[package]] +name = "allocator-api2" +version = "0.2.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5c6cb57a04249c6480766f7f7cef5467412af1490f8d1e243141daddada3264f" + [[package]] name = "analyzer" version = "0.1.0" @@ -151,9 +169,9 @@ checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" [[package]] name = "async-compression" -version = "0.4.8" +version = "0.4.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07dbbf24db18d609b1462965249abdf49129ccad073ec257da372adc83259c60" +checksum = "4e9eabd7a98fe442131a17c316bd9349c43695e49e730c3c8e12cfb5f4da2693" dependencies = [ "flate2", "futures-core", @@ -164,9 +182,9 @@ dependencies = [ [[package]] name = "async-recursion" -version = "1.1.0" +version = "1.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "30c5ef0ede93efbf733c1a727f3b6b5a1060bbedd5600183e66f6e4be4af0ec5" +checksum = "3b43422f69d8ff38f95f1b2bb76517c91589a924d1559a0e935d7c8ce0274c11" dependencies = [ "proc-macro2", "quote", @@ -300,6 +318,26 @@ dependencies = [ "smallvec", ] +[[package]] +name = "bindgen" +version = "0.69.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a00dc851838a2120612785d195287475a3ac45514741da670b735818822129a0" +dependencies = [ + "bitflags 2.5.0", + "cexpr", + "clang-sys", + "itertools 0.12.1", + "lazy_static", + "lazycell", + "proc-macro2", + "quote", + "regex", + "rustc-hash", + "shlex", + "syn 2.0.60", +] + [[package]] name = "bitflags" version = "1.3.2" @@ -366,6 +404,15 @@ dependencies = [ "syn_derive", ] +[[package]] +name = "btoi" +version = "0.4.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9dd6407f73a9b8b6162d8a2ef999fe6afd7cc15902ebf42c5cd296addf17e0ad" +dependencies = [ + "num-traits", +] + [[package]] name = "bumpalo" version = "3.16.0" @@ -408,9 +455,9 @@ checksum = "514de17de45fdb8dc022b1a7975556c53c86f9f0aa5f534b98977b171857c2c9" [[package]] name = "cargo-deb" -version = "2.1.0" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8978c203da69c3bd93d0e676477d9f4cf1df2a62e14fc4a68665259f94c6c0e2" +checksum = "769d6bfa0f70c0e584f46d6bfc78d094f708fa249f2b7b524ab6bed62eb3df36" dependencies = [ "ar", "cargo_toml", @@ -435,9 +482,9 @@ dependencies = [ [[package]] name = "cargo_toml" -version = "0.19.2" +version = "0.20.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a98356df42a2eb1bd8f1793ae4ee4de48e384dd974ce5eac8eee802edb7492be" +checksum = "c8cb1d556b8b8f36e5ca74938008be3ac102f5dcb5b68a0477e4249ae2291cd3" dependencies = [ "serde", "toml", @@ -477,6 +524,20 @@ name = "cc" version = "1.0.95" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d32a725bc159af97c3e629873bb9f88fb8cf8a4867175f76dc987815ea07c83b" +dependencies = [ + "jobserver", + "libc", + "once_cell", +] + +[[package]] +name = "cexpr" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6fac387a98bb7c37292057cffc56d62ecb629900026402633ae9160df93a8766" +dependencies = [ + "nom", +] [[package]] name = "cfg-if" @@ -515,6 +576,17 @@ dependencies = [ "inout", ] +[[package]] +name = "clang-sys" +version = "1.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "67523a3b4be3ce1989d607a828d036249522dd9c1c8de7f4dd2dae43a37369d1" +dependencies = [ + "glob", + "libc", + "libloading", +] + [[package]] name = "clap" version = "4.5.4" @@ -555,6 +627,15 @@ version = "0.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "98cc8fbded0c607b7ba9dd60cd98df59af97e84d24e49c8557331cfc26d301ce" +[[package]] +name = "cmake" +version = "0.1.50" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a31c789563b815f77f4250caee12365734369f942439b7defd71e18a48197130" +dependencies = [ + "cc", +] + [[package]] name = "colorchoice" version = "1.0.0" @@ -638,6 +719,19 @@ dependencies = [ "cfg-if", ] +[[package]] +name = "crossbeam" +version = "0.8.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1137cd7e7fc0fb5d3c5a8678be38ec56e819125d8d7907411fe24ccb943faca8" +dependencies = [ + "crossbeam-channel", + "crossbeam-deque", + "crossbeam-epoch", + "crossbeam-queue", + "crossbeam-utils", +] + [[package]] name = "crossbeam-channel" version = "0.5.12" @@ -666,6 +760,15 @@ dependencies = [ "crossbeam-utils", ] +[[package]] +name = "crossbeam-queue" +version = "0.3.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "df0346b5d5e76ac2fe4e327c5fd1118d6be7c51dfb18f9b7922923f287471e35" +dependencies = [ + "crossbeam-utils", +] + [[package]] name = "crossbeam-utils" version = "0.8.19" @@ -689,7 +792,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "978747c1d849a7d2ee5e8adc0159961c48fb7e5db2f06af6723b80123bb53856" dependencies = [ "cfg-if", - "hashbrown 0.14.3", + "hashbrown 0.14.5", "lock_api", "once_cell", "parking_lot_core", @@ -844,9 +947,9 @@ checksum = "4443176a9f2c162692bd3d352d745ef9413eec5782a80d8fd6f8a1ac692a07f7" [[package]] name = "fastrand" -version = "2.0.2" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "658bd65b1cf4c852a3cc96f18a8ce7b5640f6b703f905c7d74532294c2a63984" +checksum = "9fc0510504f03c51ada170672ac806f1f105a88aa97a5281117e1ddc3368e51a" [[package]] name = "filetime" @@ -856,7 +959,7 @@ checksum = "1ee447700ac8aa0b2f2bd7bc4462ad686ba06baa6727ac149a2d6277f0d240fd" dependencies = [ "cfg-if", "libc", - "redox_syscall", + "redox_syscall 0.4.1", "windows-sys 0.52.0", ] @@ -874,9 +977,9 @@ checksum = "0ce7134b9999ecaf8bcd65542e436736ef32ddca1b3e06094cb6ec5755203b80" [[package]] name = "flate2" -version = "1.0.28" +version = "1.0.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "46303f565772937ffe1d394a4fac6f411c6013172fadde9dcdb1e147a086940e" +checksum = "5f54427cfd1c7829e2a139fcefea601bf088ebca651d2bf53ebc600eac295dae" dependencies = [ "crc32fast", "miniz_oxide", @@ -1006,15 +1109,14 @@ dependencies = [ [[package]] name = "gcp-bigquery-client" -version = "0.18.1" +version = "0.20.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a7fe3895eb99784b8ad2776688b41e068e918d18ebb736dafb1a321bce46c749" +checksum = "ebc3e5c4b8a072ca074ab0d4f53dc6b04f45eb9bc0cc046a4a1428c8498af71e" dependencies = [ "async-stream", "async-trait", "dyn-clone", - "hyper 0.14.28", - "hyper-rustls 0.25.0", + "hyper 1.3.1", "log", "reqwest", "serde", @@ -1096,14 +1198,18 @@ version = "0.12.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" dependencies = [ - "ahash", + "ahash 0.7.8", ] [[package]] name = "hashbrown" -version = "0.14.3" +version = "0.14.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "290f1a1d9242c78d09ce40a5e87e7554ee637af1351968159f4952f028f75604" +checksum = "e5274423e17b7c9fc20b6e7e208532f9b19825d82dfd615708b70edd83df41f1" +dependencies = [ + "ahash 0.8.11", + "allocator-api2", +] [[package]] name = "hdrhistogram" @@ -1384,7 +1490,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "168fb715dda47215e360912c096649d23d58bf392ac62f73919e831745e40f26" dependencies = [ "equivalent", - "hashbrown 0.14.3", + "hashbrown 0.14.5", ] [[package]] @@ -1427,6 +1533,15 @@ version = "1.0.11" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "49f1f14873335454500d59611f1cf4a4b0f786f9ac11f4312a78e4cf2566695b" +[[package]] +name = "jobserver" +version = "0.1.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d2b099aaa34a9751c5bf0878add70444e1ed2dd73f347be99003d4577277de6e" +dependencies = [ + "libc", +] + [[package]] name = "js-sys" version = "0.3.69" @@ -1451,6 +1566,15 @@ dependencies = [ "simple_asn1", ] +[[package]] +name = "keyed_priority_queue" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4ee7893dab2e44ae5f9d0173f26ff4aa327c10b01b06a72b52dd9405b628640d" +dependencies = [ + "indexmap 2.2.6", +] + [[package]] name = "lazy_static" version = "1.4.0" @@ -1460,11 +1584,27 @@ dependencies = [ "spin 0.5.2", ] +[[package]] +name = "lazycell" +version = "1.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "830d08ce1d1d941e6b30645f1a0eb5643013d835ce3779a5fc208261dbe10f55" + [[package]] name = "libc" -version = "0.2.153" +version = "0.2.154" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c198f91728a82281a64e1f4f9eeb25d82cb32a5de251c6bd1b5154d63a8e7bd" +checksum = "ae743338b92ff9146ce83992f766a31066a91a8c84a45e0e9f21e7cf6de6d346" + +[[package]] +name = "libloading" +version = "0.8.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0c2a198fb6b0eada2a8df47933734e6d35d350665a33a3593d7164fa52c75c19" +dependencies = [ + "cfg-if", + "windows-targets 0.52.5", +] [[package]] name = "libm" @@ -1480,9 +1620,9 @@ checksum = "01cda141df6706de531b6c46c3a33ecca755538219bd484262fa09410c13539c" [[package]] name = "lock_api" -version = "0.4.11" +version = "0.4.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3c168f8615b12bc01f9c17e2eb0cc07dcae1940121185446edc3744920e8ef45" +checksum = "07af8b9cdd281b7915f413fa73f29ebd5d55d0d3f0155584dade1ff18cea1b17" dependencies = [ "autocfg", "scopeguard", @@ -1494,6 +1634,15 @@ version = "0.4.21" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "90ed8c1e510134f979dbc4f070f87d4313098b704861a105fe34231c70a3901c" +[[package]] +name = "lru" +version = "0.12.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d3262e75e648fce39813cb56ac41f3c3e3f65217ebf3844d818d1f9398cfb0dc" +dependencies = [ + "hashbrown 0.14.5", +] + [[package]] name = "lzma-sys" version = "0.1.20" @@ -1570,6 +1719,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a4a650543ca06a924e8b371db273b2756685faae30f8487da1b56505a8f78b0c" dependencies = [ "libc", + "log", "wasi", "windows-sys 0.48.0", ] @@ -1580,6 +1730,78 @@ version = "0.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "defc4c55412d89136f966bbb339008b474350e5e6e78d2714439c386b3137a03" +[[package]] +name = "mysql_async" +version = "0.34.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fbfe87d7e35cb72363326216cc1712b865d8d4f70abf3b2d2e6b251fb6b2f427" +dependencies = [ + "bytes", + "crossbeam", + "flate2", + "futures-core", + "futures-sink", + "futures-util", + "keyed_priority_queue", + "lazy_static", + "lru", + "mio", + "mysql_common", + "once_cell", + "pem", + "percent-encoding", + "pin-project", + "rand", + "rustls 0.22.4", + "rustls-pemfile 2.1.2", + "serde", + "serde_json", + "socket2", + "thiserror", + "tokio", + "tokio-rustls 0.25.0", + "tokio-util", + "twox-hash", + "url", + "webpki", + "webpki-roots", +] + +[[package]] +name = "mysql_common" +version = "0.32.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0ccdc1fe2bb3ef97e07ba4397327ed45509a1e2e499e2f8265243879cbc7313c" +dependencies = [ + "base64 0.21.7", + "bindgen", + "bitflags 2.5.0", + "btoi", + "byteorder", + "bytes", + "cc", + "chrono", + "cmake", + "crc32fast", + "flate2", + "lazy_static", + "num-bigint", + "num-traits", + "rand", + "regex", + "rust_decimal", + "saturating", + "serde", + "serde_json", + "sha1", + "sha2", + "smallvec", + "subprocess", + "thiserror", + "uuid", + "zstd", +] + [[package]] name = "nom" version = "7.1.3" @@ -1712,9 +1934,9 @@ checksum = "b15813163c1d831bf4a13c3610c05c0d03b39feb07f7e09fa234dac9b15aaf39" [[package]] name = "parking_lot" -version = "0.12.1" +version = "0.12.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3742b2c103b9f06bc9fff0a37ff4912935851bee6d36f3c02bcc755bcfec228f" +checksum = "7e4af0ca4f6caed20e900d564c242b8e5d4903fdacf31d3daf527b66fe6f42fb" dependencies = [ "lock_api", "parking_lot_core", @@ -1722,15 +1944,15 @@ dependencies = [ [[package]] name = "parking_lot_core" -version = "0.9.9" +version = "0.9.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4c42a9226546d68acdd9c0a280d17ce19bfe27a46bf68784e4066115788d008e" +checksum = "1e401f977ab385c9e4e3ab30627d6f26d00e2c73eef317493c4ec6d468726cf8" dependencies = [ "cfg-if", "libc", - "redox_syscall", + "redox_syscall 0.5.1", "smallvec", - "windows-targets 0.48.5", + "windows-targets 0.52.5", ] [[package]] @@ -1780,6 +2002,16 @@ dependencies = [ "hmac", ] +[[package]] +name = "peer-ast" +version = "0.1.0" +dependencies = [ + "anyhow", + "pt", + "rust_decimal", + "sqlparser", +] + [[package]] name = "peer-bigquery" version = "0.1.0" @@ -1787,9 +2019,9 @@ dependencies = [ "anyhow", "async-trait", "chrono", - "dashmap", "futures", "gcp-bigquery-client", + "peer-ast", "peer-connections", "peer-cursor", "pgwire", @@ -1803,7 +2035,6 @@ dependencies = [ "tracing", "uuid", "value", - "yup-oauth2", ] [[package]] @@ -1825,10 +2056,37 @@ version = "0.1.0" dependencies = [ "anyhow", "async-trait", + "dashmap", "futures", "pgwire", "sqlparser", "tokio", + "tracing", + "value", +] + +[[package]] +name = "peer-mysql" +version = "0.1.0" +dependencies = [ + "anyhow", + "async-trait", + "chrono", + "futures", + "mysql_async", + "peer-ast", + "peer-connections", + "peer-cursor", + "pgwire", + "pt", + "rust_decimal", + "serde", + "serde_bytes", + "serde_json", + "sqlparser", + "tokio", + "tokio-stream", + "tracing", "value", ] @@ -1869,7 +2127,6 @@ dependencies = [ "base64 0.22.0", "catalog", "chrono", - "dashmap", "futures", "hex", "jsonwebtoken", @@ -1925,6 +2182,7 @@ dependencies = [ "peer-bigquery", "peer-connections", "peer-cursor", + "peer-mysql", "peer-postgres", "peer-snowflake", "peerdb-parser", @@ -1933,7 +2191,7 @@ dependencies = [ "pt", "rand", "serde_json", - "sha256", + "similar", "sqlparser", "time", "tokio", @@ -1980,29 +2238,24 @@ dependencies = [ [[package]] name = "pgwire" -version = "0.19.2" +version = "0.22.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9b3469097db38009e1cb3b92ecb66770aa8b623b97a2aff69afdd8b0dded397d" +checksum = "3770f56e1e8a608c6de40011b9a00c6b669c14d121024411701b4bc3b2a5be99" dependencies = [ "async-trait", - "base64 0.21.7", "bytes", "chrono", "derive-new", "futures", "hex", - "log", "md5", "postgres-types", "rand", "ring", - "stringprep", "thiserror", - "time", "tokio", - "tokio-rustls 0.25.0", + "tokio-rustls 0.26.0", "tokio-util", - "x509-certificate", ] [[package]] @@ -2403,6 +2656,15 @@ dependencies = [ "bitflags 1.3.2", ] +[[package]] +name = "redox_syscall" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "469052894dcb553421e483e4209ee581a45100d31b4018de03e5a7ad86374a7e" +dependencies = [ + "bitflags 2.5.0", +] + [[package]] name = "refinery" version = "0.8.14" @@ -2629,11 +2891,17 @@ version = "0.1.23" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d626bb9dae77e28219937af045c257c28bfd3f69333c512553507f5f9798cb76" +[[package]] +name = "rustc-hash" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "08d43f7aa6b08d49f382cde6a7982047c3426db949b1424bc4b7ec9ae12c6ce2" + [[package]] name = "rustix" -version = "0.38.33" +version = "0.38.34" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e3cc72858054fcff6d7dea32df2aeaee6a7c24227366d7ea429aada2f26b16ad" +checksum = "70dc5ec042f7a43c4a73241207cecc9873a06d45debb38b329f8541d85c2730f" dependencies = [ "bitflags 2.5.0", "errno", @@ -2662,6 +2930,7 @@ version = "0.23.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "afabcee0551bd1aa3e18e5adbf2c0544722014b899adb31bd186ec638d3da97e" dependencies = [ + "log", "once_cell", "ring", "rustls-pki-types", @@ -2704,15 +2973,15 @@ dependencies = [ [[package]] name = "rustls-pki-types" -version = "1.4.1" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ecd36cc4259e3e4514335c4a138c6b43171a8d61d8f5c9348f9fc7529416f247" +checksum = "beb461507cee2c2ff151784c52762cf4d9ff6a61f3e80968600ed24fa837fa54" [[package]] name = "rustls-webpki" -version = "0.102.2" +version = "0.102.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "faaa0a62740bedb9b2ef5afa303da42764c012f743917351dc9a237ea1663610" +checksum = "f3bce581c0dd41bce533ce695a1437fa16a7ab5ac3ccfa99fe1a620a7885eabf" dependencies = [ "ring", "rustls-pki-types", @@ -2749,6 +3018,12 @@ dependencies = [ "winapi-util", ] +[[package]] +name = "saturating" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ece8e78b2f38ec51c51f5d475df0a7187ba5111b2a28bdc761ee05b075d40a71" + [[package]] name = "schannel" version = "0.1.23" @@ -2815,9 +3090,9 @@ dependencies = [ [[package]] name = "serde" -version = "1.0.198" +version = "1.0.199" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9846a40c979031340571da2545a4e5b7c4163bdae79b301d5f86d03979451fcc" +checksum = "0c9f6e76df036c77cd94996771fb40db98187f096dd0b9af39c6c6e452ba966a" dependencies = [ "serde_derive", ] @@ -2833,9 +3108,9 @@ dependencies = [ [[package]] name = "serde_derive" -version = "1.0.198" +version = "1.0.199" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e88edab869b01783ba905e7d0153f9fc1a6505a96e4ad3018011eedb838566d9" +checksum = "11bd257a6541e141e42ca6d24ae26f7714887b47e89aa739099104c7e4d3b7fc" dependencies = [ "proc-macro2", "quote", @@ -2875,10 +3150,10 @@ dependencies = [ ] [[package]] -name = "sha2" -version = "0.10.8" +name = "sha1" +version = "0.10.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "793db75ad2bcafc3ffa7c68b215fee268f537982cd901d132f89c6343f3a3dc8" +checksum = "e3bf829a2d51ab4a5ddf1352d8470c140cadc8301b2ae1789db023f01cedd6ba" dependencies = [ "cfg-if", "cpufeatures", @@ -2886,16 +3161,14 @@ dependencies = [ ] [[package]] -name = "sha256" -version = "1.5.0" +name = "sha2" +version = "0.10.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "18278f6a914fa3070aa316493f7d2ddfb9ac86ebc06fa3b83bffda487e9065b0" +checksum = "793db75ad2bcafc3ffa7c68b215fee268f537982cd901d132f89c6343f3a3dc8" dependencies = [ - "async-trait", - "bytes", - "hex", - "sha2", - "tokio", + "cfg-if", + "cpufeatures", + "digest", ] [[package]] @@ -2907,6 +3180,12 @@ dependencies = [ "lazy_static", ] +[[package]] +name = "shlex" +version = "1.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0fda2ff0d084019ba4d7c6f371c95d8fd75ce3524c3cb8fb653a3023f6323e64" + [[package]] name = "signal-hook-registry" version = "1.4.2" @@ -2938,6 +3217,12 @@ version = "0.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f27f6278552951f1f2b8cf9da965d10969b2efdea95a6ec47987ab46edfe263a" +[[package]] +name = "similar" +version = "2.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fa42c91313f1d05da9b26f267f931cf178d4aba455b4c4622dd7355eb80c6640" + [[package]] name = "simple_asn1" version = "0.6.2" @@ -2979,9 +3264,9 @@ checksum = "3c5e1a9a646d36c3599cd173a41282daf47c44583ad367b8e6837255952e5c67" [[package]] name = "socket2" -version = "0.5.6" +version = "0.5.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "05ffd9c0a93b7543e062e759284fcf5f5e3b098501104bfbdde4d404db792871" +checksum = "ce305eb0b4296696835b71df73eb912e0f1ffd2556a501fcede6e0c50349191c" dependencies = [ "libc", "windows-sys 0.52.0", @@ -3011,8 +3296,8 @@ dependencies = [ [[package]] name = "sqlparser" -version = "0.41.0" -source = "git+https://github.com/peerdb-io/sqlparser-rs.git?branch=ps-es-parsing#e86bcd9ed3dd9ac7be94569f0ae1fc20c440dafa" +version = "0.45.0" +source = "git+https://github.com/peerdb-io/sqlparser-rs.git?branch=main#d76527bd14ceb5f0b739feb37a8912a9ad7f864c" dependencies = [ "log", "sqlparser_derive", @@ -3020,14 +3305,20 @@ dependencies = [ [[package]] name = "sqlparser_derive" -version = "0.2.1" -source = "git+https://github.com/peerdb-io/sqlparser-rs.git?branch=ps-es-parsing#e86bcd9ed3dd9ac7be94569f0ae1fc20c440dafa" +version = "0.2.2" +source = "git+https://github.com/peerdb-io/sqlparser-rs.git?branch=main#d76527bd14ceb5f0b739feb37a8912a9ad7f864c" dependencies = [ "proc-macro2", "quote", "syn 2.0.60", ] +[[package]] +name = "static_assertions" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a2eb9349b6444b326872e140eb1cf5e7c522154d69e7a0ffb0fb81c06b37543f" + [[package]] name = "stringprep" version = "0.1.4" @@ -3045,6 +3336,16 @@ version = "0.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7da8b5736845d9f2fcb837ea5d9e2628564b3b043a70948a3f0b778838c5fb4f" +[[package]] +name = "subprocess" +version = "0.2.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0c2e86926081dda636c546d8c5e641661049d7562a68f5488be4a1f7f66f6086" +dependencies = [ + "libc", + "winapi", +] + [[package]] name = "subtle" version = "2.5.0" @@ -3367,7 +3668,7 @@ dependencies = [ "serde", "serde_spanned", "toml_datetime", - "winnow 0.6.6", + "winnow 0.6.7", ] [[package]] @@ -3562,6 +3863,17 @@ version = "0.2.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e421abadd41a4225275504ea4d6566923418b7f05506fbc9c0fe86ba7396114b" +[[package]] +name = "twox-hash" +version = "1.6.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "97fee6b57c6a41524a810daee9286c02d7752c4253064d0b05472833a438f675" +dependencies = [ + "cfg-if", + "rand", + "static_assertions", +] + [[package]] name = "typed-arena" version = "2.0.2" @@ -3597,9 +3909,9 @@ dependencies = [ [[package]] name = "unicode-width" -version = "0.1.11" +version = "0.1.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e51733f11c9c4f72aa0c160008246859e340b00807569a0da0e7a1079b27ba85" +checksum = "68f5e5f3158ecfd4b8ff6fe086db7c8467a2dfdac97fe420f2b7c4aa97af66d6" [[package]] name = "untrusted" @@ -3609,11 +3921,11 @@ checksum = "8ecb6da28b8a351d773b68d5825ac39017e680750f980f3a1a85cd8dd28a47c1" [[package]] name = "ureq" -version = "2.9.6" +version = "2.9.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "11f214ce18d8b2cbe84ed3aa6486ed3f5b285cf8d8fbdbce9f3f767a724adc35" +checksum = "d11a831e3c0b56e438a28308e7c810799e3c118417f342d30ecec080105395cd" dependencies = [ - "base64 0.21.7", + "base64 0.22.0", "encoding_rs", "flate2", "log", @@ -3797,6 +4109,16 @@ dependencies = [ "wasm-bindgen", ] +[[package]] +name = "webpki" +version = "0.22.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ed63aea5ce73d0ff405984102c42de94fc55a6b75765d621c65262469b3c9b53" +dependencies = [ + "ring", + "untrusted", +] + [[package]] name = "webpki-roots" version = "0.26.1" @@ -3812,7 +4134,7 @@ version = "1.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a44ab49fad634e88f55bf8f9bb3abd2f27d7204172a112c7c9987e01c1c94ea9" dependencies = [ - "redox_syscall", + "redox_syscall 0.4.1", "wasite", "web-sys", ] @@ -3835,11 +4157,11 @@ checksum = "ac3b87c63620426dd9b991e5ce0329eff545bccbbb34f3be09ff6fb6ab51b7b6" [[package]] name = "winapi-util" -version = "0.1.6" +version = "0.1.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f29e6f9198ba0d26b4c9f07dbe6f9ed633e1f3d5b8b414090084349e46a52596" +checksum = "4d4cc384e1e73b93bafa6fb4f1df8c41695c8a91cf9c4c64358067d15a7b6c6b" dependencies = [ - "winapi", + "windows-sys 0.52.0", ] [[package]] @@ -4007,9 +4329,9 @@ dependencies = [ [[package]] name = "winnow" -version = "0.6.6" +version = "0.6.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f0c976aaaa0e1f90dbb21e9587cdaf1d9679a1cde8875c0d6bd83ab96a208352" +checksum = "14b9415ee827af173ebb3f15f9083df5a122eb93572ec28741fb153356ea2578" dependencies = [ "memchr", ] @@ -4063,9 +4385,9 @@ dependencies = [ [[package]] name = "yup-oauth2" -version = "8.3.3" +version = "9.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "45b7ff561fdc7809a2adad8bce73e157d01129074098e6405d0d7dfa2d087782" +checksum = "f75463c432f5d4ca9c75047514df3d768f8ac3276ac22c9a6531af6d0a3da7ee" dependencies = [ "anyhow", "async-trait", @@ -4088,6 +4410,26 @@ dependencies = [ "url", ] +[[package]] +name = "zerocopy" +version = "0.7.32" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "74d4d3961e53fa4c9a25a8637fc2bfaf2595b3d3ae34875568a5cf64787716be" +dependencies = [ + "zerocopy-derive", +] + +[[package]] +name = "zerocopy-derive" +version = "0.7.32" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9ce1b18ccd8e73a9321186f97e46f9f04b778851177567b1975109d26a08d2a6" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.60", +] + [[package]] name = "zeroize" version = "1.7.0" @@ -4119,3 +4461,31 @@ dependencies = [ "simd-adler32", "typed-arena", ] + +[[package]] +name = "zstd" +version = "0.13.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2d789b1514203a1120ad2429eae43a7bd32b90976a7bb8a05f7ec02fa88cc23a" +dependencies = [ + "zstd-safe", +] + +[[package]] +name = "zstd-safe" +version = "7.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1cd99b45c6bc03a018c8b8a86025678c87e55526064e38f9df301989dce7ec0a" +dependencies = [ + "zstd-sys", +] + +[[package]] +name = "zstd-sys" +version = "2.0.10+zstd.1.5.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c253a4914af5bafc8fa8c86ee400827e83cf6ec01195ec1f1ed8441bf00d65aa" +dependencies = [ + "cc", + "pkg-config", +] diff --git a/nexus/Cargo.toml b/nexus/Cargo.toml index 3e4a6a062d..858243453a 100644 --- a/nexus/Cargo.toml +++ b/nexus/Cargo.toml @@ -4,9 +4,11 @@ members = [ "catalog", "flow-rs", "parser", + "peer-ast", "peer-bigquery", "peer-connections", "peer-cursor", + "peer-mysql", "peer-postgres", "peer-snowflake", "postgres-connection", @@ -18,6 +20,16 @@ members = [ resolver = "2" [workspace.dependencies] -chrono = { version = "0.4", default-features = false, features = ["serde", "std"] } -sqlparser = { git = "https://github.com/peerdb-io/sqlparser-rs.git", branch = "ps-es-parsing" } -pgwire = "0.19" +chrono = { version = "0.4", default-features = false, features = [ + "serde", + "std", +] } +dashmap = "5.0" +rust_decimal = { version = "1", default-features = false, features = [ + "tokio-pg", +] } +sqlparser = { git = "https://github.com/peerdb-io/sqlparser-rs.git", branch = "main" } +tracing = "0.1" +pgwire = { version = "0.22", default-features = false, features = [ + "server-api-ring", +] } diff --git a/nexus/analyzer/src/lib.rs b/nexus/analyzer/src/lib.rs index bf4722e596..0642966713 100644 --- a/nexus/analyzer/src/lib.rs +++ b/nexus/analyzer/src/lib.rs @@ -3,7 +3,6 @@ use std::{ collections::{HashMap, HashSet}, ops::ControlFlow, - vec, }; use anyhow::Context; @@ -16,8 +15,11 @@ use pt::{ }, }; use qrep::process_options; -use sqlparser::ast::CreateMirror::{Select, CDC}; -use sqlparser::ast::{visit_relations, visit_statements, FetchDirection, SqlOption, Statement}; +use sqlparser::ast::{ + self, visit_relations, visit_statements, + CreateMirror::{Select, CDC}, + Expr, FetchDirection, SqlOption, Statement, +}; mod qrep; @@ -51,26 +53,38 @@ impl<'a> StatementAnalyzer for PeerExistanceAnalyzer<'a> { fn analyze(&self, statement: &Statement) -> anyhow::Result { let mut peers_touched: HashSet = HashSet::new(); + let mut analyze_name = |name: &str| { + let name = name.to_lowercase(); + if self.peers.contains_key(&name) { + peers_touched.insert(name); + } + }; - // This is necessary as visit relations was not visiting drop table's object names, - // causing DROP commands for Postgres peer being interpreted as - // catalog queries. + // Necessary as visit_relations fails to deeply visit some structures. visit_statements(statement, |stmt| { - if let &Statement::Drop { names, .. } = &stmt { - for name in names { - let peer_name = name.0[0].value.to_lowercase(); - if self.peers.contains_key(&peer_name) { - peers_touched.insert(peer_name); + match stmt { + Statement::Drop { names, .. } => { + for name in names { + analyze_name(&name.0[0].value); + } + } + Statement::Declare { stmts } => { + for stmt in stmts { + if let Some(ref query) = stmt.for_query { + visit_relations(query, |relation| { + analyze_name(&relation.0[0].value); + ControlFlow::<()>::Continue(()) + }); + } } } + _ => (), } ControlFlow::<()>::Continue(()) }); + visit_relations(statement, |relation| { - let peer_name = relation.0[0].value.to_lowercase(); - if self.peers.contains_key(&peer_name) { - peers_touched.insert(peer_name); - } + analyze_name(&relation.0[0].value); ControlFlow::<()>::Continue(()) }); @@ -162,9 +176,10 @@ impl StatementAnalyzer for PeerDDLAnalyzer { } => { match create_mirror { CDC(cdc) => { - let mut flow_job_table_mappings = vec![]; - for table_mapping in &cdc.mapping_options { - flow_job_table_mappings.push(FlowJobTableMapping { + let flow_job_table_mappings = cdc + .mapping_options + .iter() + .map(|table_mapping| FlowJobTableMapping { source_table_identifier: table_mapping.source.to_string(), destination_table_identifier: table_mapping.destination.to_string(), partition_key: table_mapping @@ -176,8 +191,8 @@ impl StatementAnalyzer for PeerDDLAnalyzer { .as_ref() .map(|ss| ss.iter().map(|s| s.value.clone()).collect()) .unwrap_or_default(), - }); - } + }) + .collect::>(); // get do_initial_copy from with_options let mut raw_options = HashMap::with_capacity(cdc.with_options.len()); @@ -185,9 +200,9 @@ impl StatementAnalyzer for PeerDDLAnalyzer { raw_options.insert(&option.name.value as &str, &option.value); } let do_initial_copy = match raw_options.remove("do_initial_copy") { - Some(sqlparser::ast::Value::Boolean(b)) => *b, + Some(Expr::Value(ast::Value::Boolean(b))) => *b, // also support "true" and "false" as strings - Some(sqlparser::ast::Value::SingleQuotedString(s)) => { + Some(Expr::Value(ast::Value::SingleQuotedString(s))) => { match s.as_ref() { "true" => true, "false" => false, @@ -203,9 +218,9 @@ impl StatementAnalyzer for PeerDDLAnalyzer { // bool resync true or false, default to false if not in opts let resync = match raw_options.remove("resync") { - Some(sqlparser::ast::Value::Boolean(b)) => *b, + Some(Expr::Value(ast::Value::Boolean(b))) => *b, // also support "true" and "false" as strings - Some(sqlparser::ast::Value::SingleQuotedString(s)) => { + Some(Expr::Value(ast::Value::SingleQuotedString(s))) => { match s.as_ref() { "true" => true, "false" => false, @@ -218,99 +233,99 @@ impl StatementAnalyzer for PeerDDLAnalyzer { let publication_name: Option = match raw_options .remove("publication_name") { - Some(sqlparser::ast::Value::SingleQuotedString(s)) => Some(s.clone()), + Some(Expr::Value(ast::Value::SingleQuotedString(s))) => Some(s.clone()), _ => None, }; let replication_slot_name: Option = match raw_options .remove("replication_slot_name") { - Some(sqlparser::ast::Value::SingleQuotedString(s)) => Some(s.clone()), + Some(Expr::Value(ast::Value::SingleQuotedString(s))) => Some(s.clone()), _ => None, }; let snapshot_num_rows_per_partition: Option = match raw_options .remove("snapshot_num_rows_per_partition") { - Some(sqlparser::ast::Value::Number(n, _)) => Some(n.parse::()?), + Some(Expr::Value(ast::Value::Number(n, _))) => Some(n.parse::()?), _ => None, }; let snapshot_num_tables_in_parallel: Option = match raw_options .remove("snapshot_num_tables_in_parallel") { - Some(sqlparser::ast::Value::Number(n, _)) => Some(n.parse::()?), + Some(Expr::Value(ast::Value::Number(n, _))) => Some(n.parse::()?), _ => None, }; let snapshot_staging_path = match raw_options.remove("snapshot_staging_path") { - Some(sqlparser::ast::Value::SingleQuotedString(s)) => s.clone(), + Some(Expr::Value(ast::Value::SingleQuotedString(s))) => s.clone(), _ => String::new(), }; let snapshot_max_parallel_workers: Option = match raw_options .remove("snapshot_max_parallel_workers") { - Some(sqlparser::ast::Value::Number(n, _)) => Some(n.parse::()?), + Some(Expr::Value(ast::Value::Number(n, _))) => Some(n.parse::()?), _ => None, }; let cdc_staging_path = match raw_options.remove("cdc_staging_path") { - Some(sqlparser::ast::Value::SingleQuotedString(s)) => Some(s.clone()), + Some(Expr::Value(ast::Value::SingleQuotedString(s))) => Some(s.clone()), _ => Some("".to_string()), }; let soft_delete = match raw_options.remove("soft_delete") { - Some(sqlparser::ast::Value::Boolean(b)) => *b, + Some(Expr::Value(ast::Value::Boolean(b))) => *b, _ => false, }; let push_parallelism: Option = match raw_options .remove("push_parallelism") { - Some(sqlparser::ast::Value::Number(n, _)) => Some(n.parse::()?), + Some(Expr::Value(ast::Value::Number(n, _))) => Some(n.parse::()?), _ => None, }; let push_batch_size: Option = match raw_options .remove("push_batch_size") { - Some(sqlparser::ast::Value::Number(n, _)) => Some(n.parse::()?), + Some(Expr::Value(ast::Value::Number(n, _))) => Some(n.parse::()?), _ => None, }; let max_batch_size: Option = match raw_options.remove("max_batch_size") { - Some(sqlparser::ast::Value::Number(n, _)) => Some(n.parse::()?), + Some(Expr::Value(ast::Value::Number(n, _))) => Some(n.parse::()?), _ => None, }; let soft_delete_col_name: Option = match raw_options .remove("soft_delete_col_name") { - Some(sqlparser::ast::Value::SingleQuotedString(s)) => Some(s.clone()), + Some(Expr::Value(ast::Value::SingleQuotedString(s))) => Some(s.clone()), _ => None, }; let synced_at_col_name: Option = match raw_options .remove("synced_at_col_name") { - Some(sqlparser::ast::Value::SingleQuotedString(s)) => Some(s.clone()), + Some(Expr::Value(ast::Value::SingleQuotedString(s))) => Some(s.clone()), _ => None, }; let initial_copy_only = match raw_options.remove("initial_copy_only") { - Some(sqlparser::ast::Value::Boolean(b)) => *b, + Some(Expr::Value(ast::Value::Boolean(b))) => *b, _ => false, }; let script = match raw_options.remove("script") { - Some(sqlparser::ast::Value::SingleQuotedString(s)) => s.clone(), + Some(Expr::Value(ast::Value::SingleQuotedString(s))) => s.clone(), _ => String::new(), }; let system = match raw_options.remove("system") { - Some(sqlparser::ast::Value::SingleQuotedString(s)) => s.clone(), + Some(Expr::Value(ast::Value::SingleQuotedString(s))) => s.clone(), _ => "Q".to_string(), }; @@ -352,15 +367,15 @@ impl StatementAnalyzer for PeerDDLAnalyzer { Select(select) => { let mut raw_options = HashMap::with_capacity(select.with_options.len()); for option in &select.with_options { - raw_options.insert(&option.name.value as &str, &option.value); + if let Expr::Value(ref value) = option.value { + raw_options.insert(&option.name.value as &str, value); + } } // we treat disabled as a special option, and do not pass it to the // flow server, this is primarily used for external orchestration. let mut disabled = false; - if let Some(sqlparser::ast::Value::Boolean(b)) = - raw_options.remove("disabled") - { + if let Some(ast::Value::Boolean(b)) = raw_options.remove("disabled") { disabled = *b; } @@ -411,7 +426,7 @@ impl StatementAnalyzer for PeerDDLAnalyzer { } let query_string = match raw_options.remove("query_string") { - Some(sqlparser::ast::Value::SingleQuotedString(s)) => Some(s.clone()), + Some(Expr::Value(ast::Value::SingleQuotedString(s))) => Some(s.clone()), _ => None, }; @@ -468,12 +483,14 @@ impl StatementAnalyzer for PeerCursorAnalyzer { name, direction, .. } => { let count = match direction { + FetchDirection::ForwardAll | FetchDirection::All => usize::MAX, + FetchDirection::Next | FetchDirection::Forward { limit: None } => 1, FetchDirection::Count { - limit: sqlparser::ast::Value::Number(n, _), + limit: ast::Value::Number(n, _), } | FetchDirection::Forward { - limit: Some(sqlparser::ast::Value::Number(n, _)), - } => n.parse::(), + limit: Some(ast::Value::Number(n, _)), + } => n.parse::()?, _ => { return Err(anyhow::anyhow!( "invalid fetch direction for cursor: {:?}", @@ -481,11 +498,11 @@ impl StatementAnalyzer for PeerCursorAnalyzer { )) } }; - Ok(Some(CursorEvent::Fetch(name.value.clone(), count?))) + Ok(Some(CursorEvent::Fetch(name.value.clone(), count))) } Statement::Close { cursor } => match cursor { - sqlparser::ast::CloseCursor::All => Ok(Some(CursorEvent::CloseAll)), - sqlparser::ast::CloseCursor::Specific { name } => { + ast::CloseCursor::All => Ok(Some(CursorEvent::CloseAll)), + ast::CloseCursor::Specific { name } => { Ok(Some(CursorEvent::Close(name.to_string()))) } }, @@ -498,15 +515,10 @@ fn parse_db_options(db_type: DbType, with_options: &[SqlOption]) -> anyhow::Resu let mut opts: HashMap<&str, &str> = HashMap::with_capacity(with_options.len()); for opt in with_options { let val = match opt.value { - sqlparser::ast::Value::SingleQuotedString(ref str) => str, - sqlparser::ast::Value::Number(ref v, _) => v, - sqlparser::ast::Value::Boolean(v) => { - if v { - "true" - } else { - "false" - } - } + Expr::Value(ast::Value::SingleQuotedString(ref str)) => str, + Expr::Value(ast::Value::Number(ref v, _)) => v, + Expr::Value(ast::Value::Boolean(true)) => "true", + Expr::Value(ast::Value::Boolean(false)) => "false", _ => panic!("invalid option type for peer"), }; opts.insert(&opt.name.value, val); @@ -903,5 +915,40 @@ fn parse_db_options(db_type: DbType, with_options: &[SqlOption]) -> anyhow::Resu }) } } + DbType::Mysql => Config::MysqlConfig(pt::peerdb_peers::MySqlConfig { + host: opts.get("host").context("no host specified")?.to_string(), + port: opts + .get("port") + .context("no port specified")? + .parse::() + .context("unable to parse port as valid int")?, + user: opts + .get("user") + .cloned() + .unwrap_or_default() + .to_string(), + password: opts + .get("password") + .cloned() + .unwrap_or_default() + .to_string(), + database: opts + .get("database") + .cloned() + .unwrap_or_default() + .to_string(), + setup: opts + .get("setup") + .map(|s| s.split(';').map(String::from).collect::>()) + .unwrap_or_default(), + compression: opts + .get("compression") + .and_then(|s| s.parse::().ok()) + .unwrap_or_default(), + disable_tls: opts + .get("disable_tls") + .and_then(|s| s.parse::().ok()) + .unwrap_or_default(), + }), })) } diff --git a/nexus/analyzer/src/qrep.rs b/nexus/analyzer/src/qrep.rs index 14bdca8713..692b093d94 100644 --- a/nexus/analyzer/src/qrep.rs +++ b/nexus/analyzer/src/qrep.rs @@ -1,7 +1,7 @@ use std::collections::HashMap; use serde_json::Value; -use sqlparser::ast::Value as SqlValue; +use sqlparser::ast; enum QRepOptionType { String { @@ -96,7 +96,7 @@ const QREP_OPTIONS: &[QRepOptionType] = &[ ]; pub fn process_options( - mut raw_opts: HashMap<&str, &SqlValue>, + mut raw_opts: HashMap<&str, &ast::Value>, ) -> anyhow::Result> { let mut opts: HashMap = HashMap::new(); @@ -109,7 +109,7 @@ pub fn process_options( accepted_values, } => { if let Some(raw_value) = raw_opts.remove(*name) { - if let SqlValue::SingleQuotedString(str) = raw_value { + if let ast::Value::SingleQuotedString(str) = raw_value { if let Some(values) = accepted_values { if !values.contains(&str.as_str()) { anyhow::bail!("{} must be one of {:?}", name, values); @@ -132,7 +132,7 @@ pub fn process_options( required, } => { if let Some(raw_value) = raw_opts.remove(*name) { - if let SqlValue::Number(num_str, _) = raw_value { + if let ast::Value::Number(num_str, _) = raw_value { let num = num_str.parse::()?; if let Some(min) = min_value { if num < *min { @@ -153,7 +153,7 @@ pub fn process_options( QRepOptionType::StringArray { name } => { // read it as a string and split on comma if let Some(raw_value) = raw_opts.remove(*name) { - if let SqlValue::SingleQuotedString(str) = raw_value { + if let ast::Value::SingleQuotedString(str) = raw_value { let values: Vec = str .split(',') .map(|s| Value::String(s.trim().to_string())) @@ -170,7 +170,7 @@ pub fn process_options( required, } => { if let Some(raw_value) = raw_opts.remove(*name) { - if let SqlValue::Boolean(b) = raw_value { + if let ast::Value::Boolean(b) = raw_value { opts.insert(name.to_string(), Value::Bool(*b)); } else { anyhow::bail!("Invalid value for {}", name); diff --git a/nexus/catalog/src/lib.rs b/nexus/catalog/src/lib.rs index 2c53000759..0c3b65d2b8 100644 --- a/nexus/catalog/src/lib.rs +++ b/nexus/catalog/src/lib.rs @@ -87,13 +87,12 @@ impl Catalog { pub async fn create_peer(&self, peer: &Peer) -> anyhow::Result { let config_blob = { - let config = peer.config.clone().context("invalid peer config")?; + let config = peer.config.as_ref().context("invalid peer config")?; match config { Config::SnowflakeConfig(snowflake_config) => snowflake_config.encode_to_vec(), Config::BigqueryConfig(bigquery_config) => bigquery_config.encode_to_vec(), Config::MongoConfig(mongo_config) => mongo_config.encode_to_vec(), Config::PostgresConfig(postgres_config) => postgres_config.encode_to_vec(), - Config::EventhubConfig(eventhub_config) => eventhub_config.encode_to_vec(), Config::S3Config(s3_config) => s3_config.encode_to_vec(), Config::SqlserverConfig(sqlserver_config) => sqlserver_config.encode_to_vec(), Config::EventhubGroupConfig(eventhub_group_config) => { @@ -105,6 +104,7 @@ impl Catalog { Config::ElasticsearchConfig(elasticsearch_config) => { elasticsearch_config.encode_to_vec() } + Config::MysqlConfig(mysql_config) => mysql_config.encode_to_vec(), } }; @@ -321,6 +321,11 @@ impl Catalog { pt::peerdb_peers::ElasticsearchConfig::decode(options).with_context(err)?; Config::ElasticsearchConfig(elasticsearch_config) } + DbType::Mysql => { + let mysql_config = + pt::peerdb_peers::MySqlConfig::decode(options).with_context(err)?; + Config::MysqlConfig(mysql_config) + } }) } else { None diff --git a/nexus/flow-rs/Cargo.toml b/nexus/flow-rs/Cargo.toml index c46c0ee842..5ae9cda2d9 100644 --- a/nexus/flow-rs/Cargo.toml +++ b/nexus/flow-rs/Cargo.toml @@ -6,7 +6,7 @@ edition = "2021" [dependencies] serde_json = "1.0" anyhow = "1.0" -tracing = "0.1" +tracing.workspace = true tonic-health = "0.11" pt = { path = "../pt" } catalog = { path = "../catalog" } diff --git a/nexus/parser/Cargo.toml b/nexus/parser/Cargo.toml index 8c79ac4a18..b6aac7d88b 100644 --- a/nexus/parser/Cargo.toml +++ b/nexus/parser/Cargo.toml @@ -15,4 +15,4 @@ pt = { path = "../pt" } rand = "0.8" sqlparser.workspace = true tokio = { version = "1", features = ["full"] } -tracing = "0.1" +tracing.workspace = true diff --git a/nexus/parser/src/lib.rs b/nexus/parser/src/lib.rs index deb16af505..4f5cad356a 100644 --- a/nexus/parser/src/lib.rs +++ b/nexus/parser/src/lib.rs @@ -33,6 +33,9 @@ pub enum NexusStatement { stmt: Statement, cursor: CursorEvent, }, + Rollback { + stmt: Statement, + }, Empty, } @@ -41,16 +44,13 @@ impl NexusStatement { peers: HashMap, stmt: &Statement, ) -> PgWireResult { - let ddl = { - let pdl: PeerDDLAnalyzer = PeerDDLAnalyzer; - pdl.analyze(stmt).map_err(|e| { - PgWireError::UserError(Box::new(ErrorInfo::new( - "ERROR".to_owned(), - "internal_error".to_owned(), - e.to_string(), - ))) - }) - }?; + let ddl = PeerDDLAnalyzer.analyze(stmt).map_err(|e| { + PgWireError::UserError(Box::new(ErrorInfo::new( + "ERROR".to_owned(), + "internal_error".to_owned(), + e.to_string(), + ))) + })?; if let Some(ddl) = ddl { return Ok(NexusStatement::PeerDDL { @@ -59,8 +59,7 @@ impl NexusStatement { }); } - let peer_cursor: PeerCursorAnalyzer = Default::default(); - if let Ok(Some(cursor)) = peer_cursor.analyze(stmt) { + if let Ok(Some(cursor)) = PeerCursorAnalyzer.analyze(stmt) { return Ok(NexusStatement::PeerCursor { stmt: stmt.clone(), cursor, @@ -126,12 +125,19 @@ impl NexusQueryParser { }) } else { let stmt = stmts.remove(0); - let peers = self.get_peers_bridge().await?; - let nexus_stmt = NexusStatement::new(peers, &stmt)?; - Ok(NexusParsedStatement { - statement: nexus_stmt, - query: sql.to_owned(), - }) + if matches!(stmt, Statement::Rollback { .. }) { + Ok(NexusParsedStatement { + statement: NexusStatement::Rollback { stmt }, + query: sql.to_owned(), + }) + } else { + let peers = self.get_peers_bridge().await?; + let nexus_stmt = NexusStatement::new(peers, &stmt)?; + Ok(NexusParsedStatement { + statement: nexus_stmt, + query: sql.to_owned(), + }) + } } } } diff --git a/nexus/peer-ast/Cargo.toml b/nexus/peer-ast/Cargo.toml new file mode 100644 index 0000000000..91264b6a41 --- /dev/null +++ b/nexus/peer-ast/Cargo.toml @@ -0,0 +1,12 @@ +[package] +name = "peer-ast" +version = "0.1.0" +edition = "2021" + +# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html + +[dependencies] +anyhow = "1.0" +pt = { path = "../pt" } +rust_decimal.workspace = true +sqlparser.workspace = true diff --git a/nexus/peer-ast/src/lib.rs b/nexus/peer-ast/src/lib.rs new file mode 100644 index 0000000000..cab1728999 --- /dev/null +++ b/nexus/peer-ast/src/lib.rs @@ -0,0 +1,101 @@ +use sqlparser::ast::{Array, ArrayElemTypeDef, DataType, Expr}; + +/// Flatten Cast EXPR to List with right value type +/// For example Value(SingleQuotedString("{hash1,hash2}") must return +/// a vector Value(SingleQuotedString("hash1"), Value(SingleQuotedString("hash2"))) +pub fn flatten_expr_to_in_list(expr: &Expr) -> anyhow::Result> { + let mut list = vec![]; + // check if expr is of type Cast + if let Expr::Cast { + expr, data_type, .. + } = expr + { + // assert that expr is of type SingleQuotedString + if let Expr::Value(sqlparser::ast::Value::SingleQuotedString(s)) = expr.as_ref() { + // trim the starting and ending curly braces + let s = s.trim_start_matches('{').trim_end_matches('}'); + // split string by comma + let split = s.split(','); + // match on data type, and create a vector of Expr::Value + match data_type { + DataType::Array(ArrayElemTypeDef::AngleBracket(inner)) + | DataType::Array(ArrayElemTypeDef::SquareBracket(inner)) => match inner.as_ref() { + DataType::Text | DataType::Char(_) | DataType::Varchar(_) => { + for s in split { + list.push(Expr::Value(sqlparser::ast::Value::SingleQuotedString( + s.to_string(), + ))); + } + } + DataType::Integer(_) + | DataType::Float(_) + | DataType::BigInt(_) + | DataType::UnsignedBigInt(_) + | DataType::UnsignedInteger(_) + | DataType::UnsignedSmallInt(_) + | DataType::UnsignedTinyInt(_) + | DataType::TinyInt(_) + | DataType::UnsignedInt(_) => { + for s in split { + list.push(Expr::Value(sqlparser::ast::Value::Number( + s.to_string(), + false, + ))); + } + } + _ => { + return Err(anyhow::anyhow!( + "Unsupported inner data type for IN list: {:?}", + data_type + )) + } + }, + _ => { + return Err(anyhow::anyhow!( + "Unsupported data type for IN list: {:?}", + data_type + )) + } + } + } else if let Expr::Array(arr) = expr.as_ref() { + list = pour_array_into_list(arr, list).expect("Failed to transfer array to list"); + } + } else if let Expr::Array(arr) = expr { + list = pour_array_into_list(arr, list).expect("Failed to transfer array to list"); + } + + Ok(list) +} + +fn pour_array_into_list(arr: &Array, mut list: Vec) -> anyhow::Result> { + for element in &arr.elem { + match &element { + Expr::Value(val) => match val { + sqlparser::ast::Value::Number(_, _) => { + list.push(Expr::Value(sqlparser::ast::Value::Number( + element.to_string(), + false, + ))); + } + sqlparser::ast::Value::SingleQuotedString(_) => { + list.push(Expr::Value(sqlparser::ast::Value::UnQuotedString( + element.to_string(), + ))); + } + _ => { + return Err(anyhow::anyhow!( + "Unsupported data type for IN list: {:?}", + val + )) + } + }, + _ => { + return Err(anyhow::anyhow!( + "Unsupported element for IN list: {:?}", + element + )) + } + } + } + Ok(list) +} diff --git a/nexus/peer-bigquery/Cargo.toml b/nexus/peer-bigquery/Cargo.toml index 6626bb9840..94670fe4f6 100644 --- a/nexus/peer-bigquery/Cargo.toml +++ b/nexus/peer-bigquery/Cargo.toml @@ -9,20 +9,19 @@ edition = "2021" anyhow = "1.0" async-trait = "0.1" chrono.workspace = true -dashmap = "5.0" futures = { version = "0.3.28", features = ["executor"] } +peer-ast = { path = "../peer-ast" } peer-cursor = { path = "../peer-cursor" } peer-connections = { path = "../peer-connections" } pgwire.workspace = true pt = { path = "../pt" } -rust_decimal = { version = "1.30.0", features = [ "tokio-pg" ] } +rust_decimal.workspace = true serde = { version = "1.0", features = ["derive"] } serde_json = "1.0" serde_bytes = "0.11" sqlparser.workspace = true -tracing = "0.1" +tracing.workspace = true tokio = { version = "1.0", features = ["full"] } -gcp-bigquery-client = "0.18" +gcp-bigquery-client = "0.20" uuid = { version = "1.0", features = ["serde", "v4"] } value = { path = "../value" } -yup-oauth2 = "8.3.3" diff --git a/nexus/peer-bigquery/src/ast.rs b/nexus/peer-bigquery/src/ast.rs index 8429e0ebe1..fd8153382a 100644 --- a/nexus/peer-bigquery/src/ast.rs +++ b/nexus/peer-bigquery/src/ast.rs @@ -1,15 +1,15 @@ use std::ops::ControlFlow; +use peer_ast::flatten_expr_to_in_list; use sqlparser::ast::Value::Number; use sqlparser::ast::{ visit_expressions_mut, visit_function_arg_mut, visit_relations_mut, visit_setexpr_mut, Array, - ArrayElemTypeDef, BinaryOperator, DataType, DateTimeField, Expr, Function, FunctionArg, - FunctionArgExpr, Ident, ObjectName, Query, SetExpr, SetOperator, SetQuantifier, TimezoneInfo, + BinaryOperator, DataType, DateTimeField, Expr, Function, FunctionArg, FunctionArgExpr, Ident, + ObjectName, Query, SetExpr, SetOperator, SetQuantifier, TimezoneInfo, }; -#[derive(Default)] -pub struct BigqueryAst {} +pub struct BigqueryAst; impl BigqueryAst { pub fn is_timestamp_returning_function(&self, name: &str) -> bool { @@ -82,9 +82,8 @@ impl BigqueryAst { .. } = arg_expr { - let list = self - .flatten_expr_to_in_list(arg_expr) - .expect("failed to flatten in function"); + let list = + flatten_expr_to_in_list(arg_expr).expect("failed to flatten in function"); let rewritten_array = Array { elem: list, named: true, @@ -242,9 +241,7 @@ impl BigqueryAst { } = node { if matches!(compare_op, BinaryOperator::Eq | BinaryOperator::NotEq) { - let list = self - .flatten_expr_to_in_list(right) - .expect("failed to flatten"); + let list = flatten_expr_to_in_list(right).expect("failed to flatten"); *node = Expr::InList { expr: left.clone(), list, @@ -258,109 +255,4 @@ impl BigqueryAst { Ok(()) } - - fn pour_array_into_list(&self, arr: &Array, mut list: Vec) -> anyhow::Result> { - for element in &arr.elem { - match &element { - Expr::Value(val) => match val { - sqlparser::ast::Value::Number(_, _) => { - list.push(Expr::Value(sqlparser::ast::Value::Number( - element.to_string(), - false, - ))); - } - sqlparser::ast::Value::SingleQuotedString(_) => { - list.push(Expr::Value(sqlparser::ast::Value::UnQuotedString( - element.to_string(), - ))); - } - _ => { - return Err(anyhow::anyhow!( - "Unsupported data type for IN list: {:?}", - val - )) - } - }, - _ => { - return Err(anyhow::anyhow!( - "Unsupported element for IN list: {:?}", - element - )) - } - } - } - Ok(list) - } - /// Flatten Cast EXPR to List with right value type - /// For example Value(SingleQuotedString("{hash1,hash2}") must return - /// a vector Value(SingleQuotedString("hash1"), Value(SingleQuotedString("hash2"))) - fn flatten_expr_to_in_list(&self, expr: &Expr) -> anyhow::Result> { - let mut list = vec![]; - // check if expr is of type Cast - if let Expr::Cast { - expr, data_type, .. - } = expr - { - // assert that expr is of type SingleQuotedString - if let Expr::Value(sqlparser::ast::Value::SingleQuotedString(s)) = expr.as_ref() { - // trim the starting and ending curly braces - let s = s.trim_start_matches('{').trim_end_matches('}'); - // split string by comma - let split = s.split(','); - // match on data type, and create a vector of Expr::Value - match data_type { - DataType::Array(ArrayElemTypeDef::AngleBracket(inner)) - | DataType::Array(ArrayElemTypeDef::SquareBracket(inner)) => { - match inner.as_ref() { - DataType::Text | DataType::Char(_) | DataType::Varchar(_) => { - for s in split { - list.push(Expr::Value( - sqlparser::ast::Value::SingleQuotedString(s.to_string()), - )); - } - } - DataType::Integer(_) - | DataType::Float(_) - | DataType::BigInt(_) - | DataType::UnsignedBigInt(_) - | DataType::UnsignedInteger(_) - | DataType::UnsignedSmallInt(_) - | DataType::UnsignedTinyInt(_) - | DataType::TinyInt(_) - | DataType::UnsignedInt(_) => { - for s in split { - list.push(Expr::Value(sqlparser::ast::Value::Number( - s.to_string(), - false, - ))); - } - } - _ => { - return Err(anyhow::anyhow!( - "Unsupported inner data type for IN list: {:?}", - data_type - )) - } - } - } - _ => { - return Err(anyhow::anyhow!( - "Unsupported data type for IN list: {:?}", - data_type - )) - } - } - } else if let Expr::Array(arr) = expr.as_ref() { - list = self - .pour_array_into_list(arr, list) - .expect("Failed to transfer array to list"); - } - } else if let Expr::Array(arr) = expr { - list = self - .pour_array_into_list(arr, list) - .expect("Failed to transfer array to list"); - } - - Ok(list) - } } diff --git a/nexus/peer-bigquery/src/lib.rs b/nexus/peer-bigquery/src/lib.rs index 656760c2c8..a7a8fb8f2f 100644 --- a/nexus/peer-bigquery/src/lib.rs +++ b/nexus/peer-bigquery/src/lib.rs @@ -1,20 +1,18 @@ use std::time::Duration; use anyhow::Context; -use cursor::BigQueryCursorManager; use gcp_bigquery_client::{ model::{query_request::QueryRequest, query_response::ResultSet}, - Client, + yup_oauth2, Client, }; use peer_connections::PeerConnectionTracker; -use peer_cursor::{CursorModification, QueryExecutor, QueryOutput, Schema}; +use peer_cursor::{CursorManager, CursorModification, QueryExecutor, QueryOutput, Schema}; use pgwire::error::{ErrorInfo, PgWireError, PgWireResult}; use pt::peerdb_peers::BigqueryConfig; -use sqlparser::ast::{CloseCursor, Expr, FetchDirection, Statement, Value}; +use sqlparser::ast::{CloseCursor, Declare, Expr, FetchDirection, Statement, Value}; use stream::{BqRecordStream, BqSchema}; mod ast; -mod cursor; mod stream; pub struct BigQueryQueryExecutor { @@ -23,7 +21,7 @@ pub struct BigQueryQueryExecutor { dataset_id: String, peer_connections: PeerConnectionTracker, client: Box, - cursor_manager: BigQueryCursorManager, + cursor_manager: CursorManager, } pub async fn bq_client_from_config(config: &BigqueryConfig) -> anyhow::Result { @@ -53,15 +51,14 @@ impl BigQueryQueryExecutor { peer_connections: PeerConnectionTracker, ) -> anyhow::Result { let client = bq_client_from_config(config).await?; - let client = Box::new(client); - let cursor_manager = BigQueryCursorManager::new(); + Ok(Self { peer_name, project_id: config.project_id.clone(), dataset_id: config.dataset_id.clone(), peer_connections, - client, - cursor_manager, + client: Box::new(client), + cursor_manager: Default::default(), }) } @@ -100,8 +97,7 @@ impl QueryExecutor for BigQueryQueryExecutor { match stmt { Statement::Query(query) => { let mut query = query.clone(); - let bq_ast = ast::BigqueryAst::default(); - bq_ast + ast::BigqueryAst .rewrite(&self.dataset_id, &mut query) .context("unable to rewrite query") .map_err(|err| PgWireError::ApiError(err.into()))?; @@ -119,15 +115,31 @@ impl QueryExecutor for BigQueryQueryExecutor { ); Ok(QueryOutput::Stream(Box::pin(cursor))) } - Statement::Declare { name, query, .. } => { - let query_stmt = Statement::Query(query.clone()); - self.cursor_manager - .create_cursor(&name.value, &query_stmt, self) - .await?; - - Ok(QueryOutput::Cursor(CursorModification::Created( - name.value.clone(), - ))) + Statement::Declare { stmts } => { + if stmts.len() != 1 { + Err(PgWireError::ApiError( + "peerdb only supports singular declare statements".into(), + )) + } else if let Declare { + ref names, + for_query: Some(ref query), + .. + } = stmts[0] + { + let name = &names[0]; + let query_stmt = Statement::Query(query.clone()); + self.cursor_manager + .create_cursor(&name.value, &query_stmt, self) + .await?; + + Ok(QueryOutput::Cursor(CursorModification::Created( + name.value.clone(), + ))) + } else { + Err(PgWireError::ApiError( + "peerdb only supports declare for query statements".into(), + )) + } } Statement::Fetch { name, direction, .. @@ -136,12 +148,16 @@ impl QueryExecutor for BigQueryQueryExecutor { // Attempt to extract the count from the direction let count = match direction { + FetchDirection::ForwardAll | FetchDirection::All => usize::MAX, + FetchDirection::Next | FetchDirection::Forward { limit: None } => 1, FetchDirection::Count { limit: sqlparser::ast::Value::Number(n, _), } | FetchDirection::Forward { limit: Some(sqlparser::ast::Value::Number(n, _)), - } => n.parse::(), + } => n + .parse::() + .map_err(|err| PgWireError::ApiError(err.into()))?, _ => { return Err(PgWireError::UserError(Box::new(ErrorInfo::new( "ERROR".to_owned(), @@ -151,12 +167,6 @@ impl QueryExecutor for BigQueryQueryExecutor { } }; - // If parsing the count resulted in an error, return an internal error - let count = match count { - Ok(c) => c, - Err(err) => return Err(PgWireError::ApiError(err.into())), - }; - tracing::info!("fetching {} rows", count); // Fetch rows from the cursor manager @@ -166,14 +176,11 @@ impl QueryExecutor for BigQueryQueryExecutor { Ok(QueryOutput::Records(records)) } Statement::Close { cursor } => { - let mut closed_cursors = vec![]; - match cursor { - CloseCursor::All => { - closed_cursors = self.cursor_manager.close_all_cursors().await?; - } + let closed_cursors = match cursor { + CloseCursor::All => self.cursor_manager.close_all_cursors().await?, CloseCursor::Specific { name } => { self.cursor_manager.close(&name.value).await?; - closed_cursors.push(name.value.clone()); + vec![name.value.clone()] } }; Ok(QueryOutput::Cursor(CursorModification::Closed( @@ -202,8 +209,7 @@ impl QueryExecutor for BigQueryQueryExecutor { match stmt { Statement::Query(query) => { let mut query = query.clone(); - let bq_ast = ast::BigqueryAst::default(); - bq_ast + ast::BigqueryAst .rewrite(&self.dataset_id, &mut query) .context("unable to rewrite query") .map_err(|err| PgWireError::ApiError(err.into()))?; @@ -222,9 +228,23 @@ impl QueryExecutor for BigQueryQueryExecutor { Ok(Some(schema.schema())) } - Statement::Declare { query, .. } => { - let query_stmt = Statement::Query(query.clone()); - self.describe(&query_stmt).await + Statement::Declare { stmts } => { + if stmts.len() != 1 { + Err(PgWireError::ApiError( + "peerdb only supports singular declare statements".into(), + )) + } else if let Declare { + for_query: Some(ref query), + .. + } = stmts[0] + { + let query_stmt = Statement::Query(query.clone()); + self.describe(&query_stmt).await + } else { + Err(PgWireError::ApiError( + "peerdb only supports declare for query statements".into(), + )) + } } _ => PgWireResult::Err(PgWireError::UserError(Box::new(ErrorInfo::new( "ERROR".to_owned(), diff --git a/nexus/peer-bigquery/src/stream.rs b/nexus/peer-bigquery/src/stream.rs index 6019ec82be..d0f7e5b5b7 100644 --- a/nexus/peer-bigquery/src/stream.rs +++ b/nexus/peer-bigquery/src/stream.rs @@ -33,7 +33,7 @@ pub struct BqRecordStream { num_records: usize, } -// covnert FieldType to pgwire FieldInfo's Type +// convert FieldType to pgwire FieldInfo's Type fn convert_field_type(field_type: &FieldType) -> Type { match field_type { FieldType::Bool => Type::BOOL, diff --git a/nexus/peer-connections/Cargo.toml b/nexus/peer-connections/Cargo.toml index b57d7e53d7..78986b3b25 100644 --- a/nexus/peer-connections/Cargo.toml +++ b/nexus/peer-connections/Cargo.toml @@ -15,5 +15,5 @@ tokio-postgres = { version = "0.7.6", features = [ "with-serde_json-1", "with-uuid-1", ] } -tracing = "0.1" +tracing.workspace = true uuid = { version = "1.0" } diff --git a/nexus/peer-cursor/Cargo.toml b/nexus/peer-cursor/Cargo.toml index bead6c92b0..74a2fe9de6 100644 --- a/nexus/peer-cursor/Cargo.toml +++ b/nexus/peer-cursor/Cargo.toml @@ -8,8 +8,10 @@ edition = "2021" [dependencies] anyhow = "1.0" async-trait = "0.1" +dashmap.workspace = true futures = "0.3" pgwire.workspace = true sqlparser.workspace = true tokio = { version = "1.0", features = ["full"] } +tracing.workspace = true value = { path = "../value" } diff --git a/nexus/peer-cursor/src/lib.rs b/nexus/peer-cursor/src/lib.rs index 08a7891f87..3a31531f4f 100644 --- a/nexus/peer-cursor/src/lib.rs +++ b/nexus/peer-cursor/src/lib.rs @@ -5,8 +5,11 @@ use pgwire::{api::results::FieldInfo, error::PgWireResult}; use sqlparser::ast::Statement; use value::Value; +mod manager; pub mod util; +pub use manager::CursorManager; + pub type Schema = Arc>; pub struct Record { @@ -46,3 +49,9 @@ pub trait QueryExecutor: Send + Sync { async fn execute(&self, stmt: &Statement) -> PgWireResult; async fn describe(&self, stmt: &Statement) -> PgWireResult>; } + +pub struct Cursor { + position: usize, + stream: SendableStream, + schema: Schema, +} diff --git a/nexus/peer-bigquery/src/cursor.rs b/nexus/peer-cursor/src/manager.rs similarity index 56% rename from nexus/peer-bigquery/src/cursor.rs rename to nexus/peer-cursor/src/manager.rs index 52558600ef..5384b86362 100644 --- a/nexus/peer-bigquery/src/cursor.rs +++ b/nexus/peer-cursor/src/manager.rs @@ -1,54 +1,37 @@ use dashmap::DashMap; use futures::StreamExt; -use peer_cursor::{QueryExecutor, QueryOutput, Records, Schema, SendableStream}; use pgwire::error::{ErrorInfo, PgWireError, PgWireResult}; use sqlparser::ast::Statement; -use crate::BigQueryQueryExecutor; +use crate::{Cursor, QueryExecutor, QueryOutput, Records}; -pub struct BigQueryCursor { - position: usize, - stream: SendableStream, - schema: Schema, +#[derive(Default)] +pub struct CursorManager { + cursors: DashMap, } -pub struct BigQueryCursorManager { - cursors: DashMap, -} - -impl BigQueryCursorManager { - pub fn new() -> Self { - Self { - cursors: DashMap::new(), - } - } - +impl CursorManager { pub async fn create_cursor( &self, name: &str, stmt: &Statement, - executor: &BigQueryQueryExecutor, + executor: &dyn QueryExecutor, ) -> PgWireResult<()> { - // Execute the query to obtain a stream of records let output = executor.execute(stmt).await?; match output { QueryOutput::Stream(stream) => { - // Get the schema from the stream let schema = stream.schema(); - // Create a new cursor - let cursor = BigQueryCursor { + let cursor = Cursor { position: 0, stream, schema, }; - // Store the cursor self.cursors.insert(name.to_string(), cursor); - // log the cursor and statement tracing::info!("Created cursor {} for statement '{}'", name, stmt); Ok(()) @@ -66,28 +49,23 @@ impl BigQueryCursorManager { PgWireError::UserError(Box::new(ErrorInfo::new( "ERROR".to_owned(), "fdw_error".to_owned(), - format!("[bigquery] Cursor {} does not exist", name), + format!("Cursor {} does not exist", name), ))) })?; let mut records = Vec::new(); - let prev_end = cursor.position; - let mut cursor_position = cursor.position; - { - while cursor_position - prev_end < count { - match cursor.stream.next().await { - Some(Ok(record)) => { - records.push(record); - cursor_position += 1; - tracing::info!("cusror position: {}", cursor_position); - } - Some(Err(err)) => return Err(err), - None => break, + while records.len() < count { + match cursor.stream.next().await { + Some(Ok(record)) => { + records.push(record); } + Some(Err(err)) => return Err(err), + None => break, } } - cursor.position = cursor_position; + tracing::info!("Cursor {} fetched {} records", name, records.len()); + cursor.position += records.len(); Ok(Records { records, @@ -96,8 +74,7 @@ impl BigQueryCursorManager { } pub async fn close(&self, name: &str) -> PgWireResult<()> { - // log that we are removing the cursor from bq - tracing::info!("Removing cursor {} from BigQuery", name); + tracing::info!("Removing cursor {}", name); self.cursors .remove(name) @@ -111,10 +88,8 @@ impl BigQueryCursorManager { .map(|_| ()) } - // close all the cursors pub async fn close_all_cursors(&self) -> PgWireResult> { - // log that we are removing all the cursors from bq - tracing::info!("Removing all cursors from BigQuery"); + tracing::info!("Removing all cursors"); let keys: Vec<_> = self .cursors diff --git a/nexus/peer-mysql/Cargo.toml b/nexus/peer-mysql/Cargo.toml new file mode 100644 index 0000000000..3b28572be4 --- /dev/null +++ b/nexus/peer-mysql/Cargo.toml @@ -0,0 +1,27 @@ +[package] +name = "peer-mysql" +version = "0.1.0" +edition = "2021" + +# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html + +[dependencies] +anyhow = "1.0" +async-trait = "0.1" +chrono.workspace = true +futures = { version = "0.3.28", features = ["executor"] } +mysql_async = { version = "0.34", default-features = false, features = ["minimal-rust", "rust_decimal", "chrono", "rustls-tls"] } +peer-ast = { path = "../peer-ast" } +peer-cursor = { path = "../peer-cursor" } +peer-connections = { path = "../peer-connections" } +pgwire.workspace = true +pt = { path = "../pt" } +rust_decimal.workspace = true +serde = { version = "1.0", features = ["derive"] } +serde_json = "1.0" +serde_bytes = "0.11" +sqlparser.workspace = true +tracing.workspace = true +tokio = { version = "1.0", features = ["full"] } +tokio-stream = "0.1" +value = { path = "../value" } diff --git a/nexus/peer-mysql/src/ast.rs b/nexus/peer-mysql/src/ast.rs new file mode 100644 index 0000000000..00c12b7dbf --- /dev/null +++ b/nexus/peer-mysql/src/ast.rs @@ -0,0 +1,63 @@ +use std::ops::ControlFlow; + +use peer_ast::flatten_expr_to_in_list; +use sqlparser::ast::{ + visit_expressions_mut, visit_function_arg_mut, visit_relations_mut, Array, BinaryOperator, + DataType, Expr, FunctionArgExpr, Query, +}; + +pub fn rewrite_query(peername: &str, query: &mut Query) { + visit_relations_mut(query, |table| { + // if peer name is first part of table name, remove first part + if peername.eq_ignore_ascii_case(&table.0[0].value) { + table.0.remove(0); + } + ControlFlow::<()>::Continue(()) + }); + + // postgres_fdw sends `limit 1` as `limit 1::bigint` which mysql chokes on + if let Some(Expr::Cast { expr, .. }) = &query.limit { + query.limit = Some((**expr).clone()); + } + + visit_function_arg_mut(query, |node| { + if let FunctionArgExpr::Expr(arg_expr) = node { + if let Expr::Cast { + data_type: DataType::Array(_), + .. + } = arg_expr + { + let list = + flatten_expr_to_in_list(arg_expr).expect("failed to flatten in function"); + let rewritten_array = Array { + elem: list, + named: true, + }; + *node = FunctionArgExpr::Expr(Expr::Array(rewritten_array)); + } + } + + ControlFlow::<()>::Continue(()) + }); + + // flatten ANY to IN operation overall. + visit_expressions_mut(query, |node| { + if let Expr::AnyOp { + left, + compare_op, + right, + } = node + { + if matches!(compare_op, BinaryOperator::Eq | BinaryOperator::NotEq) { + let list = flatten_expr_to_in_list(right).expect("failed to flatten"); + *node = Expr::InList { + expr: left.clone(), + list, + negated: matches!(compare_op, BinaryOperator::NotEq), + }; + } + } + + ControlFlow::<()>::Continue(()) + }); +} diff --git a/nexus/peer-mysql/src/client.rs b/nexus/peer-mysql/src/client.rs new file mode 100644 index 0000000000..7081cc6087 --- /dev/null +++ b/nexus/peer-mysql/src/client.rs @@ -0,0 +1,53 @@ +use std::sync::Arc; + +use futures::StreamExt; +use mysql_async::{self, prelude::Queryable}; +use tokio::{spawn, sync::mpsc}; + +pub enum Response { + Row(mysql_async::Row), + Schema(Arc<[mysql_async::Column]>), + Err(mysql_async::Error), +} + +pub struct Message { + pub query: String, + pub response: mpsc::Sender, +} + +#[derive(Clone)] +pub struct MyClient { + pub chan: mpsc::Sender, +} + +impl MyClient { + pub async fn new(opts: mysql_async::Opts) -> mysql_async::Result { + let mut conn = mysql_async::Conn::new(opts).await?; + let (send, mut recv) = mpsc::channel(1); + spawn(async move { + while let Some(Message { query, response }) = recv.recv().await { + match conn.query_stream(query).await { + Ok(stream) => { + response.send(Response::Schema(stream.columns())).await.ok(); + stream + .for_each_concurrent(1, |row| async { + response + .send(match row { + Ok(row) => Response::Row(row), + Err(err) => Response::Err(err), + }) + .await + .ok(); + }) + .await; + } + Err(e) => { + response.send(Response::Err(e)).await.ok(); + } + } + } + }); + + Ok(MyClient { chan: send }) + } +} diff --git a/nexus/peer-mysql/src/lib.rs b/nexus/peer-mysql/src/lib.rs new file mode 100644 index 0000000000..d57e83b932 --- /dev/null +++ b/nexus/peer-mysql/src/lib.rs @@ -0,0 +1,232 @@ +mod ast; +mod client; +mod stream; + +use std::fmt::Write; + +use peer_cursor::{ + CursorManager, CursorModification, QueryExecutor, QueryOutput, RecordStream, Schema, +}; +use pgwire::error::{ErrorInfo, PgWireError, PgWireResult}; +use pt::peerdb_peers::MySqlConfig; +use sqlparser::ast::{CloseCursor, Declare, FetchDirection, Statement}; +use stream::MyRecordStream; + +pub struct MySqlQueryExecutor { + peer_name: String, + client: client::MyClient, + cursor_manager: CursorManager, +} + +impl MySqlQueryExecutor { + pub async fn new(peer_name: String, config: &MySqlConfig) -> anyhow::Result { + let mut opts = mysql_async::OptsBuilder::default().prefer_socket(Some(false)); // prefer_socket breaks connecting to StarRocks + if !config.user.is_empty() { + opts = opts.user(Some(config.user.clone())) + } + if !config.password.is_empty() { + opts = opts.pass(Some(config.password.clone())) + } + if !config.database.is_empty() { + opts = opts.db_name(Some(config.database.clone())) + } + if !config.disable_tls { + opts = opts.ssl_opts(mysql_async::SslOpts::default()) + } + opts = opts + .setup(config.setup.clone()) + .compression(mysql_async::Compression::new(config.compression)) + .ip_or_hostname(config.host.clone()) + .tcp_port(config.port as u16); + let client = client::MyClient::new(opts.into()).await?; + + Ok(Self { + peer_name, + client, + cursor_manager: Default::default(), + }) + } + + async fn query(&self, query: String) -> PgWireResult { + MyRecordStream::query(self.client.clone(), query).await + } + + async fn query_schema(&self, query: String) -> PgWireResult { + let stream = MyRecordStream::query(self.client.clone(), query).await?; + Ok(stream.schema()) + } +} + +#[async_trait::async_trait] +impl QueryExecutor for MySqlQueryExecutor { + // #[tracing::instrument(skip(self, stmt), fields(stmt = %stmt))] + async fn execute(&self, stmt: &Statement) -> PgWireResult { + // only support SELECT statements + match stmt { + Statement::Explain { + analyze, + format, + statement, + .. + } => { + if let Statement::Query(ref query) = **statement { + let mut query = query.clone(); + ast::rewrite_query(&self.peer_name, &mut query); + let mut querystr = String::from("EXPLAIN "); + if *analyze { + querystr.push_str("ANALYZE "); + } + if let Some(format) = format { + write!(querystr, "FORMAT={} ", format).ok(); + } + write!(querystr, "{}", query).ok(); + tracing::info!("mysql rewritten query: {}", query); + + let cursor = self.query(querystr).await?; + Ok(QueryOutput::Stream(Box::pin(cursor))) + } else { + let error = format!( + "only EXPLAIN SELECT statements are supported in mysql. got: {}", + statement + ); + Err(PgWireError::UserError(Box::new(ErrorInfo::new( + "ERROR".to_owned(), + "fdw_error".to_owned(), + error, + )))) + } + } + Statement::Query(query) => { + let mut query = query.clone(); + ast::rewrite_query(&self.peer_name, &mut query); + let query = query.to_string(); + tracing::info!("mysql rewritten query: {}", query); + + let cursor = self.query(query).await?; + Ok(QueryOutput::Stream(Box::pin(cursor))) + } + Statement::Declare { stmts } => { + if stmts.len() != 1 { + Err(PgWireError::ApiError( + "peerdb only supports singular declare statements".into(), + )) + } else if let Declare { + ref names, + for_query: Some(ref query), + .. + } = stmts[0] + { + let name = &names[0]; + let mut query = query.clone(); + ast::rewrite_query(&self.peer_name, &mut query); + let query_stmt = Statement::Query(query); + self.cursor_manager + .create_cursor(&name.value, &query_stmt, self) + .await?; + + Ok(QueryOutput::Cursor(CursorModification::Created( + name.value.clone(), + ))) + } else { + Err(PgWireError::ApiError( + "peerdb only supports declare for query statements".into(), + )) + } + } + Statement::Fetch { + name, direction, .. + } => { + tracing::info!("fetching cursor for mysql: {}", name.value); + + // Attempt to extract the count from the direction + let count = match direction { + FetchDirection::ForwardAll | FetchDirection::All => usize::MAX, + FetchDirection::Next | FetchDirection::Forward { limit: None } => 1, + FetchDirection::Count { + limit: sqlparser::ast::Value::Number(n, _), + } + | FetchDirection::Forward { + limit: Some(sqlparser::ast::Value::Number(n, _)), + } => n + .parse::() + .map_err(|err| PgWireError::ApiError(err.into()))?, + _ => { + return Err(PgWireError::UserError(Box::new(ErrorInfo::new( + "ERROR".to_owned(), + "fdw_error".to_owned(), + "only FORWARD count and COUNT count are supported in FETCH".to_owned(), + )))) + } + }; + + tracing::info!("fetching {} rows", count); + + // Fetch rows from the cursor manager + let records = self.cursor_manager.fetch(&name.value, count).await?; + + // Return the fetched records as the query output + Ok(QueryOutput::Records(records)) + } + Statement::Close { cursor } => { + let closed_cursors = match cursor { + CloseCursor::All => self.cursor_manager.close_all_cursors().await?, + CloseCursor::Specific { name } => { + self.cursor_manager.close(&name.value).await?; + vec![name.value.clone()] + } + }; + Ok(QueryOutput::Cursor(CursorModification::Closed( + closed_cursors, + ))) + } + _ => { + let error = format!( + "only SELECT statements are supported in mysql. got: {}", + stmt + ); + Err(PgWireError::UserError(Box::new(ErrorInfo::new( + "ERROR".to_owned(), + "fdw_error".to_owned(), + error, + )))) + } + } + } + + // describe the output of the query + async fn describe(&self, stmt: &Statement) -> PgWireResult> { + // print the statement + tracing::info!("[mysql] describe: {}", stmt); + // only support SELECT statements + match stmt { + Statement::Query(query) => { + let mut query = query.clone(); + ast::rewrite_query(&self.peer_name, &mut query); + Ok(Some(self.query_schema(query.to_string()).await?)) + } + Statement::Declare { stmts } => { + if stmts.len() != 1 { + Err(PgWireError::ApiError( + "peerdb only supports singular declare statements".into(), + )) + } else if let Declare { + for_query: Some(ref query), + .. + } = stmts[0] + { + let query_stmt = Statement::Query(query.clone()); + self.describe(&query_stmt).await + } else { + Err(PgWireError::ApiError( + "peerdb only supports declare for query statements".into(), + )) + } + } + _ => PgWireResult::Err(PgWireError::UserError(Box::new(ErrorInfo::new( + "ERROR".to_owned(), + "fdw_error".to_owned(), + "only SELECT statements are supported in mysql".to_owned(), + )))), + } + } +} diff --git a/nexus/peer-mysql/src/stream.rs b/nexus/peer-mysql/src/stream.rs new file mode 100644 index 0000000000..13f3c03cf4 --- /dev/null +++ b/nexus/peer-mysql/src/stream.rs @@ -0,0 +1,189 @@ +use std::{ + pin::Pin, + sync::Arc, + task::{Context, Poll}, +}; + +use crate::client::{self, MyClient}; +use futures::Stream; +use mysql_async::consts::ColumnType; +use mysql_async::{Column, Row}; +use peer_cursor::{Record, RecordStream, Schema}; +use pgwire::{ + api::{ + results::{FieldFormat, FieldInfo}, + Type, + }, + error::{PgWireError, PgWireResult}, +}; +use tokio::sync::mpsc; +use tokio_stream::wrappers::ReceiverStream; +use value::Value; + +pub struct MyRecordStream { + schema: Schema, + stream: ReceiverStream, +} + +// convert ColumnType to pgwire FieldInfo's Type +fn convert_field_type(field_type: ColumnType) -> Type { + match field_type { + ColumnType::MYSQL_TYPE_NULL | ColumnType::MYSQL_TYPE_UNKNOWN => Type::VOID, + ColumnType::MYSQL_TYPE_FLOAT => Type::FLOAT4, + ColumnType::MYSQL_TYPE_DOUBLE => Type::FLOAT8, + ColumnType::MYSQL_TYPE_YEAR => Type::INT2, + ColumnType::MYSQL_TYPE_TINY => Type::INT2, + ColumnType::MYSQL_TYPE_SHORT => Type::INT2, + ColumnType::MYSQL_TYPE_INT24 => Type::INT4, + ColumnType::MYSQL_TYPE_LONG => Type::INT4, + ColumnType::MYSQL_TYPE_LONGLONG => Type::INT8, + ColumnType::MYSQL_TYPE_DECIMAL | ColumnType::MYSQL_TYPE_NEWDECIMAL => Type::NUMERIC, + ColumnType::MYSQL_TYPE_VARCHAR + | ColumnType::MYSQL_TYPE_VAR_STRING + | ColumnType::MYSQL_TYPE_STRING + | ColumnType::MYSQL_TYPE_ENUM + | ColumnType::MYSQL_TYPE_SET => Type::TEXT, + ColumnType::MYSQL_TYPE_TINY_BLOB + | ColumnType::MYSQL_TYPE_MEDIUM_BLOB + | ColumnType::MYSQL_TYPE_LONG_BLOB + | ColumnType::MYSQL_TYPE_BLOB + | ColumnType::MYSQL_TYPE_BIT + | ColumnType::MYSQL_TYPE_GEOMETRY => Type::BYTEA, + ColumnType::MYSQL_TYPE_DATE | ColumnType::MYSQL_TYPE_NEWDATE => Type::DATE, + ColumnType::MYSQL_TYPE_TIME | ColumnType::MYSQL_TYPE_TIME2 => Type::TIME, + ColumnType::MYSQL_TYPE_TIMESTAMP + | ColumnType::MYSQL_TYPE_TIMESTAMP2 + | ColumnType::MYSQL_TYPE_DATETIME + | ColumnType::MYSQL_TYPE_DATETIME2 => Type::TIMESTAMP, + ColumnType::MYSQL_TYPE_JSON => Type::JSONB, + ColumnType::MYSQL_TYPE_TYPED_ARRAY => Type::VOID, + } +} + +pub fn schema_from_columns(columns: &[Column]) -> Schema { + Arc::new( + columns + .iter() + .map(|column| { + let datatype = convert_field_type(column.column_type()); + FieldInfo::new( + column.name_str().into_owned(), + None, + None, + datatype, + FieldFormat::Text, + ) + }) + .collect(), + ) +} + +impl MyRecordStream { + pub async fn query(conn: MyClient, query: String) -> PgWireResult { + let (send, mut recv) = mpsc::channel::(1); + conn.chan + .send(client::Message { + query, + response: send, + }) + .await + .ok(); + + if let Some(first) = recv.recv().await { + match first { + client::Response::Row(..) => panic!("row received without schema"), + client::Response::Schema(schema) => Ok(MyRecordStream { + schema: schema_from_columns(&schema), + stream: ReceiverStream::new(recv), + }), + client::Response::Err(err) => Err(PgWireError::ApiError(err.into())), + } + } else { + Err(PgWireError::InvalidStartupMessage) + } + } +} + +pub fn mysql_row_to_values(row: Row) -> Vec { + use mysql_async::from_value; + let columns = row.columns(); + row.unwrap() + .into_iter() + .zip(columns.iter()) + .map(|(val, col)| { + if val == mysql_async::Value::NULL { + Value::Null + } else { + match col.column_type() { + ColumnType::MYSQL_TYPE_NULL | ColumnType::MYSQL_TYPE_UNKNOWN => Value::Null, + ColumnType::MYSQL_TYPE_TINY => Value::TinyInt(from_value(val)), + ColumnType::MYSQL_TYPE_SHORT | ColumnType::MYSQL_TYPE_YEAR => { + Value::SmallInt(from_value(val)) + } + ColumnType::MYSQL_TYPE_LONG | ColumnType::MYSQL_TYPE_INT24 => { + Value::Integer(from_value(val)) + } + ColumnType::MYSQL_TYPE_LONGLONG => Value::BigInt(from_value(val)), + ColumnType::MYSQL_TYPE_FLOAT => Value::Float(from_value(val)), + ColumnType::MYSQL_TYPE_DOUBLE => Value::Double(from_value(val)), + ColumnType::MYSQL_TYPE_DECIMAL | ColumnType::MYSQL_TYPE_NEWDECIMAL => { + Value::Numeric(from_value(val)) + } + ColumnType::MYSQL_TYPE_VARCHAR + | ColumnType::MYSQL_TYPE_VAR_STRING + | ColumnType::MYSQL_TYPE_STRING + | ColumnType::MYSQL_TYPE_ENUM + | ColumnType::MYSQL_TYPE_SET => Value::Text(from_value(val)), + ColumnType::MYSQL_TYPE_TINY_BLOB + | ColumnType::MYSQL_TYPE_MEDIUM_BLOB + | ColumnType::MYSQL_TYPE_LONG_BLOB + | ColumnType::MYSQL_TYPE_BLOB + | ColumnType::MYSQL_TYPE_BIT + | ColumnType::MYSQL_TYPE_GEOMETRY => { + Value::Binary(from_value::>(val).into()) + } + ColumnType::MYSQL_TYPE_DATE | ColumnType::MYSQL_TYPE_NEWDATE => { + Value::Date(from_value(val)) + } + ColumnType::MYSQL_TYPE_TIME | ColumnType::MYSQL_TYPE_TIME2 => { + Value::Time(from_value(val)) + } + ColumnType::MYSQL_TYPE_TIMESTAMP + | ColumnType::MYSQL_TYPE_TIMESTAMP2 + | ColumnType::MYSQL_TYPE_DATETIME + | ColumnType::MYSQL_TYPE_DATETIME2 => Value::PostgresTimestamp(from_value(val)), + ColumnType::MYSQL_TYPE_JSON => Value::JsonB(from_value(val)), + ColumnType::MYSQL_TYPE_TYPED_ARRAY => Value::Null, + } + } + }) + .collect() +} + +impl Stream for MyRecordStream { + type Item = PgWireResult; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + let row_stream = &mut self.stream; + match Pin::new(row_stream).poll_next(cx) { + Poll::Ready(Some(client::Response::Row(row))) => Poll::Ready(Some(Ok(Record { + schema: self.schema.clone(), + values: mysql_row_to_values(row), + }))), + Poll::Ready(Some(client::Response::Schema(..))) => Poll::Ready(Some(Err( + PgWireError::ApiError("second schema received".into()), + ))), + Poll::Ready(Some(client::Response::Err(e))) => { + Poll::Ready(Some(Err(PgWireError::ApiError(e.into())))) + } + Poll::Ready(None) => Poll::Ready(None), + Poll::Pending => Poll::Pending, + } + } +} + +impl RecordStream for MyRecordStream { + fn schema(&self) -> Schema { + self.schema.clone() + } +} diff --git a/nexus/peer-postgres/Cargo.toml b/nexus/peer-postgres/Cargo.toml index d9439e5e01..873baa2673 100644 --- a/nexus/peer-postgres/Cargo.toml +++ b/nexus/peer-postgres/Cargo.toml @@ -8,7 +8,7 @@ edition = "2021" [dependencies] anyhow = "1.0" async-trait = "0.1" -rust_decimal = { version = "1.30.0", features = [ "tokio-pg" ] } +rust_decimal.workspace = true bytes = "1.0" chrono.workspace = true futures = "0.3" @@ -28,6 +28,6 @@ tokio-postgres = { version = "0.7.6", features = [ "with-serde_json-1", "with-uuid-1", ] } -tracing = "0.1" +tracing.workspace = true uuid = { version = "1.0", features = ["serde", "v4"] } value = { path = "../value" } diff --git a/nexus/peer-snowflake/Cargo.toml b/nexus/peer-snowflake/Cargo.toml index bbad53cce7..05dc416c5e 100644 --- a/nexus/peer-snowflake/Cargo.toml +++ b/nexus/peer-snowflake/Cargo.toml @@ -12,7 +12,6 @@ async-trait = "0.1.57" base64 = "0.22" catalog = { path = "../catalog" } chrono.workspace = true -dashmap = "5.0" futures = "0.3" hex = "0.4" jsonwebtoken = { version = "9.0", features = ["use_pem"] } @@ -27,6 +26,6 @@ serde_json = "1.0" sha2 = "0.10" sqlparser.workspace = true tokio = { version = "1.21", features = ["full"] } -tracing = "0.1" +tracing.workspace = true ureq = { version = "2", features = ["json", "charset"] } value = { path = "../value" } diff --git a/nexus/peer-snowflake/src/ast.rs b/nexus/peer-snowflake/src/ast.rs index 3dddd577df..0934ec5592 100644 --- a/nexus/peer-snowflake/src/ast.rs +++ b/nexus/peer-snowflake/src/ast.rs @@ -5,8 +5,7 @@ use sqlparser::ast::{ FunctionArg, FunctionArgExpr, Ident, JsonOperator, ObjectName, Query, Statement, TimezoneInfo, }; -#[derive(Default)] -pub struct SnowflakeAst {} +pub struct SnowflakeAst; impl SnowflakeAst { pub fn rewrite(&self, query: &mut Query) -> anyhow::Result<()> { diff --git a/nexus/peer-snowflake/src/cursor.rs b/nexus/peer-snowflake/src/cursor.rs deleted file mode 100644 index 318a6d04d8..0000000000 --- a/nexus/peer-snowflake/src/cursor.rs +++ /dev/null @@ -1,124 +0,0 @@ -use crate::SnowflakeQueryExecutor; -use dashmap::DashMap; -use futures::StreamExt; -use peer_cursor::{QueryExecutor, QueryOutput, Records, Schema, SendableStream}; -use pgwire::error::{ErrorInfo, PgWireError, PgWireResult}; -use sqlparser::ast::Statement; - -pub struct SnowflakeCursor { - position: usize, - stream: SendableStream, - schema: Schema, -} - -pub struct SnowflakeCursorManager { - cursors: DashMap, -} - -impl SnowflakeCursorManager { - pub fn new() -> Self { - Self { - cursors: DashMap::new(), - } - } - pub async fn create_cursor( - &self, - name: &str, - stmt: &Statement, - executor: &SnowflakeQueryExecutor, - ) -> PgWireResult<()> { - // Execute the query to obtain a stream of records - let output = executor.execute(stmt).await?; - - match output { - QueryOutput::Stream(stream) => { - // Get the schema from the stream - let schema = stream.schema(); - - // Create a new cursor - let cursor = SnowflakeCursor { - position: 0, - stream, - schema, - }; - - // Store the cursor - self.cursors.insert(name.to_string(), cursor); - - // log the cursor and statement - tracing::info!("Created cursor {} for statement '{}'", name, stmt); - - Ok(()) - } - _ => Err(PgWireError::UserError(Box::new(ErrorInfo::new( - "ERROR".to_owned(), - "fdw_error".to_owned(), - "Only SELECT queries can be used with cursors".to_owned(), - )))), - } - } - - pub async fn fetch(&self, name: &str, count: usize) -> PgWireResult { - let mut cursor = self.cursors.get_mut(name).ok_or_else(|| { - PgWireError::UserError(Box::new(ErrorInfo::new( - "ERROR".to_owned(), - "fdw_error".to_owned(), - format!("[snowflake] Cursor {} does not exist", name), - ))) - })?; - - let mut records = Vec::new(); - let prev_end = cursor.position; - let mut cursor_position = cursor.position; - { - while cursor_position - prev_end < count { - match cursor.stream.next().await { - Some(Ok(record)) => { - records.push(record); - cursor_position += 1; - tracing::info!("cursor position: {}", cursor_position); - } - Some(Err(err)) => return Err(err), - None => break, - } - } - } - - cursor.position = cursor_position; - - Ok(Records { - records, - schema: cursor.schema.clone(), - }) - } - - pub async fn close(&self, name: &str) -> PgWireResult<()> { - // log that we are removing the cursor from bq - tracing::info!("Removing cursor {} from Snowflake", name); - - self.cursors - .remove(name) - .ok_or_else(|| { - PgWireError::UserError(Box::new(ErrorInfo::new( - "ERROR".to_owned(), - "fdw_error".to_owned(), - format!("Cursor {} does not exist", name), - ))) - }) - .map(|_| ()) - } - - // close all the cursors - pub async fn close_all_cursors(&self) -> PgWireResult> { - // log that we are removing all the cursors from bq - tracing::info!("Removing all cursors from Snowflake"); - - let keys: Vec<_> = self - .cursors - .iter() - .map(|entry| entry.key().clone()) - .collect(); - self.cursors.clear(); - Ok(keys) - } -} diff --git a/nexus/peer-snowflake/src/lib.rs b/nexus/peer-snowflake/src/lib.rs index a940688b7e..d04a369d32 100644 --- a/nexus/peer-snowflake/src/lib.rs +++ b/nexus/peer-snowflake/src/lib.rs @@ -1,7 +1,6 @@ use anyhow::Context; use async_recursion::async_recursion; -use cursor::SnowflakeCursorManager; -use peer_cursor::{CursorModification, QueryExecutor, QueryOutput, Schema}; +use peer_cursor::{CursorManager, CursorModification, QueryExecutor, QueryOutput, Schema}; use pgwire::error::{ErrorInfo, PgWireError, PgWireResult}; use std::cmp::min; use std::time::Duration; @@ -12,7 +11,7 @@ use pt::peerdb_peers::SnowflakeConfig; use reqwest::{header, StatusCode}; use secrecy::ExposeSecret; use serde::{Deserialize, Serialize}; -use sqlparser::ast::{CloseCursor, FetchDirection, Query, Statement}; +use sqlparser::ast::{CloseCursor, Declare, FetchDirection, Query, Statement}; use tokio::time::sleep; use tracing::info; @@ -20,7 +19,6 @@ use crate::stream::SnowflakeSchema; mod ast; mod auth; -mod cursor; mod stream; const DEFAULT_REFRESH_THRESHOLD: u64 = 3000; @@ -101,7 +99,7 @@ pub struct SnowflakeQueryExecutor { auth: SnowflakeAuth, query_timeout: u64, reqwest_client: reqwest::Client, - cursor_manager: SnowflakeCursorManager, + cursor_manager: CursorManager, } enum QueryAttemptResult { @@ -127,7 +125,7 @@ impl SnowflakeQueryExecutor { .gzip(true) .default_headers(default_headers) .build()?; - let cursor_manager = SnowflakeCursorManager::new(); + Ok(Self { config: config.clone(), partition_number: 0, @@ -146,7 +144,7 @@ impl SnowflakeQueryExecutor { )?, query_timeout: config.query_timeout, reqwest_client, - cursor_manager, + cursor_manager: Default::default(), }) } @@ -200,8 +198,7 @@ impl SnowflakeQueryExecutor { pub async fn query(&self, query: &Query) -> PgWireResult { let mut query = query.clone(); - let ast = ast::SnowflakeAst::default(); - let _ = ast.rewrite(&mut query); + let _ = ast::SnowflakeAst.rewrite(&mut query); let query_str: String = query.to_string(); info!("Processing SnowFlake query: {}", query_str); @@ -301,8 +298,7 @@ impl QueryExecutor for SnowflakeQueryExecutor { Statement::Query(query) => { let mut new_query = query.clone(); - let snowflake_ast = ast::SnowflakeAst::default(); - snowflake_ast + ast::SnowflakeAst .rewrite(&mut new_query) .context("unable to rewrite query") .map_err(|err| PgWireError::ApiError(err.into()))?; @@ -318,15 +314,31 @@ impl QueryExecutor for SnowflakeQueryExecutor { ); Ok(QueryOutput::Stream(Box::pin(cursor))) } - Statement::Declare { name, query, .. } => { - let query_stmt = Statement::Query(query.clone()); - self.cursor_manager - .create_cursor(&name.value, &query_stmt, self) - .await?; - - Ok(QueryOutput::Cursor(CursorModification::Created( - name.value.clone(), - ))) + Statement::Declare { stmts } => { + if stmts.len() != 1 { + Err(PgWireError::ApiError( + "peerdb only supports singular declare statements".into(), + )) + } else if let Declare { + ref names, + for_query: Some(ref query), + .. + } = stmts[0] + { + let name = &names[0]; + let query_stmt = Statement::Query(query.clone()); + self.cursor_manager + .create_cursor(&name.value, &query_stmt, self) + .await?; + + Ok(QueryOutput::Cursor(CursorModification::Created( + name.value.clone(), + ))) + } else { + Err(PgWireError::ApiError( + "peerdb only supports declare for query statements".into(), + )) + } } Statement::Fetch { name, direction, .. @@ -335,12 +347,16 @@ impl QueryExecutor for SnowflakeQueryExecutor { // Attempt to extract the count from the direction let count = match direction { + FetchDirection::ForwardAll | FetchDirection::All => usize::MAX, + FetchDirection::Next | FetchDirection::Forward { limit: None } => 1, FetchDirection::Count { limit: sqlparser::ast::Value::Number(n, _), } | FetchDirection::Forward { limit: Some(sqlparser::ast::Value::Number(n, _)), - } => n.parse::(), + } => n + .parse::() + .map_err(|err| PgWireError::ApiError(err.into()))?, _ => { return Err(PgWireError::UserError(Box::new(ErrorInfo::new( "ERROR".to_owned(), @@ -350,12 +366,6 @@ impl QueryExecutor for SnowflakeQueryExecutor { } }; - // If parsing the count resulted in an error, return an internal error - let count = match count { - Ok(c) => c, - Err(err) => return Err(PgWireError::ApiError(err.into())), - }; - tracing::info!("fetching {} rows", count); // Fetch rows from the cursor manager @@ -365,14 +375,11 @@ impl QueryExecutor for SnowflakeQueryExecutor { Ok(QueryOutput::Records(records)) } Statement::Close { cursor } => { - let mut closed_cursors = vec![]; - match cursor { - CloseCursor::All => { - closed_cursors = self.cursor_manager.close_all_cursors().await?; - } + let closed_cursors = match cursor { + CloseCursor::All => self.cursor_manager.close_all_cursors().await?, CloseCursor::Specific { name } => { self.cursor_manager.close(&name.value).await?; - closed_cursors.push(name.value.clone()); + vec![name.value.clone()] } }; Ok(QueryOutput::Cursor(CursorModification::Closed( @@ -397,22 +404,33 @@ impl QueryExecutor for SnowflakeQueryExecutor { match stmt { Statement::Query(query) => { let mut new_query = query.clone(); - let sf_ast = ast::SnowflakeAst::default(); - sf_ast + ast::SnowflakeAst .rewrite(&mut new_query) .context("unable to rewrite query") .map_err(|err| PgWireError::ApiError(err.into()))?; - // new_query.limit = Some(Expr::Value(Value::Number("1".to_owned(), false))); - let result_set = self.query(&new_query).await?; let schema = SnowflakeSchema::from_result_set(&result_set); Ok(Some(schema.schema())) } - Statement::Declare { query, .. } => { - let query_stmt = Statement::Query(query.clone()); - self.describe(&query_stmt).await + Statement::Declare { stmts } => { + if stmts.len() != 1 { + Err(PgWireError::ApiError( + "peerdb only supports singular declare statements".into(), + )) + } else if let Declare { + for_query: Some(ref query), + .. + } = stmts[0] + { + let query_stmt = Statement::Query(query.clone()); + self.describe(&query_stmt).await + } else { + Err(PgWireError::ApiError( + "peerdb only supports declare for query statements".into(), + )) + } } _ => PgWireResult::Err(PgWireError::UserError(Box::new(ErrorInfo::new( "ERROR".to_owned(), diff --git a/nexus/postgres-connection/Cargo.toml b/nexus/postgres-connection/Cargo.toml index e4df6aa275..cb86472016 100644 --- a/nexus/postgres-connection/Cargo.toml +++ b/nexus/postgres-connection/Cargo.toml @@ -13,4 +13,4 @@ urlencoding = "2" tokio-postgres = "0.7.2" tokio-postgres-rustls = "0.12" tokio = { version = "1", features = ["full"] } -tracing = "0.1" +tracing.workspace = true diff --git a/nexus/pt/src/lib.rs b/nexus/pt/src/lib.rs index 23487c80b3..137cb3909f 100644 --- a/nexus/pt/src/lib.rs +++ b/nexus/pt/src/lib.rs @@ -26,6 +26,7 @@ impl From for DbType { PeerType::Postgres => DbType::Postgres, PeerType::S3 => DbType::S3, PeerType::SQLServer => DbType::Sqlserver, + PeerType::MySql => DbType::Mysql, PeerType::Kafka => DbType::Kafka, PeerType::Eventhubs => DbType::Eventhubs, PeerType::PubSub => DbType::Pubsub, diff --git a/nexus/server/Cargo.toml b/nexus/server/Cargo.toml index 8c87a2480d..fc12071bc4 100644 --- a/nexus/server/Cargo.toml +++ b/nexus/server/Cargo.toml @@ -37,13 +37,14 @@ bytes = "1.0" catalog = { path = "../catalog" } clap = { version = "4.0", features = ["derive", "env"] } console-subscriber = "0.2" -dashmap = "5.0" +dashmap.workspace = true dotenvy = "0.15.7" flow-rs = { path = "../flow-rs" } futures = { version = "0.3.28", features = ["executor"] } peer-bigquery = { path = "../peer-bigquery" } peer-connections = { path = "../peer-connections" } peer-cursor = { path = "../peer-cursor" } +peer-mysql = { path = "../peer-mysql" } peer-postgres = { path = "../peer-postgres" } peer-snowflake = { path = "../peer-snowflake" } peerdb-parser = { path = "../parser" } @@ -54,7 +55,7 @@ serde_json = "1.0" rand = "0.8" time = "0.3" tokio = { version = "1", features = ["full"] } -tracing = "0.1" +tracing.workspace = true tracing-appender = "0.2" tracing-subscriber = "0.3" uuid = "1.0" @@ -62,4 +63,4 @@ cargo-deb = "2.0" [dev-dependencies] postgres = "0.19.4" -sha256 = "1.0.3" +similar = "2" diff --git a/nexus/server/src/main.rs b/nexus/server/src/main.rs index d5cb4c0dcc..51d2999890 100644 --- a/nexus/server/src/main.rs +++ b/nexus/server/src/main.rs @@ -14,7 +14,6 @@ use cursor::PeerCursors; use dashmap::{mapref::entry::Entry as DashEntry, DashMap}; use flow_rs::grpc::{FlowGrpcClient, PeerValidationResult}; use futures::join; -use peer_bigquery::BigQueryQueryExecutor; use peer_connections::{PeerConnectionTracker, PeerConnections}; use peer_cursor::{ util::{records_to_query_response, sendable_stream_to_query_response}, @@ -27,9 +26,12 @@ use pgwire::{ md5pass::{hash_md5_password, MakeMd5PasswordAuthStartupHandler}, AuthSource, LoginInfo, Password, ServerParameterProvider, }, - portal::{Format, Portal}, - query::{ExtendedQueryHandler, SimpleQueryHandler, StatementOrPortal}, - results::{DescribeResponse, Response, Tag}, + portal::Portal, + query::{ExtendedQueryHandler, SimpleQueryHandler}, + results::{ + DescribePortalResponse, DescribeResponse, DescribeStatementResponse, Response, Tag, + }, + stmt::StoredStatement, ClientInfo, MakeHandler, Type, }, error::{ErrorInfo, PgWireError, PgWireResult}, @@ -67,10 +69,7 @@ impl AuthSource for FixedPasswordAuthSource { let salt = rand::thread_rng().gen::<[u8; 4]>(); let password = &self.password; let hash_password = hash_md5_password(login_info.user().unwrap_or(""), password, &salt); - Ok(Password::new( - Some(salt.to_vec()), - hash_password.as_bytes().to_vec(), - )) + Ok(Password::new(Some(salt.to_vec()), Vec::from(hash_password))) } } @@ -125,7 +124,7 @@ impl NexusBackend { Ok(vec![res]) } QueryOutput::Cursor(cm) => { - tracing::info!("cursor modification: {:?}", cm); + tracing::info!("cursor modification: {:?} {}", cm, peer_holder.is_some()); let mut peer_cursors = self.peer_cursors.lock().await; match cm { peer_cursor::CursorModification::Created(cursor_name) => { @@ -145,13 +144,6 @@ impl NexusBackend { } } - fn is_peer_validity_supported(peer_type: i32) -> bool { - let unsupported_peer_types = [ - 11, // EVENTHUBS - ]; - !unsupported_peer_types.contains(&peer_type) - } - async fn check_for_mirror( catalog: &Catalog, flow_name: &str, @@ -192,10 +184,14 @@ impl NexusBackend { async fn validate_peer<'a>(&self, peer: &Peer) -> anyhow::Result<()> { //if flow handler does not exist, skip validation - if self.flow_handler.is_none() { + let mut flow_handler = if let Some(ref flow_handler) = self.flow_handler { + flow_handler.as_ref() + } else { return Ok(()); } - let mut flow_handler = self.flow_handler.as_ref().unwrap().lock().await; + .lock() + .await; + let validate_request = pt::peerdb_route::ValidatePeerRequest { peer: Some(Peer { name: peer.name.clone(), @@ -361,16 +357,13 @@ impl NexusBackend { peer, if_not_exists: _, } => { - let peer_type = peer.r#type; - if Self::is_peer_validity_supported(peer_type) { - self.validate_peer(peer).await.map_err(|e| { - PgWireError::UserError(Box::new(ErrorInfo::new( - "ERROR".to_owned(), - "internal_error".to_owned(), - e.to_string(), - ))) - })?; - } + self.validate_peer(peer).await.map_err(|e| { + PgWireError::UserError(Box::new(ErrorInfo::new( + "ERROR".to_owned(), + "internal_error".to_owned(), + e.to_string(), + ))) + })?; self.catalog.create_peer(peer.as_ref()).await.map_err(|e| { PgWireError::UserError(Box::new(ErrorInfo::new( @@ -787,6 +780,11 @@ impl NexusBackend { self.execute_statement(executor.as_ref(), &stmt, None).await } + NexusStatement::Rollback { stmt } => { + self.execute_statement(self.catalog.as_ref(), &stmt, None) + .await + } + NexusStatement::Empty => Ok(vec![Response::EmptyQuery]), } } @@ -832,7 +830,7 @@ impl NexusBackend { DashEntry::Vacant(entry) => { let executor: Arc = match &peer.config { Some(Config::BigqueryConfig(ref c)) => { - let executor = BigQueryQueryExecutor::new( + let executor = peer_bigquery::BigQueryQueryExecutor::new( peer.name.clone(), c, self.peer_connections.clone(), @@ -840,6 +838,11 @@ impl NexusBackend { .await?; Arc::new(executor) } + Some(Config::MysqlConfig(ref c)) => { + let executor = + peer_mysql::MySqlQueryExecutor::new(peer.name.clone(), c).await?; + Arc::new(executor) + } Some(Config::PostgresConfig(ref c)) => { let executor = peer_postgres::PostgresQueryExecutor::new(peer.name.clone(), c).await?; @@ -859,6 +862,61 @@ impl NexusBackend { } }) } + + async fn do_describe(&self, stmt: &NexusParsedStatement) -> PgWireResult> { + tracing::info!("[eqp] do_describe: {}", stmt.query); + let stmt = &stmt.statement; + match stmt { + NexusStatement::PeerDDL { .. } => Ok(None), + NexusStatement::PeerCursor { .. } => Ok(None), + NexusStatement::Empty => Ok(None), + NexusStatement::Rollback { .. } => Ok(None), + NexusStatement::PeerQuery { stmt, assoc } => { + let schema: Option = match assoc { + QueryAssociation::Peer(peer) => match &peer.config { + Some(Config::BigqueryConfig(_)) => { + let executor = self.get_peer_executor(peer).await.map_err(|err| { + PgWireError::ApiError( + format!("unable to get peer executor: {:?}", err).into(), + ) + })?; + executor.describe(stmt).await? + } + Some(Config::MysqlConfig(_)) => { + let executor = self.get_peer_executor(peer).await.map_err(|err| { + PgWireError::ApiError( + format!("unable to get peer executor: {:?}", err).into(), + ) + })?; + executor.describe(stmt).await? + } + Some(Config::PostgresConfig(_)) => { + let executor = self.get_peer_executor(peer).await.map_err(|err| { + PgWireError::ApiError( + format!("unable to get peer executor: {:?}", err).into(), + ) + })?; + executor.describe(stmt).await? + } + Some(Config::SnowflakeConfig(_)) => { + let executor = self.get_peer_executor(peer).await.map_err(|err| { + PgWireError::ApiError( + format!("unable to get peer executor: {:?}", err).into(), + ) + })?; + executor.describe(stmt).await? + } + _ => { + panic!("peer type not supported: {:?}", peer) + } + }, + QueryAssociation::Catalog => self.catalog.describe(stmt).await?, + }; + + Ok(if self.peerdb_fdw_mode { None } else { schema }) + } + } + } } #[async_trait] @@ -955,91 +1013,38 @@ impl ExtendedQueryHandler for NexusBackend { } } - async fn do_describe( + async fn do_describe_portal( &self, _client: &mut C, - target: StatementOrPortal<'_, Self::Statement>, - ) -> PgWireResult + target: &Portal, + ) -> PgWireResult where C: ClientInfo + Unpin + Send + Sync, { - let (param_types, stmt, _format) = match target { - StatementOrPortal::Statement(stmt) => { - let param_types = Some(&stmt.parameter_types); - (param_types, &stmt.statement, &Format::UnifiedBinary) - } - StatementOrPortal::Portal(portal) => ( - None, - &portal.statement.statement, - &portal.result_column_format, - ), - }; + Ok( + if let Some(schema) = self.do_describe(&target.statement.statement).await? { + DescribePortalResponse::new((*schema).clone()) + } else { + DescribePortalResponse::no_data() + }, + ) + } - tracing::info!("[eqp] do_describe: {}", stmt.query); - let stmt = &stmt.statement; - match stmt { - NexusStatement::PeerDDL { .. } => Ok(DescribeResponse::no_data()), - NexusStatement::PeerCursor { .. } => Ok(DescribeResponse::no_data()), - NexusStatement::Empty => Ok(DescribeResponse::no_data()), - NexusStatement::PeerQuery { stmt, assoc } => { - let schema: Option = match assoc { - QueryAssociation::Peer(peer) => { - // if the peer is of type bigquery, let us route the query to bq. - match &peer.config { - Some(Config::BigqueryConfig(_)) => { - let executor = - self.get_peer_executor(peer).await.map_err(|err| { - PgWireError::ApiError( - format!("unable to get peer executor: {:?}", err) - .into(), - ) - })?; - executor.describe(stmt).await? - } - Some(Config::PostgresConfig(_)) => { - let executor = - self.get_peer_executor(peer).await.map_err(|err| { - PgWireError::ApiError( - format!("unable to get peer executor: {:?}", err) - .into(), - ) - })?; - executor.describe(stmt).await? - } - Some(Config::SnowflakeConfig(_)) => { - let executor = - self.get_peer_executor(peer).await.map_err(|err| { - PgWireError::ApiError( - format!("unable to get peer executor: {:?}", err) - .into(), - ) - })?; - executor.describe(stmt).await? - } - Some(_peer) => { - panic!("peer type not supported: {:?}", peer) - } - None => { - panic!("peer type not supported: {:?}", peer) - } - } - } - QueryAssociation::Catalog => self.catalog.describe(stmt).await?, - }; - if let Some(described_schema) = schema { - if self.peerdb_fdw_mode { - Ok(DescribeResponse::no_data()) - } else { - Ok(DescribeResponse::new( - param_types.cloned(), - (*described_schema).clone(), - )) - } - } else { - Ok(DescribeResponse::no_data()) - } - } - } + async fn do_describe_statement( + &self, + _client: &mut C, + target: &StoredStatement, + ) -> PgWireResult + where + C: ClientInfo + Unpin + Send + Sync, + { + Ok( + if let Some(schema) = self.do_describe(&target.statement).await? { + DescribeStatementResponse::new(target.parameter_types.clone(), (*schema).clone()) + } else { + DescribeStatementResponse::no_data() + }, + ) } } diff --git a/nexus/server/tests/results/expected/bq.sql.out b/nexus/server/tests/results/expected/bq.sql.out index 2c186752a4..914bfd8ec9 100644 --- a/nexus/server/tests/results/expected/bq.sql.out +++ b/nexus/server/tests/results/expected/bq.sql.out @@ -151,7 +151,7 @@ test 1 2 2022-06-22 23:35:53.000000 -true +t 12 1 1 diff --git a/nexus/server/tests/results/expected/postgres.sql.out b/nexus/server/tests/results/expected/postgres.sql.out index 7f13da52f5..ab820deeb0 100644 --- a/nexus/server/tests/results/expected/postgres.sql.out +++ b/nexus/server/tests/results/expected/postgres.sql.out @@ -5,8 +5,8 @@ 4294967295 281474976710656 1 -false -true +f +t 2005-10-10 2005-10-10 00:00:00.000000 2005-10-11 00:00:00.000000 @@ -51,17 +51,17 @@ deadbeef 255.255.255.0 192.168.0.0/24 17 -true +t 26 17 -true +t 26 24 -true +t 26 24 -true +t 26 1 -true +t 1 diff --git a/nexus/server/tests/server_test.rs b/nexus/server/tests/server_test.rs index 7738e0115c..49ec206e93 100644 --- a/nexus/server/tests/server_test.rs +++ b/nexus/server/tests/server_test.rs @@ -1,4 +1,3 @@ -use postgres::{Client, NoTls, SimpleQueryMessage}; use std::{ fs::{read_dir, File}, io::{prelude::*, BufReader, Write}, @@ -7,7 +6,12 @@ use std::{ thread, time::Duration, }; + +use postgres::{Client, NoTls, SimpleQueryMessage}; +use similar::TextDiff; + mod create_peers; + fn input_files() -> Vec { let sql_directory = read_dir("tests/sql").unwrap(); sql_directory @@ -16,8 +20,9 @@ fn input_files() -> Vec { sql_file .path() .file_name() - .and_then(|n| n.to_str().map(String::from)) + .and_then(|n| n.to_str()) .filter(|n| n.ends_with(".sql")) + .map(String::from) }) }) .collect::>() @@ -112,8 +117,8 @@ fn server_test() { let expected_output_path = ["tests/results/expected/", file, ".out"].concat(); let mut output_file = File::create(["tests/results/actual/", file, ".out"].concat()) .expect("Unable to create result file"); + for query in queries { - let mut output = Vec::new(); dbg!(query.as_str()); // filter out comments and empty lines @@ -132,59 +137,43 @@ fn server_test() { match res[0] { // Fetch column names for the output SimpleQueryMessage::Row(ref simplerow) => { - for column in simplerow.columns() { - column_names.push(column.name()); - } + column_names.extend(simplerow.columns().iter().map(|column| column.name())); } SimpleQueryMessage::CommandComplete(_x) => (), _ => (), }; - res.iter().for_each(|row| { - for column_head in &column_names { - let row_parse = match row { - SimpleQueryMessage::Row(ref simplerow) => simplerow.get(column_head), - SimpleQueryMessage::CommandComplete(_x) => None, - _ => None, - }; - - let row_value = match row_parse { - None => { - continue; + let mut output = String::new(); + for row in res { + if let SimpleQueryMessage::Row(simplerow) = row { + for idx in 0..simplerow.len() { + if let Some(val) = simplerow.get(idx) { + output.push_str(val); + output.push('\n'); } - Some(x) => x, - }; - output.push(row_value.to_owned()); + } } - }); - - for i in &output { - let output_line = (*i).as_bytes(); - output_file - .write_all(output_line) - .expect("Unable to write query output"); - output_file - .write_all("\n".as_bytes()) - .expect("Output file write failure"); } + output_file + .write_all(output.as_bytes()) + .expect("Unable to write query output"); // flush the output file output_file.flush().expect("Unable to flush output file"); } - // Compare hash of expected and obtained files - let obtained_file = std::fs::read(&actual_output_path).unwrap(); - let expected_file = std::fs::read(&expected_output_path).unwrap(); - let obtained_hash = sha256::digest(obtained_file.as_slice()); - let expected_hash = sha256::digest(expected_file.as_slice()); - + let obtained_file = std::fs::read_to_string(&actual_output_path).unwrap(); + let expected_file = std::fs::read_to_string(&expected_output_path).unwrap(); // if there is a mismatch, print the diff, along with the path. - if obtained_hash != expected_hash { - tracing::info!("expected: {expected_output_path}"); - tracing::info!("obtained: {actual_output_path}"); - } + if obtained_file != expected_file { + tracing::info!("failed: {file}"); + let diff = TextDiff::from_lines(&expected_file, &obtained_file); + for change in diff.iter_all_changes() { + print!("{}{}", change.tag(), change); + } - assert_eq!(obtained_hash, expected_hash); + panic!("result didn't match expected output"); + } }); } diff --git a/nexus/value/Cargo.toml b/nexus/value/Cargo.toml index 2d41188e76..4071b1dea2 100644 --- a/nexus/value/Cargo.toml +++ b/nexus/value/Cargo.toml @@ -9,7 +9,6 @@ edition = "2021" base64 = "0.22" bytes = "1.1" chrono.workspace = true -rust_decimal = { version = "1.30.0", features = [ "tokio-pg" ] } serde = { version = "1.0", features = ["derive"] } serde_json = "1.0" hex = "0.4" @@ -17,4 +16,5 @@ pgwire.workspace = true postgres = { version = "0.19", features = ["with-chrono-0_4"] } postgres-inet = "0.19.0" postgres-types = { version = "0.2.5", features = ["array-impls"] } +rust_decimal.workspace = true uuid = { version = "1.0", features = ["serde", "v4"] } diff --git a/protos/peers.proto b/protos/peers.proto index 69d748c5a4..13cabf58cf 100644 --- a/protos/peers.proto +++ b/protos/peers.proto @@ -126,6 +126,17 @@ message SqlServerConfig { string database = 5; } +message MySqlConfig { + string host = 1; + uint32 port = 2; + string user = 3; + string password = 4; + string database = 5; + repeated string setup = 6; + uint32 compression = 7; + bool disable_tls = 8; +} + message KafkaConfig { repeated string servers = 1; string username = 2; @@ -158,6 +169,7 @@ enum DBType { POSTGRES = 3; S3 = 5; SQLSERVER = 6; + MYSQL = 7; CLICKHOUSE = 8; KAFKA = 9; PUBSUB = 10; @@ -173,7 +185,6 @@ message Peer { BigqueryConfig bigquery_config = 4; MongoConfig mongo_config = 5; PostgresConfig postgres_config = 6; - EventHubConfig eventhub_config = 7; S3Config s3_config = 8; SqlServerConfig sqlserver_config = 9; EventHubGroupConfig eventhub_group_config = 10; @@ -181,5 +192,6 @@ message Peer { KafkaConfig kafka_config = 12; PubSubConfig pubsub_config = 13; ElasticsearchConfig elasticsearch_config = 14; + MySqlConfig mysql_config = 15; } } diff --git a/ui/app/api/peers/getTruePeer.ts b/ui/app/api/peers/getTruePeer.ts index e21323a07b..c8b5c55a37 100644 --- a/ui/app/api/peers/getTruePeer.ts +++ b/ui/app/api/peers/getTruePeer.ts @@ -6,6 +6,7 @@ import { EventHubConfig, EventHubGroupConfig, KafkaConfig, + MySqlConfig, Peer, PostgresConfig, PubSubConfig, @@ -26,6 +27,7 @@ export const getTruePeer = (peer: CatalogPeer) => { | EventHubConfig | EventHubGroupConfig | KafkaConfig + | MySqlConfig | PostgresConfig | PubSubConfig | S3Config @@ -45,10 +47,6 @@ export const getTruePeer = (peer: CatalogPeer) => { config = PostgresConfig.decode(options); newPeer.postgresConfig = config; break; - case 4: - config = EventHubConfig.decode(options); - newPeer.eventhubConfig = config; - break; case 5: config = S3Config.decode(options); newPeer.s3Config = config; @@ -58,8 +56,8 @@ export const getTruePeer = (peer: CatalogPeer) => { newPeer.sqlserverConfig = config; break; case 7: - config = EventHubGroupConfig.decode(options); - newPeer.eventhubGroupConfig = config; + config = MySqlConfig.decode(options); + newPeer.mysqlConfig = config; break; case 8: config = ClickhouseConfig.decode(options); From 5a1734f09fbb64e110d19e259beadd7552c0d224 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Wed, 1 May 2024 14:21:37 +0000 Subject: [PATCH 10/41] chore: update dependencies (#1660) --- flow/go.mod | 35 ++-- flow/go.sum | 86 ++++---- nexus/Cargo.lock | 28 +-- ui/package-lock.json | 472 +++++++++++++++++++++---------------------- ui/package.json | 26 +-- 5 files changed, 320 insertions(+), 327 deletions(-) diff --git a/flow/go.mod b/flow/go.mod index d3bc41e81a..71ab6c6557 100644 --- a/flow/go.mod +++ b/flow/go.mod @@ -4,13 +4,13 @@ go 1.22.2 require ( cloud.google.com/go v0.112.2 - cloud.google.com/go/bigquery v1.60.0 + cloud.google.com/go/bigquery v1.61.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.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.1 + github.com/ClickHouse/clickhouse-go/v2 v2.23.2 github.com/PeerDB-io/glua64 v1.0.1 github.com/PeerDB-io/gluabit32 v1.0.2 github.com/PeerDB-io/gluaflatbuffers v1.0.1 @@ -22,7 +22,7 @@ require ( github.com/aws/aws-sdk-go-v2/credentials v1.17.11 github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.16.15 github.com/aws/aws-sdk-go-v2/service/s3 v1.53.1 - github.com/aws/aws-sdk-go-v2/service/ses v1.22.4 + github.com/aws/aws-sdk-go-v2/service/ses v1.22.5 github.com/aws/aws-sdk-go-v2/service/sns v1.29.4 github.com/aws/smithy-go v1.20.2 github.com/cockroachdb/pebble v1.1.0 @@ -33,7 +33,7 @@ require ( github.com/jackc/pgerrcode v0.0.0-20240316143900-6e2875d9b438 github.com/jackc/pglogrepl v0.0.0-20240307033717-828fbfe908e9 github.com/jackc/pgx/v5 v5.5.5 - github.com/jmoiron/sqlx v1.3.5 + github.com/jmoiron/sqlx v1.4.0 github.com/joho/godotenv v1.5.1 github.com/klauspost/compress v1.17.8 github.com/lib/pq v1.10.9 @@ -48,7 +48,7 @@ require ( github.com/twmb/franz-go/plugin/kslog v1.0.0 github.com/twpayne/go-geos v0.17.1 github.com/urfave/cli/v3 v3.0.0-alpha9 - github.com/youmark/pkcs8 v0.0.0-20201027041543-1326539a0a0a + github.com/youmark/pkcs8 v0.0.0-20240424034433-3c2c7870ae76 github.com/yuin/gopher-lua v1.1.1 go.temporal.io/api v1.32.0 go.temporal.io/sdk v1.26.1 @@ -56,21 +56,20 @@ require ( golang.org/x/crypto v0.22.0 golang.org/x/mod v0.17.0 golang.org/x/sync v0.7.0 - google.golang.org/api v0.176.0 - google.golang.org/genproto/googleapis/api v0.0.0-20240415180920-8c6c420018be + google.golang.org/api v0.177.0 + google.golang.org/genproto/googleapis/api v0.0.0-20240429193739-8cf5692501f6 google.golang.org/grpc v1.63.2 - google.golang.org/protobuf v1.33.0 + google.golang.org/protobuf v1.34.0 ) require ( - cloud.google.com/go/auth v0.2.2 // indirect - cloud.google.com/go/auth/oauth2adapt v0.2.1 // indirect + cloud.google.com/go/auth v0.3.0 // indirect + cloud.google.com/go/auth/oauth2adapt v0.2.2 // indirect github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4 // indirect github.com/99designs/keyring v1.2.2 // indirect github.com/ClickHouse/ch-go v0.61.5 // indirect github.com/DataDog/zstd v1.5.5 // indirect github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c // indirect - github.com/apache/arrow/go/v14 v14.0.2 // indirect github.com/apache/arrow/go/v15 v15.0.2 // indirect github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.1 // indirect github.com/aws/aws-sdk-go-v2/internal/ini v1.8.0 // indirect @@ -109,11 +108,11 @@ require ( github.com/segmentio/asm v1.2.0 // indirect github.com/sirupsen/logrus v1.9.3 // indirect github.com/twmb/franz-go/pkg/kmsg v1.7.0 // indirect - go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.50.0 // indirect - go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.50.0 // indirect - go.opentelemetry.io/otel v1.25.0 // indirect - go.opentelemetry.io/otel/metric v1.25.0 // indirect - go.opentelemetry.io/otel/trace v1.25.0 // indirect + go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.51.0 // indirect + go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.51.0 // indirect + go.opentelemetry.io/otel v1.26.0 // indirect + go.opentelemetry.io/otel/metric v1.26.0 // indirect + go.opentelemetry.io/otel/trace v1.26.0 // indirect golang.org/x/term v0.19.0 // indirect ) @@ -178,7 +177,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-20240415180920-8c6c420018be // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240415180920-8c6c420018be // indirect + google.golang.org/genproto v0.0.0-20240429193739-8cf5692501f6 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240429193739-8cf5692501f6 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/flow/go.sum b/flow/go.sum index c9df341342..5857b2424b 100644 --- a/flow/go.sum +++ b/flow/go.sum @@ -1,12 +1,12 @@ 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/auth v0.3.0 h1:PRyzEpGfx/Z9e8+lHsbkoUVXD0gnu4MNmm7Gp8TQNIs= +cloud.google.com/go/auth v0.3.0/go.mod h1:lBv6NKTWp8E3LPzmO1TbiiRKc4drLOfHsgmlH9ogv5w= +cloud.google.com/go/auth/oauth2adapt v0.2.2 h1:+TTV8aXpjeChS9M+aTtN/TjdQnzJvmzKFt//oWu7HX4= +cloud.google.com/go/auth/oauth2adapt v0.2.2/go.mod h1:wcYjgpZI9+Yu7LyYBg4pqSiaRkfEK3GQcpb7C/uyF1Q= +cloud.google.com/go/bigquery v1.61.0 h1:w2Goy9n6gh91LVi6B2Sc+HpBl8WbWhIyzdvVvrAuEIw= +cloud.google.com/go/bigquery v1.61.0/go.mod h1:PjZUje0IocbuTOdq4DBOJLNYB0WF3pAKBHzAYyxCwFo= 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= @@ -21,6 +21,8 @@ cloud.google.com/go/pubsub v1.37.0 h1:0uEEfaB1VIJzabPpwpZf44zWAKAme3zwKKxHk7vJQx cloud.google.com/go/pubsub v1.37.0/go.mod h1:YQOQr1uiUM092EXwKs56OPT650nwnawc+8/IjoUeGzQ= cloud.google.com/go/storage v1.40.0 h1:VEpDQV5CJxFmJ6ueWNsKxcr1QAYOXEgxDa+sBbJahPw= cloud.google.com/go/storage v1.40.0/go.mod h1:Rrj7/hKlG87BLqDJYtwR0fbPld8uJPbQ2ucUMY7Ir0g= +filippo.io/edwards25519 v1.1.0 h1:FNf4tywRC1HmFuKW5xopWpigGjJKiJSV0Cqo0cJWDaA= +filippo.io/edwards25519 v1.1.0/go.mod h1:BxyFTGdWcka3PhytdK4V28tE5sGfRvvvRV7EaN4VDT4= github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4 h1:/vQbFIOMbk2FiG/kXiLl8BRyzTWDw7gX/Hz7Dd5eDMs= github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4/go.mod h1:hN7oaIRCjzsZ2dE+yG5k+rsdt3qcwykqK6HVGcKwsw4= github.com/99designs/keyring v1.2.2 h1:pZd3neh/EmUzWONb35LxQfvuY7kiSXAq3HQd97+XBn0= @@ -54,8 +56,8 @@ github.com/AzureAD/microsoft-authentication-library-for-go v1.2.2/go.mod h1:wP83 github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= 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/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/ClickHouse/clickhouse-go/v2 v2.23.2 h1:+DAKPMnxLS7pduQZsrJc8OhdLS2L9MfDEJ2TS+hpYDM= +github.com/ClickHouse/clickhouse-go/v2 v2.23.2/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= @@ -78,8 +80,6 @@ github.com/alecthomas/repr v0.4.0 h1:GhI2A8MACjfegCPVq9f1FLvIBS+DrQ2KQBFZP1iFzXc github.com/alecthomas/repr v0.4.0/go.mod h1:Fr0507jx4eOXV7AlPV6AVZLYrLIuIeSOWtW57eE/O/4= github.com/andybalholm/brotli v1.1.0 h1:eLKJA0d02Lf0mVpIDgYnqXcUn0GqVmEFny3VuID1U3M= github.com/andybalholm/brotli v1.1.0/go.mod h1:sms7XGricyQI9K10gOSf56VKKWS4oLer58Q+mhRPtnY= -github.com/apache/arrow/go/v14 v14.0.2 h1:N8OkaJEOfI3mEZt07BIkvo4sC6XDbL+48MBPWO5IONw= -github.com/apache/arrow/go/v14 v14.0.2/go.mod h1:u3fgh3EdgN/YQ8cVQRguVW3R+seMybFg8QBQ5LU+eBY= github.com/apache/arrow/go/v15 v15.0.2 h1:60IliRbiyTWCWjERBCkO1W4Qun9svcYoZrSLcyOsMLE= github.com/apache/arrow/go/v15 v15.0.2/go.mod h1:DGXsR3ajT524njufqf95822i+KTh+yea1jass9YXgjA= github.com/aws/aws-sdk-go-v2 v1.26.1 h1:5554eUqIYVWpU0YmeeYZ0wU64H2VLBs8TlhRB2L+EkA= @@ -112,8 +112,8 @@ github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.17.5 h1:f9RyWNtS8oH7cZ github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.17.5/go.mod h1:h5CoMZV2VF297/VLhRhO1WF+XYWOzXo+4HsObA4HjBQ= github.com/aws/aws-sdk-go-v2/service/s3 v1.53.1 h1:6cnno47Me9bRykw9AEv9zkXE+5or7jz8TsskTTccbgc= github.com/aws/aws-sdk-go-v2/service/s3 v1.53.1/go.mod h1:qmdkIIAC+GCLASF7R2whgNrJADz0QZPX+Seiw/i4S3o= -github.com/aws/aws-sdk-go-v2/service/ses v1.22.4 h1:MNU3UWV47ylAAdlU+VxuyItYfuGGp00MvCBxdVAI3kM= -github.com/aws/aws-sdk-go-v2/service/ses v1.22.4/go.mod h1:M/ZQn5uXL4BP1qolIWrlN2SeoUFngJtU/oCwR4WOfZU= +github.com/aws/aws-sdk-go-v2/service/ses v1.22.5 h1:BHNBbGzYmnFt3BILoSNIWj/akVhYgQ+8Am7w3js/w8k= +github.com/aws/aws-sdk-go-v2/service/ses v1.22.5/go.mod h1:M/ZQn5uXL4BP1qolIWrlN2SeoUFngJtU/oCwR4WOfZU= github.com/aws/aws-sdk-go-v2/service/sns v1.29.4 h1:VhW/J21SPH9bNmk1IYdZtzqA6//N2PB5Py5RexNmLVg= github.com/aws/aws-sdk-go-v2/service/sns v1.29.4/go.mod h1:DojKGyWXa4p+e+C+GpG7qf02QaE68Nrg2v/UAXQhKhU= github.com/aws/aws-sdk-go-v2/service/sso v1.20.5 h1:vN8hEbpRnL7+Hopy9dzmRle1xmDc7o8tmY0klsr175w= @@ -192,8 +192,8 @@ github.com/go-logr/logr v1.4.1 h1:pKouT5E8xu9zeFC39JXRDukb6JFQPXM5p5I91188VAQ= github.com/go-logr/logr v1.4.1/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY= github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag= github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE= -github.com/go-sql-driver/mysql v1.6.0 h1:BCTh4TKNUYmOmMUcQ3IipzF5prigylS7XXjEkfCHuOE= -github.com/go-sql-driver/mysql v1.6.0/go.mod h1:DCzpHaOWr8IXmIStZouvnhqoel9Qv2LBy8hT2VhHyBg= +github.com/go-sql-driver/mysql v1.8.1 h1:LedoTUt/eveggdHS9qUFC1EFSa8bU2+1pZjSRpvNJ1Y= +github.com/go-sql-driver/mysql v1.8.1/go.mod h1:wEBSXgmK//2ZFJyE+qWnIsVGmvmEKlqwuVSjsCm7DZg= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/go-test/deep v1.0.4 h1:u2CU3YKy9I2pmu9pX0eq50wCgjfGIt539SqR7FbHiho= github.com/go-test/deep v1.0.4/go.mod h1:wGDj63lr65AM2AQyKZd/NYHGb0R+1RLqB8NKt3aSFNA= @@ -245,8 +245,8 @@ github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/ github.com/google/go-cmp v0.5.7/go.mod h1:n+brtR0CgQNWTVd5ZUFpTBC8YFBDLK/h/bpaJ8/DtOE= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= -github.com/google/martian/v3 v3.3.2 h1:IqNFLAmvJOgVlpdEBiQbDc2EwKW77amAycfTuWKdfvw= -github.com/google/martian/v3 v3.3.2/go.mod h1:oBOf6HBosgwRXnUGWUB05QECsc6uvmMiJ3+6W4l/CUk= +github.com/google/martian/v3 v3.3.3 h1:DIhPTQrbPkgs2yJYdXU/eNACCG5DVQjySNRNlflZ9Fc= +github.com/google/martian/v3 v3.3.3/go.mod h1:iEPrYcgCF7jA9OtScMFQyAlZZ4YXTKEtJ1E6RWzmBA0= github.com/google/s2a-go v0.1.7 h1:60BLSyTrOV4/haCDW4zb1guZItoSq8foHCXrAnjBo/o= github.com/google/s2a-go v0.1.7/go.mod h1:50CgR4k1jNlWBu4UfS4AcfhVe1r6pdZPygJ3R8F0Qdw= github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= @@ -290,8 +290,8 @@ github.com/jmespath/go-jmespath v0.4.0 h1:BEgLn5cpjn8UN1mAw4NjwDrS35OdebyEtFe+9Y github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHWvzYPziyZiYoo= github.com/jmespath/go-jmespath/internal/testify v1.5.1 h1:shLQSRRSCCPj3f2gpwzGwWFoC7ycTf1rcQZHOlsJ6N8= github.com/jmespath/go-jmespath/internal/testify v1.5.1/go.mod h1:L3OGu8Wl2/fWfCI6z80xFu9LTZmf1ZRjMHUOPmWr69U= -github.com/jmoiron/sqlx v1.3.5 h1:vFFPA71p1o5gAeqtEAwLU4dnX2napprKtHr7PYIcN3g= -github.com/jmoiron/sqlx v1.3.5/go.mod h1:nRVWtLre0KfCLJvgxzCsLVMogSvQ1zNJtpYr2Ccp0mQ= +github.com/jmoiron/sqlx v1.4.0 h1:1PLqN7S1UYp5t4SrVVnt4nUVNemrDAtxlulVe+Qgm3o= +github.com/jmoiron/sqlx v1.4.0/go.mod h1:ZrZ7UsYB/weZdl2Bxg6jCRO9c3YHl8r3ahlKmRT4JLY= github.com/joho/godotenv v1.5.1 h1:7eLL/+HRGLY0ldzfGMeQkb7vMd0as4CfYvUVzLqw0N0= github.com/joho/godotenv v1.5.1/go.mod h1:f4LDr5Voq0i2e/R5DDNOoa2zzDfwtkZa6DnEwAbqwq4= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= @@ -312,13 +312,12 @@ github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/kylelemons/godebug v1.1.0 h1:RPNrshWIDI6G2gRW9EHilWtl7Z6Sb1BR0xunSBf0SNc= github.com/kylelemons/godebug v1.1.0/go.mod h1:9/0rRGxNHcop5bhtWyNeEfOS8JIWk580+fNqagV/RAw= -github.com/lib/pq v1.2.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= github.com/lib/pq v1.10.9 h1:YXG7RB+JIjhP29X+OtkiDnYaXQwpS4JEWq7dtCCRUEw= github.com/lib/pq v1.10.9/go.mod h1:AlVN5x4E4T544tWzH6hKfbfQvm3HdbOxrmggDNAPY9o= github.com/linkedin/goavro/v2 v2.12.0 h1:rIQQSj8jdAUlKQh6DttK8wCRv4t4QO09g1C4aBWXslg= 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/mattn/go-sqlite3 v1.14.22 h1:2gZY6PC6kBnID23Tichd1K+Z0oS6nE/XwU+Vz/5o4kU= +github.com/mattn/go-sqlite3 v1.14.22/go.mod h1:Uh1q+B4BYcTPb+yiD3kU8Ct7aC0hY9fxUwlHK0RXw+Y= 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= @@ -406,8 +405,8 @@ github.com/xdg-go/stringprep v1.0.3/go.mod h1:W3f5j4i+9rC0kuIEJL0ky1VpHXQU3ocBgk github.com/xrash/smetrics v0.0.0-20240312152122-5f08fbb34913 h1:+qGGcbkzsfDQNPPe9UDgpxAWQrhbbBXOYJFQDq/dtJw= github.com/xrash/smetrics v0.0.0-20240312152122-5f08fbb34913/go.mod h1:4aEEwZQutDLsQv2Deui4iYQ6DWTxR14g6m8Wv88+Xqk= github.com/youmark/pkcs8 v0.0.0-20181117223130-1be2e3e5546d/go.mod h1:rHwXgn7JulP+udvsHwJoVG1YGAP6VLg4y9I5dyZdqmA= -github.com/youmark/pkcs8 v0.0.0-20201027041543-1326539a0a0a h1:fZHgsYlfvtyqToslyjUt3VOPF4J7aK/3MPcK7xp3PDk= -github.com/youmark/pkcs8 v0.0.0-20201027041543-1326539a0a0a/go.mod h1:ul22v+Nro/R083muKhosV54bj5niojjWZvU8xrevuH4= +github.com/youmark/pkcs8 v0.0.0-20240424034433-3c2c7870ae76 h1:tBiBTKHnIjovYoLX/TPkcf+OjqqKGQrPtGT3Foz+Pgo= +github.com/youmark/pkcs8 v0.0.0-20240424034433-3c2c7870ae76/go.mod h1:SQliXeA7Dhkt//vS29v3zpbEwoa+zb2Cn5xj5uO4K5U= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= @@ -422,18 +421,18 @@ go.einride.tech/aip v0.66.0/go.mod h1:qAhMsfT7plxBX+Oy7Huol6YUvZ0ZzdUz26yZsQwfl1 go.mongodb.org/mongo-driver v1.11.4/go.mod h1:PTSz5yu21bkT/wXpkS7WR5f0ddqw5quethTUn9WM+2g= go.opencensus.io v0.24.0 h1:y73uSU6J157QMP2kn2r30vwW1A2W2WFwSCGnAVxeaD0= go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo= -go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.50.0 h1:zvpPXY7RfYAGSdYQLjp6zxdJNSYD/+FFoCTQN9IPxBs= -go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.50.0/go.mod h1:BMn8NB1vsxTljvuorms2hyOs8IBuuBEq0pl7ltOfy30= -go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.50.0 h1:cEPbyTSEHlQR89XVlyo78gqluF8Y3oMeBkXGWzQsfXY= -go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.50.0/go.mod h1:DKdbWcT4GH1D0Y3Sqt/PFXt2naRKDWtU+eE6oLdFNA8= -go.opentelemetry.io/otel v1.25.0 h1:gldB5FfhRl7OJQbUHt/8s0a7cE8fbsPAtdpRaApKy4k= -go.opentelemetry.io/otel v1.25.0/go.mod h1:Wa2ds5NOXEMkCmUou1WA7ZBfLTHWIsp034OVD7AO+Vg= -go.opentelemetry.io/otel/metric v1.25.0 h1:LUKbS7ArpFL/I2jJHdJcqMGxkRdxpPHE0VU/D4NuEwA= -go.opentelemetry.io/otel/metric v1.25.0/go.mod h1:rkDLUSd2lC5lq2dFNrX9LGAbINP5B7WBkC78RXCpH5s= +go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.51.0 h1:A3SayB3rNyt+1S6qpI9mHPkeHTZbD7XILEqWnYZb2l0= +go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.51.0/go.mod h1:27iA5uvhuRNmalO+iEUdVn5ZMj2qy10Mm+XRIpRmyuU= +go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.51.0 h1:Xs2Ncz0gNihqu9iosIZ5SkBbWo5T8JhhLJFMQL1qmLI= +go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.51.0/go.mod h1:vy+2G/6NvVMpwGX/NyLqcC41fxepnuKHk16E6IZUcJc= +go.opentelemetry.io/otel v1.26.0 h1:LQwgL5s/1W7YiiRwxf03QGnWLb2HW4pLiAhaA5cZXBs= +go.opentelemetry.io/otel v1.26.0/go.mod h1:UmLkJHUAidDval2EICqBMbnAd0/m2vmpf/dAM+fvFs4= +go.opentelemetry.io/otel/metric v1.26.0 h1:7S39CLuY5Jgg9CrnA9HHiEjGMF/X2VHvoXGgSllRz30= +go.opentelemetry.io/otel/metric v1.26.0/go.mod h1:SY+rHOI4cEawI9a7N1A4nIg/nTQXe1ccCNWYOJUrpX4= go.opentelemetry.io/otel/sdk v1.24.0 h1:YMPPDNymmQN3ZgczicBY3B6sf9n62Dlj9pWD3ucgoDw= 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.opentelemetry.io/otel/trace v1.26.0 h1:1ieeAUb4y0TE26jUFrCIXKpTuVK7uJGN9/Z/2LP5sQA= +go.opentelemetry.io/otel/trace v1.26.0/go.mod h1:4iDxvGDQuUkHve82hJJ8UqrwswHYsZuWCBllGV2U2y0= 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= @@ -446,7 +445,6 @@ go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9i go.uber.org/zap v1.18.1/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20200302210943-78000ba7a073/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.22.0 h1:g1v0xeRhjcugydODzvb3mEM9SQ0HGp9s/nh3COQ/C30= @@ -534,20 +532,20 @@ 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.176.0 h1:dHj1/yv5Dm/eQTXiP9hNCRT3xzJHWXeNdRq29XbMxoE= -google.golang.org/api v0.176.0/go.mod h1:Rra+ltKu14pps/4xTycZfobMgLpbosoaaL7c+SEMrO8= +google.golang.org/api v0.177.0 h1:8a0p/BbPa65GlqGWtUKxot4p0TV8OGOfyTjtmkXNXmk= +google.golang.org/api v0.177.0/go.mod h1:srbhue4MLjkjbkux5p3dw/ocYOSZTaIEvf7bCOnFQDw= 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-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/genproto v0.0.0-20240429193739-8cf5692501f6 h1:MTmrc2F5TZKDKXigcZetYkH04YwqtOPEQJwh4PPOgfk= +google.golang.org/genproto v0.0.0-20240429193739-8cf5692501f6/go.mod h1:2ROWwqCIx97Y7CSyp11xB8fori0wzvD6+gbacaf5c8I= +google.golang.org/genproto/googleapis/api v0.0.0-20240429193739-8cf5692501f6 h1:DTJM0R8LECCgFeUwApvcEJHz85HLagW8uRENYxHh1ww= +google.golang.org/genproto/googleapis/api v0.0.0-20240429193739-8cf5692501f6/go.mod h1:10yRODfgim2/T8csjQsMPgZOMvtytXKTDRzH6HRGzRw= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240429193739-8cf5692501f6 h1:DujSIu+2tC9Ht0aPNA7jgj23Iq8Ewi5sgkQ++wdvonE= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240429193739-8cf5692501f6/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= @@ -567,8 +565,8 @@ google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpAD google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.33.0 h1:uNO2rsAINq/JlFpSdYEKIZ0uKD/R9cpdv0T+yoGwGmI= -google.golang.org/protobuf v1.33.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= +google.golang.org/protobuf v1.34.0 h1:Qo/qEd2RZPCf2nKuorzksSknv0d3ERwp1vFG38gSmH4= +google.golang.org/protobuf v1.34.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/nexus/Cargo.lock b/nexus/Cargo.lock index 7d9188ff6d..400bdd24ca 100644 --- a/nexus/Cargo.lock +++ b/nexus/Cargo.lock @@ -298,9 +298,9 @@ checksum = "9d297deb1925b89f2ccc13d7635fa0714f12c87adce1c75356b39ca9b7178567" [[package]] name = "base64" -version = "0.22.0" +version = "0.22.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9475866fec1451be56a3c2400fd081ff546538961565ccb5b7142cbd22bc7a51" +checksum = "72b3254f16251a8381aa12e40e3c4d2f0199f8c6508fbecb9d91f575e0fbb8c6" [[package]] name = "base64ct" @@ -382,9 +382,9 @@ dependencies = [ [[package]] name = "borsh" -version = "1.4.0" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0901fc8eb0aca4c83be0106d6f2db17d86a08dfc2c25f0e84464bf381158add6" +checksum = "dbe5b10e214954177fb1dc9fbd20a1a2608fe99e6c832033bdc7cea287a20d77" dependencies = [ "borsh-derive", "cfg_aliases", @@ -392,9 +392,9 @@ dependencies = [ [[package]] name = "borsh-derive" -version = "1.4.0" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "51670c3aa053938b0ee3bd67c3817e471e626151131b934038e83c5bf8de48f5" +checksum = "d7a8646f94ab393e43e8b35a2558b1624bed28b97ee09c5d15456e3c9463f46d" dependencies = [ "once_cell", "proc-macro-crate", @@ -521,9 +521,9 @@ dependencies = [ [[package]] name = "cc" -version = "1.0.95" +version = "1.0.96" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d32a725bc159af97c3e629873bb9f88fb8cf8a4867175f76dc987815ea07c83b" +checksum = "065a29261d53ba54260972629f9ca6bffa69bac13cd1fed61420f7fa68b9f8bd" dependencies = [ "jobserver", "libc", @@ -2124,7 +2124,7 @@ dependencies = [ "anyhow", "async-recursion", "async-trait", - "base64 0.22.0", + "base64 0.22.1", "catalog", "chrono", "futures", @@ -2207,7 +2207,7 @@ version = "3.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8e459365e590736a54c3fa561947c84837534b8e9af6fc5bf781307e82658fae" dependencies = [ - "base64 0.22.0", + "base64 0.22.1", "serde", ] @@ -2768,7 +2768,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "566cafdd92868e0939d3fb961bd0dc25fcfaaed179291093b3d43e6b3150ea10" dependencies = [ "async-compression", - "base64 0.22.0", + "base64 0.22.1", "bytes", "futures-core", "futures-util", @@ -2967,7 +2967,7 @@ version = "2.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "29993a25686778eb88d4189742cd713c9bce943bc54251a33509dc63cbacf73d" dependencies = [ - "base64 0.22.0", + "base64 0.22.1", "rustls-pki-types", ] @@ -3925,7 +3925,7 @@ version = "2.9.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d11a831e3c0b56e438a28308e7c810799e3c118417f342d30ecec080105395cd" dependencies = [ - "base64 0.22.0", + "base64 0.22.1", "encoding_rs", "flate2", "log", @@ -3982,7 +3982,7 @@ checksum = "830b7e5d4d90034032940e4ace0d9a9a057e7a45cd94e6c007832e39edb82f6d" name = "value" version = "0.1.0" dependencies = [ - "base64 0.22.0", + "base64 0.22.1", "bytes", "chrono", "hex", diff --git a/ui/package-lock.json b/ui/package-lock.json index 67f78371b9..93a22cff58 100644 --- a/ui/package-lock.json +++ b/ui/package-lock.json @@ -10,7 +10,7 @@ "dependencies": { "@grpc/grpc-js": "^1.10.6", "@monaco-editor/react": "^4.6.0", - "@prisma/client": "^5.12.1", + "@prisma/client": "^5.13.0", "@radix-ui/react-checkbox": "^1.0.4", "@radix-ui/react-collapsible": "^1.0.3", "@radix-ui/react-dialog": "^1.0.5", @@ -26,42 +26,42 @@ "@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.16.1", + "@tremor/react": "^3.16.2", "@types/node": "^20.12.7", - "@types/react": "^18.2.79", - "@types/react-dom": "^18.2.25", + "@types/react": "^18.3.1", + "@types/react-dom": "^18.3.0", "classnames": "^2.5.1", "long": "^5.2.3", - "lucide-react": "^0.372.0", - "material-symbols": "^0.17.2", + "lucide-react": "^0.376.0", + "material-symbols": "^0.17.4", "moment": "^2.30.1", "moment-timezone": "^0.5.45", - "next": "^14.2.2", + "next": "^14.2.3", "next-auth": "^4.24.7", "prop-types": "^15.8.1", "protobufjs": "^7.2.6", - "react": "18.2.0", - "react-dom": "18.2.0", + "react": "^18.3.1", + "react-dom": "^18.3.1", "react-select": "^5.8.0", "react-spinners": "^0.13.8", "react-toastify": "^10.0.5", - "styled-components": "^6.1.8", + "styled-components": "^6.1.9", "swr": "^2.2.5", "usehooks-ts": "^3.1.0", - "zod": "^3.23.0" + "zod": "^3.23.5" }, "devDependencies": { "autoprefixer": "^10.4.19", "copy-webpack-plugin": "^12.0.2", "eslint": "^8.57.0", - "eslint-config-next": "^14.2.2", + "eslint-config-next": "^14.2.3", "eslint-config-prettier": "^9.1.0", "gh-pages": "^6.1.1", "less": "^4.2.0", "postcss": "^8.4.38", "prettier": "^3.2.5", "prettier-plugin-organize-imports": "^3.2.4", - "prisma": "^5.12.1", + "prisma": "^5.13.0", "string-width": "^7.1.0", "tailwindcss": "^3.4.3", "tailwindcss-animate": "^1.0.7", @@ -69,15 +69,6 @@ "webpack": "^5.91.0" } }, - "node_modules/@aashutoshrathi/word-wrap": { - "version": "1.2.6", - "resolved": "https://registry.npmjs.org/@aashutoshrathi/word-wrap/-/word-wrap-1.2.6.tgz", - "integrity": "sha512-1Yjs2SvM8TflER/OD3cOjhWWOZb58A2t7wpE2S9XfBYTiIl+XFhQG2bjy4Pu1I+EAlCNUzRDYDdFwFYUKvXcIA==", - "dev": true, - "engines": { - "node": ">=0.10.0" - } - }, "node_modules/@alloc/quick-lru": { "version": "5.2.0", "resolved": "https://registry.npmjs.org/@alloc/quick-lru/-/quick-lru-5.2.0.tgz", @@ -121,19 +112,19 @@ } }, "node_modules/@babel/helper-validator-identifier": { - "version": "7.22.20", - "resolved": "https://registry.npmjs.org/@babel/helper-validator-identifier/-/helper-validator-identifier-7.22.20.tgz", - "integrity": "sha512-Y4OZ+ytlatR8AI+8KZfKuL5urKp7qey08ha31L8b3BwewJAoJamTzyvxPR/5D+KkdJCGPq/+8TukHBlY10FX9A==", + "version": "7.24.5", + "resolved": "https://registry.npmjs.org/@babel/helper-validator-identifier/-/helper-validator-identifier-7.24.5.tgz", + "integrity": "sha512-3q93SSKX2TWCG30M2G2kwaKeTYgEUp5Snjuj8qm729SObL6nbtUldAi37qbxkD5gg3xnBio+f9nqpSepGZMvxA==", "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/highlight": { - "version": "7.24.2", - "resolved": "https://registry.npmjs.org/@babel/highlight/-/highlight-7.24.2.tgz", - "integrity": "sha512-Yac1ao4flkTxTteCDZLEvdxg2fZfz1v8M4QpaGypq/WPDqg3ijHYbDfs+LG5hvzSoqaSZ9/Z9lKSP3CjZjv+pA==", + "version": "7.24.5", + "resolved": "https://registry.npmjs.org/@babel/highlight/-/highlight-7.24.5.tgz", + "integrity": "sha512-8lLmua6AVh/8SLJRRVD6V8p73Hir9w5mJrhE+IPpILG31KKlI9iz5zmBYKcWPS59qSfgP9RaSBQSHHE81WKuEw==", "dependencies": { - "@babel/helper-validator-identifier": "^7.22.20", + "@babel/helper-validator-identifier": "^7.24.5", "chalk": "^2.4.2", "js-tokens": "^4.0.0", "picocolors": "^1.0.0" @@ -207,9 +198,9 @@ } }, "node_modules/@babel/runtime": { - "version": "7.24.4", - "resolved": "https://registry.npmjs.org/@babel/runtime/-/runtime-7.24.4.tgz", - "integrity": "sha512-dkxf7+hn8mFBwKjs9bvBlArzLVxVbS8usaPUDd5p2a9JCL9tB8OaOVN1isD4+Xyk4ns89/xeOmbQvgdK7IIVdA==", + "version": "7.24.5", + "resolved": "https://registry.npmjs.org/@babel/runtime/-/runtime-7.24.5.tgz", + "integrity": "sha512-Nms86NXrsaeU9vbBJKni6gXiEXZ4CVpYVzEjDH9Sb8vmZ3UljyA1GSOJl/6LGPO8EHLuSF9H+IxNXHPX8QHJ4g==", "dependencies": { "regenerator-runtime": "^0.14.0" }, @@ -218,12 +209,12 @@ } }, "node_modules/@babel/types": { - "version": "7.24.0", - "resolved": "https://registry.npmjs.org/@babel/types/-/types-7.24.0.tgz", - "integrity": "sha512-+j7a5c253RfKh8iABBhywc8NSfP5LURe7Uh4qpsh6jc+aLJguvmIUBdjSdEMQv2bENrCR5MfRdjGo7vzS/ob7w==", + "version": "7.24.5", + "resolved": "https://registry.npmjs.org/@babel/types/-/types-7.24.5.tgz", + "integrity": "sha512-6mQNsaLeXTw0nxYUYu+NSa4Hx4BlF1x1x8/PMFbiR+GBSr+2DkECc69b8hgy2frEodNcvPffeH8YfWd3LI6jhQ==", "dependencies": { - "@babel/helper-string-parser": "^7.23.4", - "@babel/helper-validator-identifier": "^7.22.20", + "@babel/helper-string-parser": "^7.24.1", + "@babel/helper-validator-identifier": "^7.24.5", "to-fast-properties": "^2.0.0" }, "engines": { @@ -406,17 +397,17 @@ } }, "node_modules/@floating-ui/core": { - "version": "1.6.0", - "resolved": "https://registry.npmjs.org/@floating-ui/core/-/core-1.6.0.tgz", - "integrity": "sha512-PcF++MykgmTj3CIyOQbKA/hDzOAiqI3mhuoN44WRCopIs1sgoDoU4oty4Jtqaj/y3oDU6fnVSm4QG0a3t5i0+g==", + "version": "1.6.1", + "resolved": "https://registry.npmjs.org/@floating-ui/core/-/core-1.6.1.tgz", + "integrity": "sha512-42UH54oPZHPdRHdw6BgoBD6cg/eVTmVrFcgeRDM3jbO7uxSoipVcmcIGFcA5jmOHO5apcyvBhkSKES3fQJnu7A==", "dependencies": { - "@floating-ui/utils": "^0.2.1" + "@floating-ui/utils": "^0.2.0" } }, "node_modules/@floating-ui/dom": { - "version": "1.6.3", - "resolved": "https://registry.npmjs.org/@floating-ui/dom/-/dom-1.6.3.tgz", - "integrity": "sha512-RnDthu3mzPlQ31Ss/BTwQ1zjzIhr3lk1gZB1OC56h/1vEtaXkESrOqL5fQVMfXpwGtRwX+YsZBdyHtJMQnkArw==", + "version": "1.6.4", + "resolved": "https://registry.npmjs.org/@floating-ui/dom/-/dom-1.6.4.tgz", + "integrity": "sha512-0G8R+zOvQsAG1pg2Q99P21jiqxqGBW1iRe/iXHsBRBxnpXKFI8QwbB4x5KmYLggNO5m34IQgOIu9SCRfR/WWiQ==", "dependencies": { "@floating-ui/core": "^1.0.0", "@floating-ui/utils": "^0.2.0" @@ -437,11 +428,11 @@ } }, "node_modules/@floating-ui/react-dom": { - "version": "2.0.8", - "resolved": "https://registry.npmjs.org/@floating-ui/react-dom/-/react-dom-2.0.8.tgz", - "integrity": "sha512-HOdqOt3R3OGeTKidaLvJKcgg75S6tibQ3Tif4eyd91QnIJWr0NLvoXFpJA/j8HqkFSL68GDca9AuyWEHlhyClw==", + "version": "2.0.9", + "resolved": "https://registry.npmjs.org/@floating-ui/react-dom/-/react-dom-2.0.9.tgz", + "integrity": "sha512-q0umO0+LQK4+p6aGyvzASqKbKOJcAHJ7ycE9CuUvfx3s9zTHWmGJTPOIlM/hmSBfUfg/XfY5YhLBLR/LHwShQQ==", "dependencies": { - "@floating-ui/dom": "^1.6.1" + "@floating-ui/dom": "^1.0.0" }, "peerDependencies": { "react": ">=16.8.0", @@ -461,9 +452,9 @@ } }, "node_modules/@floating-ui/utils": { - "version": "0.2.1", - "resolved": "https://registry.npmjs.org/@floating-ui/utils/-/utils-0.2.1.tgz", - "integrity": "sha512-9TANp6GPoMtYzQdt54kfAyMmz1+osLlXdg2ENroU7zzrtflTLrrC/lgrIfaSe+Wu0b89GKccT7vxXA0MoAIO+Q==" + "version": "0.2.2", + "resolved": "https://registry.npmjs.org/@floating-ui/utils/-/utils-0.2.2.tgz", + "integrity": "sha512-J4yDIIthosAsRZ5CPYP/jQvUAQtlZTTD/4suA08/FEnlxqW3sKS9iAhgsa9VYLZ6vDHn/ixJgIqRQPotoBjxIw==" }, "node_modules/@grpc/grpc-js": { "version": "1.10.6", @@ -698,23 +689,23 @@ } }, "node_modules/@next/env": { - "version": "14.2.2", - "resolved": "https://registry.npmjs.org/@next/env/-/env-14.2.2.tgz", - "integrity": "sha512-sk72qRfM1Q90XZWYRoJKu/UWlTgihrASiYw/scb15u+tyzcze3bOuJ/UV6TBOQEeUaxOkRqGeuGUdiiuxc5oqw==" + "version": "14.2.3", + "resolved": "https://registry.npmjs.org/@next/env/-/env-14.2.3.tgz", + "integrity": "sha512-W7fd7IbkfmeeY2gXrzJYDx8D2lWKbVoTIj1o1ScPHNzvp30s1AuoEFSdr39bC5sjxJaxTtq3OTCZboNp0lNWHA==" }, "node_modules/@next/eslint-plugin-next": { - "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==", + "version": "14.2.3", + "resolved": "https://registry.npmjs.org/@next/eslint-plugin-next/-/eslint-plugin-next-14.2.3.tgz", + "integrity": "sha512-L3oDricIIjgj1AVnRdRor21gI7mShlSwU/1ZGHmqM3LzHhXXhdkrfeNY5zif25Bi5Dd7fiJHsbhoZCHfXYvlAw==", "dev": true, "dependencies": { "glob": "10.3.10" } }, "node_modules/@next/swc-darwin-arm64": { - "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==", + "version": "14.2.3", + "resolved": "https://registry.npmjs.org/@next/swc-darwin-arm64/-/swc-darwin-arm64-14.2.3.tgz", + "integrity": "sha512-3pEYo/RaGqPP0YzwnlmPN2puaF2WMLM3apt5jLW2fFdXD9+pqcoTzRk+iZsf8ta7+quAe4Q6Ms0nR0SFGFdS1A==", "cpu": [ "arm64" ], @@ -727,9 +718,9 @@ } }, "node_modules/@next/swc-darwin-x64": { - "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==", + "version": "14.2.3", + "resolved": "https://registry.npmjs.org/@next/swc-darwin-x64/-/swc-darwin-x64-14.2.3.tgz", + "integrity": "sha512-6adp7waE6P1TYFSXpY366xwsOnEXM+y1kgRpjSRVI2CBDOcbRjsJ67Z6EgKIqWIue52d2q/Mx8g9MszARj8IEA==", "cpu": [ "x64" ], @@ -742,9 +733,9 @@ } }, "node_modules/@next/swc-linux-arm64-gnu": { - "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==", + "version": "14.2.3", + "resolved": "https://registry.npmjs.org/@next/swc-linux-arm64-gnu/-/swc-linux-arm64-gnu-14.2.3.tgz", + "integrity": "sha512-cuzCE/1G0ZSnTAHJPUT1rPgQx1w5tzSX7POXSLaS7w2nIUJUD+e25QoXD/hMfxbsT9rslEXugWypJMILBj/QsA==", "cpu": [ "arm64" ], @@ -757,9 +748,9 @@ } }, "node_modules/@next/swc-linux-arm64-musl": { - "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==", + "version": "14.2.3", + "resolved": "https://registry.npmjs.org/@next/swc-linux-arm64-musl/-/swc-linux-arm64-musl-14.2.3.tgz", + "integrity": "sha512-0D4/oMM2Y9Ta3nGuCcQN8jjJjmDPYpHX9OJzqk42NZGJocU2MqhBq5tWkJrUQOQY9N+In9xOdymzapM09GeiZw==", "cpu": [ "arm64" ], @@ -772,9 +763,9 @@ } }, "node_modules/@next/swc-linux-x64-gnu": { - "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==", + "version": "14.2.3", + "resolved": "https://registry.npmjs.org/@next/swc-linux-x64-gnu/-/swc-linux-x64-gnu-14.2.3.tgz", + "integrity": "sha512-ENPiNnBNDInBLyUU5ii8PMQh+4XLr4pG51tOp6aJ9xqFQ2iRI6IH0Ds2yJkAzNV1CfyagcyzPfROMViS2wOZ9w==", "cpu": [ "x64" ], @@ -787,9 +778,9 @@ } }, "node_modules/@next/swc-linux-x64-musl": { - "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==", + "version": "14.2.3", + "resolved": "https://registry.npmjs.org/@next/swc-linux-x64-musl/-/swc-linux-x64-musl-14.2.3.tgz", + "integrity": "sha512-BTAbq0LnCbF5MtoM7I/9UeUu/8ZBY0i8SFjUMCbPDOLv+un67e2JgyN4pmgfXBwy/I+RHu8q+k+MCkDN6P9ViQ==", "cpu": [ "x64" ], @@ -802,9 +793,9 @@ } }, "node_modules/@next/swc-win32-arm64-msvc": { - "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==", + "version": "14.2.3", + "resolved": "https://registry.npmjs.org/@next/swc-win32-arm64-msvc/-/swc-win32-arm64-msvc-14.2.3.tgz", + "integrity": "sha512-AEHIw/dhAMLNFJFJIJIyOFDzrzI5bAjI9J26gbO5xhAKHYTZ9Or04BesFPXiAYXDNdrwTP2dQceYA4dL1geu8A==", "cpu": [ "arm64" ], @@ -817,9 +808,9 @@ } }, "node_modules/@next/swc-win32-ia32-msvc": { - "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==", + "version": "14.2.3", + "resolved": "https://registry.npmjs.org/@next/swc-win32-ia32-msvc/-/swc-win32-ia32-msvc-14.2.3.tgz", + "integrity": "sha512-vga40n1q6aYb0CLrM+eEmisfKCR45ixQYXuBXxOOmmoV8sYST9k7E3US32FsY+CkkF7NtzdcebiFT4CHuMSyZw==", "cpu": [ "ia32" ], @@ -832,9 +823,9 @@ } }, "node_modules/@next/swc-win32-x64-msvc": { - "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==", + "version": "14.2.3", + "resolved": "https://registry.npmjs.org/@next/swc-win32-x64-msvc/-/swc-win32-x64-msvc-14.2.3.tgz", + "integrity": "sha512-Q1/zm43RWynxrO7lW4ehciQVj+5ePBhOK+/K2P7pLFX3JaJ/IZVC69SHidrmZSOkqz7ECIOhhy7XhAFG4JYyHA==", "cpu": [ "x64" ], @@ -896,9 +887,9 @@ } }, "node_modules/@prisma/client": { - "version": "5.12.1", - "resolved": "https://registry.npmjs.org/@prisma/client/-/client-5.12.1.tgz", - "integrity": "sha512-6/JnizEdlSBxDIdiLbrBdMW5NqDxOmhXAJaNXiPpgzAPr/nLZResT6MMpbOHLo5yAbQ1Vv5UU8PTPRzb0WIxdA==", + "version": "5.13.0", + "resolved": "https://registry.npmjs.org/@prisma/client/-/client-5.13.0.tgz", + "integrity": "sha512-uYdfpPncbZ/syJyiYBwGZS8Gt1PTNoErNYMuqHDa2r30rNSFtgTA/LXsSk55R7pdRTMi5pHkeP9B14K6nHmwkg==", "hasInstallScript": true, "engines": { "node": ">=16.13" @@ -913,48 +904,48 @@ } }, "node_modules/@prisma/debug": { - "version": "5.12.1", - "resolved": "https://registry.npmjs.org/@prisma/debug/-/debug-5.12.1.tgz", - "integrity": "sha512-kd/wNsR0klrv79o1ITsbWxYyh4QWuBidvxsXSParPsYSu0ircUmNk3q4ojsgNc3/81b0ozg76iastOG43tbf8A==", + "version": "5.13.0", + "resolved": "https://registry.npmjs.org/@prisma/debug/-/debug-5.13.0.tgz", + "integrity": "sha512-699iqlEvzyCj9ETrXhs8o8wQc/eVW+FigSsHpiskSFydhjVuwTJEfj/nIYqTaWFYuxiWQRfm3r01meuW97SZaQ==", "devOptional": true }, "node_modules/@prisma/engines": { - "version": "5.12.1", - "resolved": "https://registry.npmjs.org/@prisma/engines/-/engines-5.12.1.tgz", - "integrity": "sha512-HQDdglLw2bZR/TXD2Y+YfDMvi5Q8H+acbswqOsWyq9pPjBLYJ6gzM+ptlTU/AV6tl0XSZLU1/7F4qaWa8bqpJA==", + "version": "5.13.0", + "resolved": "https://registry.npmjs.org/@prisma/engines/-/engines-5.13.0.tgz", + "integrity": "sha512-hIFLm4H1boj6CBZx55P4xKby9jgDTeDG0Jj3iXtwaaHmlD5JmiDkZhh8+DYWkTGchu+rRF36AVROLnk0oaqhHw==", "devOptional": true, "hasInstallScript": true, "dependencies": { - "@prisma/debug": "5.12.1", - "@prisma/engines-version": "5.12.0-21.473ed3124229e22d881cb7addf559799debae1ab", - "@prisma/fetch-engine": "5.12.1", - "@prisma/get-platform": "5.12.1" + "@prisma/debug": "5.13.0", + "@prisma/engines-version": "5.13.0-23.b9a39a7ee606c28e3455d0fd60e78c3ba82b1a2b", + "@prisma/fetch-engine": "5.13.0", + "@prisma/get-platform": "5.13.0" } }, "node_modules/@prisma/engines-version": { - "version": "5.12.0-21.473ed3124229e22d881cb7addf559799debae1ab", - "resolved": "https://registry.npmjs.org/@prisma/engines-version/-/engines-version-5.12.0-21.473ed3124229e22d881cb7addf559799debae1ab.tgz", - "integrity": "sha512-6yvO8s80Tym61aB4QNtYZfWVmE3pwqe807jEtzm8C5VDe7nw8O1FGX3TXUaXmWV0fQTIAfRbeL2Gwrndabp/0g==", + "version": "5.13.0-23.b9a39a7ee606c28e3455d0fd60e78c3ba82b1a2b", + "resolved": "https://registry.npmjs.org/@prisma/engines-version/-/engines-version-5.13.0-23.b9a39a7ee606c28e3455d0fd60e78c3ba82b1a2b.tgz", + "integrity": "sha512-AyUuhahTINGn8auyqYdmxsN+qn0mw3eg+uhkp8zwknXYIqoT3bChG4RqNY/nfDkPvzWAPBa9mrDyBeOnWSgO6A==", "devOptional": true }, "node_modules/@prisma/fetch-engine": { - "version": "5.12.1", - "resolved": "https://registry.npmjs.org/@prisma/fetch-engine/-/fetch-engine-5.12.1.tgz", - "integrity": "sha512-qSs3KcX1HKcea1A+hlJVK/ljj0PNIUHDxAayGMvgJBqmaN32P9tCidlKz1EGv6WoRFICYnk3Dd/YFLBwnFIozA==", + "version": "5.13.0", + "resolved": "https://registry.npmjs.org/@prisma/fetch-engine/-/fetch-engine-5.13.0.tgz", + "integrity": "sha512-Yh4W+t6YKyqgcSEB3odBXt7QyVSm0OQlBSldQF2SNXtmOgMX8D7PF/fvH6E6qBCpjB/yeJLy/FfwfFijoHI6sA==", "devOptional": true, "dependencies": { - "@prisma/debug": "5.12.1", - "@prisma/engines-version": "5.12.0-21.473ed3124229e22d881cb7addf559799debae1ab", - "@prisma/get-platform": "5.12.1" + "@prisma/debug": "5.13.0", + "@prisma/engines-version": "5.13.0-23.b9a39a7ee606c28e3455d0fd60e78c3ba82b1a2b", + "@prisma/get-platform": "5.13.0" } }, "node_modules/@prisma/get-platform": { - "version": "5.12.1", - "resolved": "https://registry.npmjs.org/@prisma/get-platform/-/get-platform-5.12.1.tgz", - "integrity": "sha512-pgIR+pSvhYHiUcqXVEZS31NrFOTENC9yFUdEAcx7cdQBoZPmHVjtjN4Ss6NzVDMYPrKJJ51U14EhEoeuBlMioQ==", + "version": "5.13.0", + "resolved": "https://registry.npmjs.org/@prisma/get-platform/-/get-platform-5.13.0.tgz", + "integrity": "sha512-B/WrQwYTzwr7qCLifQzYOmQhZcFmIFhR81xC45gweInSUn2hTEbfKUPd2keAog+y5WI5xLAFNJ3wkXplvSVkSw==", "devOptional": true, "dependencies": { - "@prisma/debug": "5.12.1" + "@prisma/debug": "5.13.0" } }, "node_modules/@protobufjs/aspromise": { @@ -2042,11 +2033,11 @@ } }, "node_modules/@tanstack/react-virtual": { - "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==", + "version": "3.5.0", + "resolved": "https://registry.npmjs.org/@tanstack/react-virtual/-/react-virtual-3.5.0.tgz", + "integrity": "sha512-rtvo7KwuIvqK9zb0VZ5IL7fiJAEnG+0EiFZz8FUOs+2mhGqdGmjKIaT1XU7Zq0eFqL0jonLlhbayJI/J2SA/Bw==", "dependencies": { - "@tanstack/virtual-core": "3.4.0" + "@tanstack/virtual-core": "3.5.0" }, "funding": { "type": "github", @@ -2058,26 +2049,26 @@ } }, "node_modules/@tanstack/virtual-core": { - "version": "3.4.0", - "resolved": "https://registry.npmjs.org/@tanstack/virtual-core/-/virtual-core-3.4.0.tgz", - "integrity": "sha512-75jXqXxqq5M5Veb9KP1STi8kA5u408uOOAefk2ftHDGCpUk3RP6zX++QqfbmHJTBiU72NQ+ghgCZVts/Wocz8Q==", + "version": "3.5.0", + "resolved": "https://registry.npmjs.org/@tanstack/virtual-core/-/virtual-core-3.5.0.tgz", + "integrity": "sha512-KnPRCkQTyqhanNC0K63GBG3wA8I+D1fQuVnAvcBF8f13akOKeQp1gSbu6f77zCxhEk727iV5oQnbHLYzHrECLg==", "funding": { "type": "github", "url": "https://github.com/sponsors/tannerlinsley" } }, "node_modules/@tremor/react": { - "version": "3.16.1", - "resolved": "https://registry.npmjs.org/@tremor/react/-/react-3.16.1.tgz", - "integrity": "sha512-ablAsFL7twiUYf57gMAS2ar6YTajsQ7fs63vqlC2i9DpLuhFSrwEp69y1a6bxXP+KH0etYRMj6xxCvdCdiy8mQ==", + "version": "3.16.2", + "resolved": "https://registry.npmjs.org/@tremor/react/-/react-3.16.2.tgz", + "integrity": "sha512-Isdc+Sf4WHlnrAAO8Hk/nK84HiXzCZvb6ZFRHrzOkF+APm6nDhvKPRorXcXZ2BKSS5T5L0QVsid5fIxly8kRdA==", "dependencies": { "@floating-ui/react": "^0.19.2", - "@headlessui/react": "^1.7.18", + "@headlessui/react": "^1.7.19", "@headlessui/tailwindcss": "^0.2.0", "date-fns": "^2.30.0", "react-day-picker": "^8.9.1", "react-transition-state": "^2.1.1", - "recharts": "^2.10.3", + "recharts": "^2.12.6", "tailwind-merge": "^1.14.0" }, "peerDependencies": { @@ -2196,18 +2187,18 @@ "integrity": "sha512-5zvhXYtRNRluoE/jAp4GVsSduVUzNWKkOZrCDBWYtE7biZywwdC2AcEzg+cSMLFRfVgeAFqpfNabiPjxFddV1Q==" }, "node_modules/@types/react": { - "version": "18.2.79", - "resolved": "https://registry.npmjs.org/@types/react/-/react-18.2.79.tgz", - "integrity": "sha512-RwGAGXPl9kSXwdNTafkOEuFrTBD5SA2B3iEB96xi8+xu5ddUa/cpvyVCSNn+asgLCTHkb5ZxN8gbuibYJi4s1w==", + "version": "18.3.1", + "resolved": "https://registry.npmjs.org/@types/react/-/react-18.3.1.tgz", + "integrity": "sha512-V0kuGBX3+prX+DQ/7r2qsv1NsdfnCLnTgnRJ1pYnxykBhGMz+qj+box5lq7XsO5mtZsBqpjwwTu/7wszPfMBcw==", "dependencies": { "@types/prop-types": "*", "csstype": "^3.0.2" } }, "node_modules/@types/react-dom": { - "version": "18.2.25", - "resolved": "https://registry.npmjs.org/@types/react-dom/-/react-dom-18.2.25.tgz", - "integrity": "sha512-o/V48vf4MQh7juIKZU2QGDfli6p1+OOi5oXx36Hffpc9adsHeXjVp8rHuPkjd8VT8sOJ2Zp05HR7CdpGTIUFUA==", + "version": "18.3.0", + "resolved": "https://registry.npmjs.org/@types/react-dom/-/react-dom-18.3.0.tgz", + "integrity": "sha512-EhwApuTmMBmXuFOikhQLIBUn6uFg81SwLMOAUgodJF14SOBOCMdU04gDoYi0WOJJHD144TL32z4yDqCW3dnkQg==", "dependencies": { "@types/react": "*" } @@ -2609,15 +2600,15 @@ } }, "node_modules/ajv-formats/node_modules/ajv": { - "version": "8.12.0", - "resolved": "https://registry.npmjs.org/ajv/-/ajv-8.12.0.tgz", - "integrity": "sha512-sRu1kpcO9yLtYxBKvqfTeh9KzZEwO3STyX1HT+4CaDzC6HpTGYhIhPIzj9XuKU7KYDwnaeh5hcOwjy1QuJzBPA==", + "version": "8.13.0", + "resolved": "https://registry.npmjs.org/ajv/-/ajv-8.13.0.tgz", + "integrity": "sha512-PRA911Blj99jR5RMeTunVbNXMF6Lp4vZXnk5GQjcnUWUTsrXtekg/pnmFFI2u/I36Y/2bITGS30GZCXei6uNkA==", "dev": true, "dependencies": { - "fast-deep-equal": "^3.1.1", + "fast-deep-equal": "^3.1.3", "json-schema-traverse": "^1.0.0", "require-from-string": "^2.0.2", - "uri-js": "^4.2.2" + "uri-js": "^4.4.1" }, "funding": { "type": "github", @@ -3112,9 +3103,9 @@ } }, "node_modules/caniuse-lite": { - "version": "1.0.30001612", - "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001612.tgz", - "integrity": "sha512-lFgnZ07UhaCcsSZgWW0K5j4e69dK1u/ltrL9lTUiFOwNHs12S3UMIEYgBV0Z6C6hRDev7iRnMzzYmKabYdXF9g==", + "version": "1.0.30001614", + "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001614.tgz", + "integrity": "sha512-jmZQ1VpmlRwHgdP1/uiKzgiAuGOfLEJsYFP4+GBou/QQ4U6IOJCB4NP1c+1p9RGLpwObcT94jA5/uO+F1vBbog==", "funding": [ { "type": "opencollective", @@ -3247,9 +3238,9 @@ } }, "node_modules/clsx": { - "version": "2.1.0", - "resolved": "https://registry.npmjs.org/clsx/-/clsx-2.1.0.tgz", - "integrity": "sha512-m3iNNWpd9rl3jvvcBnu70ylMdrXt8Vlq4HYadnU5fwcOtvkSQWPmj7amUcDT2qYI7risszBjI5AUIUox9D16pg==", + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/clsx/-/clsx-2.1.1.tgz", + "integrity": "sha512-eYm0QWBtUrBWZWG0d386OGAw16Z995PiOVo2B7bjWSbHedGl5e0ZWaq65kOGgUSNesEIDkB9ISbTg/JK9dhCZA==", "engines": { "node": ">=6" } @@ -3726,9 +3717,9 @@ "integrity": "sha512-I88TYZWc9XiYHRQ4/3c5rjjfgkjhLyW2luGIheGERbNQ6OY7yTybanSpDXZa8y7VUP9YmDcYa+eyq4ca7iLqWA==" }, "node_modules/electron-to-chromium": { - "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==", + "version": "1.4.752", + "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.752.tgz", + "integrity": "sha512-P3QJreYI/AUTcfBVrC4zy9KvnZWekViThgQMX/VpJ+IsOBbcX5JFpORM4qWapwWQ+agb2nYAOyn/4PMXOk0m2Q==", "dev": true }, "node_modules/email-addresses": { @@ -3858,14 +3849,14 @@ } }, "node_modules/es-iterator-helpers": { - "version": "1.0.18", - "resolved": "https://registry.npmjs.org/es-iterator-helpers/-/es-iterator-helpers-1.0.18.tgz", - "integrity": "sha512-scxAJaewsahbqTYrGKJihhViaM6DDZDDoucfvzNbK0pOren1g/daDQ3IAhzn+1G14rBG7w+i5N+qul60++zlKA==", + "version": "1.0.19", + "resolved": "https://registry.npmjs.org/es-iterator-helpers/-/es-iterator-helpers-1.0.19.tgz", + "integrity": "sha512-zoMwbCcH5hwUkKJkT8kDIBZSz9I6mVG//+lDCinLCGov4+r7NIy0ld8o03M0cJxl2spVf6ESYVS6/gpIfq1FFw==", "dev": true, "dependencies": { "call-bind": "^1.0.7", "define-properties": "^1.2.1", - "es-abstract": "^1.23.0", + "es-abstract": "^1.23.3", "es-errors": "^1.3.0", "es-set-tostringtag": "^2.0.3", "function-bind": "^1.1.2", @@ -3883,9 +3874,9 @@ } }, "node_modules/es-module-lexer": { - "version": "1.5.0", - "resolved": "https://registry.npmjs.org/es-module-lexer/-/es-module-lexer-1.5.0.tgz", - "integrity": "sha512-pqrTKmwEIgafsYZAGw9kszYzmagcE/n4dbgwGWLEXg7J4QFJVQRBld8j3Q3GNez79jzxZshq0bcT962QHOghjw==", + "version": "1.5.2", + "resolved": "https://registry.npmjs.org/es-module-lexer/-/es-module-lexer-1.5.2.tgz", + "integrity": "sha512-l60ETUTmLqbVbVHv1J4/qj+M8nq7AwMzEcg3kmJDt9dCNrTk+yHcYFf/Kw75pMDwd9mPcIGCG5LcS20SxYRzFA==", "dev": true }, "node_modules/es-object-atoms": { @@ -4015,12 +4006,12 @@ } }, "node_modules/eslint-config-next": { - "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==", + "version": "14.2.3", + "resolved": "https://registry.npmjs.org/eslint-config-next/-/eslint-config-next-14.2.3.tgz", + "integrity": "sha512-ZkNztm3Q7hjqvB1rRlOX8P9E/cXRL9ajRcs8jufEtwMfTVYRqnmtnaSu57QqHyBlovMuiB8LEzfLBkh5RYV6Fg==", "dev": true, "dependencies": { - "@next/eslint-plugin-next": "14.2.2", + "@next/eslint-plugin-next": "14.2.3", "@rushstack/eslint-patch": "^1.3.3", "@typescript-eslint/parser": "^5.4.2 || ^6.0.0 || 7.0.0 - 7.2.0", "eslint-import-resolver-node": "^0.3.6", @@ -4247,9 +4238,9 @@ } }, "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==", + "version": "4.6.2", + "resolved": "https://registry.npmjs.org/eslint-plugin-react-hooks/-/eslint-plugin-react-hooks-4.6.2.tgz", + "integrity": "sha512-QzliNJq4GinDBcD8gPB5v0wh6g8q3SUi6EFF0x8N/BL9PoVs0atuGc47ozMRyOWAKdwaZ5OnbOEa3WR+dSGKuQ==", "dev": true, "engines": { "node": ">=10" @@ -4891,12 +4882,13 @@ } }, "node_modules/globalthis": { - "version": "1.0.3", - "resolved": "https://registry.npmjs.org/globalthis/-/globalthis-1.0.3.tgz", - "integrity": "sha512-sFdI5LyBiNTHjRd7cGPWapiHWMOXKyuBNX/cWJ3NfzrZQVa8GI/8cofCl74AOVqq9W5kNmguTIzJ/1s2gyI9wA==", + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/globalthis/-/globalthis-1.0.4.tgz", + "integrity": "sha512-DpLKbNU4WylpxJykQujfCcwYWiV/Jhm50Goo0wrVILAv5jOr9d+H+UR3PhSCD2rCCEIg0uc+G+muBTwD54JhDQ==", "dev": true, "dependencies": { - "define-properties": "^1.1.3" + "define-properties": "^1.2.1", + "gopd": "^1.0.1" }, "engines": { "node": ">= 0.4" @@ -5864,9 +5856,9 @@ } }, "node_modules/lucide-react": { - "version": "0.372.0", - "resolved": "https://registry.npmjs.org/lucide-react/-/lucide-react-0.372.0.tgz", - "integrity": "sha512-0cKdqmilHXWUwWAWnf6CrrjHD8YaqPMtLrmEHXolZusNTr9epULCsiJwIOHk2q1yFxdEwd96D4zShlAj67UJdA==", + "version": "0.376.0", + "resolved": "https://registry.npmjs.org/lucide-react/-/lucide-react-0.376.0.tgz", + "integrity": "sha512-g91IX3ERD6yUR1TL2dsL4BkcGygpZz/EsqjAeL/kcRQV0EApIOr/9eBfKhYOVyQIcGGuotFGjF3xKLHMEz+b7g==", "peerDependencies": { "react": "^16.5.1 || ^17.0.0 || ^18.0.0" } @@ -5896,9 +5888,9 @@ } }, "node_modules/material-symbols": { - "version": "0.17.2", - "resolved": "https://registry.npmjs.org/material-symbols/-/material-symbols-0.17.2.tgz", - "integrity": "sha512-PfquyvrUTbqUBa4FLqGLBM5eaAlnNMuhqB7rXu+OuGfmmvx+Ste8dmuNEDNC+GTcqtyvDmfCL/EWz3z0XXjncA==" + "version": "0.17.4", + "resolved": "https://registry.npmjs.org/material-symbols/-/material-symbols-0.17.4.tgz", + "integrity": "sha512-5zI+rSzAidMJxAIrQCVwnp4rMjFnx8aQg68lfFXtaDeksZzJ7m8eDl16y9bRNxMosuYbLKeDHDbOWHPJJTSLhQ==" }, "node_modules/memoize-one": { "version": "6.0.0", @@ -6014,9 +6006,9 @@ } }, "node_modules/monaco-editor": { - "version": "0.47.0", - "resolved": "https://registry.npmjs.org/monaco-editor/-/monaco-editor-0.47.0.tgz", - "integrity": "sha512-VabVvHvQ9QmMwXu4du008ZDuyLnHs9j7ThVFsiJoXSOQk18+LF89N4ADzPbFenm0W4V2bGHnFBztIRQTgBfxzw==", + "version": "0.48.0", + "resolved": "https://registry.npmjs.org/monaco-editor/-/monaco-editor-0.48.0.tgz", + "integrity": "sha512-goSDElNqFfw7iDHMg8WDATkfcyeLTNpBHQpO8incK6p5qZt5G/1j41X0xdGzpIkGojGXM+QiRQyLjnfDVvrpwA==", "peer": true }, "node_modules/ms": { @@ -6082,11 +6074,11 @@ "dev": true }, "node_modules/next": { - "version": "14.2.2", - "resolved": "https://registry.npmjs.org/next/-/next-14.2.2.tgz", - "integrity": "sha512-oGwUaa2bCs47FbuxWMpOoXtBMPYpvTPgdZr3UAo+pu7Ns00z9otmYpoeV1HEiYL06AlRQQIA/ypK526KjJfaxg==", + "version": "14.2.3", + "resolved": "https://registry.npmjs.org/next/-/next-14.2.3.tgz", + "integrity": "sha512-dowFkFTR8v79NPJO4QsBUtxv0g9BrS/phluVpMAt2ku7H+cbcBJlopXjkWlwxrk/xGqMemr7JkGPGemPrLLX7A==", "dependencies": { - "@next/env": "14.2.2", + "@next/env": "14.2.3", "@swc/helpers": "0.5.5", "busboy": "1.6.0", "caniuse-lite": "^1.0.30001579", @@ -6101,15 +6093,15 @@ "node": ">=18.17.0" }, "optionalDependencies": { - "@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" + "@next/swc-darwin-arm64": "14.2.3", + "@next/swc-darwin-x64": "14.2.3", + "@next/swc-linux-arm64-gnu": "14.2.3", + "@next/swc-linux-arm64-musl": "14.2.3", + "@next/swc-linux-x64-gnu": "14.2.3", + "@next/swc-linux-x64-musl": "14.2.3", + "@next/swc-win32-arm64-msvc": "14.2.3", + "@next/swc-win32-ia32-msvc": "14.2.3", + "@next/swc-win32-x64-msvc": "14.2.3" }, "peerDependencies": { "@opentelemetry/api": "^1.1.0", @@ -6376,17 +6368,17 @@ } }, "node_modules/optionator": { - "version": "0.9.3", - "resolved": "https://registry.npmjs.org/optionator/-/optionator-0.9.3.tgz", - "integrity": "sha512-JjCoypp+jKn1ttEFExxhetCKeJt9zhAgAve5FXHixTvFDW/5aEktX9bufBKLRRMdU7bNtpLfcGu94B3cdEJgjg==", + "version": "0.9.4", + "resolved": "https://registry.npmjs.org/optionator/-/optionator-0.9.4.tgz", + "integrity": "sha512-6IpQ7mKUxRcZNLIObR0hz7lxsapSSIYNZJwXPGeF0mTVqGKFIXj1DQcMoT22S3ROcLyY/rz0PWaWZ9ayWmad9g==", "dev": true, "dependencies": { - "@aashutoshrathi/word-wrap": "^1.2.3", "deep-is": "^0.1.3", "fast-levenshtein": "^2.0.6", "levn": "^0.4.1", "prelude-ls": "^1.2.1", - "type-check": "^0.4.0" + "type-check": "^0.4.0", + "word-wrap": "^1.2.5" }, "engines": { "node": ">= 0.8.0" @@ -6515,9 +6507,9 @@ } }, "node_modules/path-scurry/node_modules/lru-cache": { - "version": "10.2.0", - "resolved": "https://registry.npmjs.org/lru-cache/-/lru-cache-10.2.0.tgz", - "integrity": "sha512-2bIM8x+VAf6JT4bKAljS1qUWgMsqZRPGJS6FSahIMPVvctcNhyVp7AJu7quxOW9jwkryBReKZY5tY5JYv2n/7Q==", + "version": "10.2.2", + "resolved": "https://registry.npmjs.org/lru-cache/-/lru-cache-10.2.2.tgz", + "integrity": "sha512-9hp3Vp2/hFQUiIwKo8XCeFVnrg8Pk3TYNPIR7tJADKi5YfcF7vEaK7avFHTlSy3kOKYaJQaalfEo6YuXdceBOQ==", "engines": { "node": "14 || >=16.14" } @@ -6767,9 +6759,9 @@ } }, "node_modules/postcss-load-config/node_modules/yaml": { - "version": "2.4.1", - "resolved": "https://registry.npmjs.org/yaml/-/yaml-2.4.1.tgz", - "integrity": "sha512-pIXzoImaqmfOrL7teGUBt/T7ZDnyeGBWyXQBvOVhLkWLN37GXv8NMLK406UY6dS51JfcQHsmcW5cJ441bHg6Lg==", + "version": "2.4.2", + "resolved": "https://registry.npmjs.org/yaml/-/yaml-2.4.2.tgz", + "integrity": "sha512-B3VqDZ+JAg1nZpaEmWtTXUlBneoGx6CPM9b0TENK6aoSu5t73dItudwdgmi6tHlIZZId4dZ9skcAQ2UbcyAeVA==", "bin": { "yaml": "bin.mjs" }, @@ -6813,9 +6805,9 @@ "integrity": "sha512-1NNCs6uurfkVbeXG4S8JFT9t19m45ICnif8zWLd5oPSZ50QnwMfK+H3jv408d4jw/7Bttv5axS5IiHoLaVNHeQ==" }, "node_modules/preact": { - "version": "10.20.2", - "resolved": "https://registry.npmjs.org/preact/-/preact-10.20.2.tgz", - "integrity": "sha512-S1d1ernz3KQ+Y2awUxKakpfOg2CEmJmwOP+6igPx6dgr6pgDvenqYviyokWso2rhHvGtTlWWnJDa7RaPbQerTg==", + "version": "10.21.0", + "resolved": "https://registry.npmjs.org/preact/-/preact-10.21.0.tgz", + "integrity": "sha512-aQAIxtzWEwH8ou+OovWVSVNlFImL7xUCwJX3YMqA3U8iKCNC34999fFOnWjYNsylgfPgMexpbk7WYOLtKr/mxg==", "funding": { "type": "opencollective", "url": "https://opencollective.com/preact" @@ -6882,13 +6874,13 @@ "integrity": "sha512-WuxUnVtlWL1OfZFQFuqvnvs6MiAGk9UNsBostyBOB0Is9wb5uRESevA6rnl/rkksXaGX3GzZhPup5d6Vp1nFew==" }, "node_modules/prisma": { - "version": "5.12.1", - "resolved": "https://registry.npmjs.org/prisma/-/prisma-5.12.1.tgz", - "integrity": "sha512-SkMnb6wyIxTv9ACqiHBI2u9gD6y98qXRoCoLEnZsF6yee5Qg828G+ARrESN+lQHdw4maSZFFSBPPDpvSiVTo0Q==", + "version": "5.13.0", + "resolved": "https://registry.npmjs.org/prisma/-/prisma-5.13.0.tgz", + "integrity": "sha512-kGtcJaElNRAdAGsCNykFSZ7dBKpL14Cbs+VaQ8cECxQlRPDjBlMHNFYeYt0SKovAVy2Y65JXQwB3A5+zIQwnTg==", "devOptional": true, "hasInstallScript": true, "dependencies": { - "@prisma/engines": "5.12.1" + "@prisma/engines": "5.13.0" }, "bin": { "prisma": "build/index.js" @@ -6975,9 +6967,9 @@ } }, "node_modules/react": { - "version": "18.2.0", - "resolved": "https://registry.npmjs.org/react/-/react-18.2.0.tgz", - "integrity": "sha512-/3IjMdb2L9QbBdWiW5e3P2/npwMBaU9mHCSCUzNln0ZCYbcfTsGbTJrU/kGemdH2IWmB2ioZ+zkxtmq6g09fGQ==", + "version": "18.3.1", + "resolved": "https://registry.npmjs.org/react/-/react-18.3.1.tgz", + "integrity": "sha512-wS+hAgJShR0KhEvPJArfuPVN1+Hz1t0Y6n5jLrGQbkb4urgPE/0Rve+1kMB1v/oWgHgm4WIcV+i7F2pTVj+2iQ==", "dependencies": { "loose-envify": "^1.1.0" }, @@ -6999,15 +6991,15 @@ } }, "node_modules/react-dom": { - "version": "18.2.0", - "resolved": "https://registry.npmjs.org/react-dom/-/react-dom-18.2.0.tgz", - "integrity": "sha512-6IMTriUmvsjHUjNtEDudZfuDQUoWXVxKHhlEGSk81n4YFS+r/Kl99wXiwlVXtPBtJenozv2P+hxDsw9eA7Xo6g==", + "version": "18.3.1", + "resolved": "https://registry.npmjs.org/react-dom/-/react-dom-18.3.1.tgz", + "integrity": "sha512-5m4nQKp+rZRb09LNH59GM4BxTh9251/ylbKIbpe7TpGxfJ+9kv6BLkLBXIjjspbgbnIBNqlI23tRnTWT0snUIw==", "dependencies": { "loose-envify": "^1.1.0", - "scheduler": "^0.23.0" + "scheduler": "^0.23.2" }, "peerDependencies": { - "react": "^18.2.0" + "react": "^18.3.1" } }, "node_modules/react-is": { @@ -7440,9 +7432,9 @@ "optional": true }, "node_modules/scheduler": { - "version": "0.23.0", - "resolved": "https://registry.npmjs.org/scheduler/-/scheduler-0.23.0.tgz", - "integrity": "sha512-CtuThmgHNg7zIZWAXi3AsyIzA3n4xx7aNyjwC2VJldO2LMVDhFK+63xGqq6CsJH4rTAt6/M+N4GhZiDYPx9eUw==", + "version": "0.23.2", + "resolved": "https://registry.npmjs.org/scheduler/-/scheduler-0.23.2.tgz", + "integrity": "sha512-UOShsPwz7NrMUqhR6t0hWjFduvOzbtv7toDH1/hIrfRNIDBnnBWd0CwJTGvTpngVlmwGCdP9/Zl/tVrDqcuYzQ==", "dependencies": { "loose-envify": "^1.1.0" } @@ -7467,15 +7459,15 @@ } }, "node_modules/schema-utils/node_modules/ajv": { - "version": "8.12.0", - "resolved": "https://registry.npmjs.org/ajv/-/ajv-8.12.0.tgz", - "integrity": "sha512-sRu1kpcO9yLtYxBKvqfTeh9KzZEwO3STyX1HT+4CaDzC6HpTGYhIhPIzj9XuKU7KYDwnaeh5hcOwjy1QuJzBPA==", + "version": "8.13.0", + "resolved": "https://registry.npmjs.org/ajv/-/ajv-8.13.0.tgz", + "integrity": "sha512-PRA911Blj99jR5RMeTunVbNXMF6Lp4vZXnk5GQjcnUWUTsrXtekg/pnmFFI2u/I36Y/2bITGS30GZCXei6uNkA==", "dev": true, "dependencies": { - "fast-deep-equal": "^3.1.1", + "fast-deep-equal": "^3.1.3", "json-schema-traverse": "^1.0.0", "require-from-string": "^2.0.2", - "uri-js": "^4.2.2" + "uri-js": "^4.4.1" }, "funding": { "type": "github", @@ -7871,12 +7863,12 @@ } }, "node_modules/styled-components": { - "version": "6.1.8", - "resolved": "https://registry.npmjs.org/styled-components/-/styled-components-6.1.8.tgz", - "integrity": "sha512-PQ6Dn+QxlWyEGCKDS71NGsXoVLKfE1c3vApkvDYS5KAK+V8fNWGhbSUEo9Gg2iaID2tjLXegEW3bZDUGpofRWw==", + "version": "6.1.9", + "resolved": "https://registry.npmjs.org/styled-components/-/styled-components-6.1.9.tgz", + "integrity": "sha512-aBOqs0uMsYufFXSE4q6cA6Ty1fwZuMk4BJRHfiGSna59F1otnxiDelwhN4fEwmBtIymmF0ZqXHnpSigr2ps9Cg==", "dependencies": { "@emotion/is-prop-valid": "1.2.1", - "@emotion/unitless": "0.8.0", + "@emotion/unitless": "0.8.1", "@types/stylis": "4.2.0", "css-to-react-native": "3.2.0", "csstype": "3.1.2", @@ -7897,11 +7889,6 @@ "react-dom": ">= 16.8.0" } }, - "node_modules/styled-components/node_modules/@emotion/unitless": { - "version": "0.8.0", - "resolved": "https://registry.npmjs.org/@emotion/unitless/-/unitless-0.8.0.tgz", - "integrity": "sha512-VINS5vEYAscRl2ZUDiT3uMPlrFQupiKgHz5AA4bCH1miKBg4qtwkim1qPmJj/4WG6TreYMY111rEFsjupcOKHw==" - }, "node_modules/styled-components/node_modules/csstype": { "version": "3.1.2", "resolved": "https://registry.npmjs.org/csstype/-/csstype-3.1.2.tgz", @@ -8112,9 +8099,9 @@ } }, "node_modules/terser": { - "version": "5.30.4", - "resolved": "https://registry.npmjs.org/terser/-/terser-5.30.4.tgz", - "integrity": "sha512-xRdd0v64a8mFK9bnsKVdoNP9GQIKUAaJPTaqEQDL4w/J8WaW4sWXXoMZ+6SimPkfT5bElreXf8m9HnmPc3E1BQ==", + "version": "5.31.0", + "resolved": "https://registry.npmjs.org/terser/-/terser-5.31.0.tgz", + "integrity": "sha512-Q1JFAoUKE5IMfI4Z/lkE/E6+SwgzO+x4tq4v1AyBLRj8VSYvRO6A/rQrPg1yud4g0En9EKI1TvFRF2tQFcoUkg==", "dev": true, "dependencies": { "@jridgewell/source-map": "^0.3.3", @@ -8443,9 +8430,9 @@ } }, "node_modules/update-browserslist-db": { - "version": "1.0.13", - "resolved": "https://registry.npmjs.org/update-browserslist-db/-/update-browserslist-db-1.0.13.tgz", - "integrity": "sha512-xebP81SNcPuNpPP3uzeW1NYXxI3rxyJzF3pD6sH4jE7o/IX+WtSpwnVU+qIsDPyk0d3hmFQ7mjqc6AtV604hbg==", + "version": "1.0.14", + "resolved": "https://registry.npmjs.org/update-browserslist-db/-/update-browserslist-db-1.0.14.tgz", + "integrity": "sha512-JixKH8GR2pWYshIPUg/NujK3JO7JiqEEUiNArE86NQyrgUuZeTlZQN3xuS/yiV5Kb48ev9K6RqNkaJjXsdg7Jw==", "dev": true, "funding": [ { @@ -8462,7 +8449,7 @@ } ], "dependencies": { - "escalade": "^3.1.1", + "escalade": "^3.1.2", "picocolors": "^1.0.0" }, "bin": { @@ -8536,9 +8523,9 @@ } }, "node_modules/use-sync-external-store": { - "version": "1.2.0", - "resolved": "https://registry.npmjs.org/use-sync-external-store/-/use-sync-external-store-1.2.0.tgz", - "integrity": "sha512-eEgnFxGQ1Ife9bzYs6VLi8/4X6CObHMw9Qr9tPY43iKwsPw8xE8+EFsf/2cFZ5S3esXgpWgtSCtLNS41F+sKPA==", + "version": "1.2.2", + "resolved": "https://registry.npmjs.org/use-sync-external-store/-/use-sync-external-store-1.2.2.tgz", + "integrity": "sha512-PElTlVMwpblvbNqQ82d2n6RjStvdSoNe9FG28kNfz3WiXilJm4DdNkEzRhCZuIDwY8U08WVihhGR5iRqAwfDiw==", "peerDependencies": { "react": "^16.8.0 || ^17.0.0 || ^18.0.0" } @@ -8793,6 +8780,15 @@ "url": "https://github.com/sponsors/ljharb" } }, + "node_modules/word-wrap": { + "version": "1.2.5", + "resolved": "https://registry.npmjs.org/word-wrap/-/word-wrap-1.2.5.tgz", + "integrity": "sha512-BN22B5eaMMI9UMtjrGd5g5eCYPpCPDUy0FJXbYsaT5zYxjFOckS53SQDE3pWkVoWpHXVb3BrYcEN4Twa55B5cA==", + "dev": true, + "engines": { + "node": ">=0.10.0" + } + }, "node_modules/wrap-ansi": { "version": "8.1.0", "resolved": "https://registry.npmjs.org/wrap-ansi/-/wrap-ansi-8.1.0.tgz", @@ -8979,9 +8975,9 @@ } }, "node_modules/zod": { - "version": "3.23.0", - "resolved": "https://registry.npmjs.org/zod/-/zod-3.23.0.tgz", - "integrity": "sha512-OFLT+LTocvabn6q76BTwVB0hExEBS0IduTr3cqZyMqEDbOnYmcU+y0tUAYbND4uwclpBGi4I4UUBGzylWpjLGA==", + "version": "3.23.5", + "resolved": "https://registry.npmjs.org/zod/-/zod-3.23.5.tgz", + "integrity": "sha512-fkwiq0VIQTksNNA131rDOsVJcns0pfVUjHzLrNBiF/O/Xxb5lQyEXkhZWcJ7npWsYlvs+h0jFWXXy4X46Em1JA==", "funding": { "url": "https://github.com/sponsors/colinhacks" } diff --git a/ui/package.json b/ui/package.json index cbdb2ec6fa..65d07edb40 100644 --- a/ui/package.json +++ b/ui/package.json @@ -12,7 +12,7 @@ "dependencies": { "@grpc/grpc-js": "^1.10.6", "@monaco-editor/react": "^4.6.0", - "@prisma/client": "^5.12.1", + "@prisma/client": "^5.13.0", "@radix-ui/react-checkbox": "^1.0.4", "@radix-ui/react-collapsible": "^1.0.3", "@radix-ui/react-dialog": "^1.0.5", @@ -28,42 +28,42 @@ "@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.16.1", + "@tremor/react": "^3.16.2", "@types/node": "^20.12.7", - "@types/react": "^18.2.79", - "@types/react-dom": "^18.2.25", + "@types/react": "^18.3.1", + "@types/react-dom": "^18.3.0", "classnames": "^2.5.1", "long": "^5.2.3", - "lucide-react": "^0.372.0", - "material-symbols": "^0.17.2", + "lucide-react": "^0.376.0", + "material-symbols": "^0.17.4", "moment": "^2.30.1", "moment-timezone": "^0.5.45", - "next": "^14.2.2", + "next": "^14.2.3", "next-auth": "^4.24.7", "prop-types": "^15.8.1", "protobufjs": "^7.2.6", - "react": "18.2.0", - "react-dom": "18.2.0", + "react": "^18.3.1", + "react-dom": "^18.3.1", "react-select": "^5.8.0", "react-spinners": "^0.13.8", "react-toastify": "^10.0.5", - "styled-components": "^6.1.8", + "styled-components": "^6.1.9", "swr": "^2.2.5", "usehooks-ts": "^3.1.0", - "zod": "^3.23.0" + "zod": "^3.23.5" }, "devDependencies": { "autoprefixer": "^10.4.19", "copy-webpack-plugin": "^12.0.2", "eslint": "^8.57.0", - "eslint-config-next": "^14.2.2", + "eslint-config-next": "^14.2.3", "eslint-config-prettier": "^9.1.0", "gh-pages": "^6.1.1", "less": "^4.2.0", "postcss": "^8.4.38", "prettier": "^3.2.5", "prettier-plugin-organize-imports": "^3.2.4", - "prisma": "^5.12.1", + "prisma": "^5.13.0", "string-width": "^7.1.0", "tailwindcss": "^3.4.3", "tailwindcss-animate": "^1.0.7", From 5b6ca385d5fe4c556201326e0db8e71c88249e4c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Thu, 2 May 2024 19:10:18 +0000 Subject: [PATCH 11/41] More MySQL rewrite rules (#1661) 1. also remove CAST from OFFSET 2. remove explicit `public` schema. Needed for UDF passthrough 3. `jsonb` isn't in MySQL, so rewrite '...'::jsonb literals, in particular mapping [1,2,3] to array for StarRocks This can be leveraged to use array functions in StarRocks: ```sql CREATE OR REPLACE FUNCTION public.cardinality(jsonb) RETURNS int AS $$ BEGIN RETURN jsonb_array_length($1); END $$ LANGUAGE PLPGSQL IMMUTABLE; CREATE OR REPLACE FUNCTION public.array_intersect(jsonb, jsonb) RETURNS jsonb AS $$ BEGIN RETURN (select jsonb_agg(a) from jsonb_array_elements($1) a join jsonb_array_elements($2) b on a = b); END $$ LANGUAGE PLPGSQL IMMUTABLE; -- where postgres_fdw is specified in `extensions` option of fdw server ALTER EXTENSION postgres_fdw ADD FUNCTION public.cardinality(jsonb); ALTER EXTENSION postgres_fdw ADD FUNCTION public.array_intersect(jsonb, jsonb); ``` After which `select array_intersect(fdwtbl.col, '[1,2,3]') from fdwtbl` can be pushed down, but function will also work if postgres for some reason can't push down --- nexus/analyzer/src/lib.rs | 6 +--- nexus/analyzer/src/qrep.rs | 3 +- nexus/catalog/src/lib.rs | 10 ++---- nexus/flow-rs/src/grpc.rs | 8 ++--- nexus/peer-mysql/src/ast.rs | 54 +++++++++++++++++++++++++++++-- nexus/server/src/main.rs | 2 +- nexus/server/tests/server_test.rs | 4 +-- 7 files changed, 64 insertions(+), 23 deletions(-) diff --git a/nexus/analyzer/src/lib.rs b/nexus/analyzer/src/lib.rs index 0642966713..83bea8d59c 100644 --- a/nexus/analyzer/src/lib.rs +++ b/nexus/analyzer/src/lib.rs @@ -922,11 +922,7 @@ fn parse_db_options(db_type: DbType, with_options: &[SqlOption]) -> anyhow::Resu .context("no port specified")? .parse::() .context("unable to parse port as valid int")?, - user: opts - .get("user") - .cloned() - .unwrap_or_default() - .to_string(), + user: opts.get("user").cloned().unwrap_or_default().to_string(), password: opts .get("password") .cloned() diff --git a/nexus/analyzer/src/qrep.rs b/nexus/analyzer/src/qrep.rs index 692b093d94..41ab1aa40a 100644 --- a/nexus/analyzer/src/qrep.rs +++ b/nexus/analyzer/src/qrep.rs @@ -196,8 +196,7 @@ pub fn process_options( // If mode is upsert, we need unique key columns if opts.get("mode") == Some(&Value::String(String::from("upsert"))) - && (opts.get("unique_key_columns").is_none() - || opts.get("unique_key_columns") == Some(&Value::Array(vec![]))) + && opts.get("unique_key_columns").map(|ukc| ukc == &Value::Array(Vec::new())).unwrap_or(true) { anyhow::bail!("For upsert mode, unique_key_columns must be specified"); } diff --git a/nexus/catalog/src/lib.rs b/nexus/catalog/src/lib.rs index 0c3b65d2b8..e8a569f669 100644 --- a/nexus/catalog/src/lib.rs +++ b/nexus/catalog/src/lib.rs @@ -449,9 +449,9 @@ impl Catalog { ) .await?; - if job.flow_options.get("destination_table_name").is_none() { + let Some(destination_table_name) = job.flow_options.get("destination_table_name") else { return Err(anyhow!("destination_table_name not found in flow options")); - } + }; let _rows = self .pg @@ -462,11 +462,7 @@ impl Catalog { &source_peer_id, &destination_peer_id, &job.description, - &job.flow_options - .get("destination_table_name") - .unwrap() - .as_str() - .unwrap(), + &destination_table_name.as_str().unwrap(), &job.query_string, &serde_json::to_value(job.flow_options.clone()) .context("unable to serialize flow options")?, diff --git a/nexus/flow-rs/src/grpc.rs b/nexus/flow-rs/src/grpc.rs index 35382ca287..70c463e44c 100644 --- a/nexus/flow-rs/src/grpc.rs +++ b/nexus/flow-rs/src/grpc.rs @@ -199,9 +199,9 @@ impl FlowGrpcClient { for (key, value) in &job.flow_options { match value { Value::String(s) => match key.as_str() { - "destination_table_name" => cfg.destination_table_identifier = s.clone(), - "watermark_column" => cfg.watermark_column = s.clone(), - "watermark_table_name" => cfg.watermark_table = s.clone(), + "destination_table_name" => cfg.destination_table_identifier.clone_from(s), + "watermark_column" => cfg.watermark_column.clone_from(s), + "watermark_table_name" => cfg.watermark_table.clone_from(s), "mode" => { let mut wm = QRepWriteMode { write_type: QRepWriteType::QrepWriteModeAppend as i32, @@ -229,7 +229,7 @@ impl FlowGrpcClient { _ => return anyhow::Result::Err(anyhow::anyhow!("invalid mode {}", s)), } } - "staging_path" => cfg.staging_path = s.clone(), + "staging_path" => cfg.staging_path.clone_from(s), _ => return anyhow::Result::Err(anyhow::anyhow!("invalid str option {}", key)), }, Value::Number(n) => match key.as_str() { diff --git a/nexus/peer-mysql/src/ast.rs b/nexus/peer-mysql/src/ast.rs index 00c12b7dbf..e581cdbb88 100644 --- a/nexus/peer-mysql/src/ast.rs +++ b/nexus/peer-mysql/src/ast.rs @@ -1,15 +1,39 @@ use std::ops::ControlFlow; use peer_ast::flatten_expr_to_in_list; +use serde_json::{self, Value as JsonValue}; use sqlparser::ast::{ visit_expressions_mut, visit_function_arg_mut, visit_relations_mut, Array, BinaryOperator, - DataType, Expr, FunctionArgExpr, Query, + DataType, Expr, FunctionArgExpr, Offset, Query, Value, }; +fn json_to_expr(val: JsonValue) -> Expr { + match val { + JsonValue::Null => Expr::Value(Value::Null), + JsonValue::Bool(x) => Expr::Value(Value::Boolean(x)), + JsonValue::Number(x) => Expr::Value(Value::Number(x.to_string(), false)), + JsonValue::String(x) => Expr::Value(Value::SingleQuotedString(x)), + JsonValue::Array(x) => Expr::Array(Array { + elem: x.into_iter().map(json_to_expr).collect::>(), + named: false, + }), + JsonValue::Object(x) => Expr::Cast { + data_type: DataType::JSON, + expr: Box::new(Expr::Value(Value::SingleQuotedString( + JsonValue::Object(x).to_string(), + ))), + format: None, + }, + } +} + pub fn rewrite_query(peername: &str, query: &mut Query) { visit_relations_mut(query, |table| { // if peer name is first part of table name, remove first part - if peername.eq_ignore_ascii_case(&table.0[0].value) { + // remove `public.` to facilitate mysql global function push down + if table.0.len() > 1 + && (peername.eq_ignore_ascii_case(&table.0[0].value) || table.0[0].value == "public") + { table.0.remove(0); } ControlFlow::<()>::Continue(()) @@ -19,6 +43,16 @@ pub fn rewrite_query(peername: &str, query: &mut Query) { if let Some(Expr::Cast { expr, .. }) = &query.limit { query.limit = Some((**expr).clone()); } + if let Some(Offset { + value: Expr::Cast { expr, .. }, + rows, + }) = &query.offset + { + query.offset = Some(Offset { + value: (**expr).clone(), + rows: *rows, + }); + } visit_function_arg_mut(query, |node| { if let FunctionArgExpr::Expr(arg_expr) = node { @@ -34,6 +68,22 @@ pub fn rewrite_query(peername: &str, query: &mut Query) { named: true, }; *node = FunctionArgExpr::Expr(Expr::Array(rewritten_array)); + } else if let Expr::Cast { + data_type: DataType::JSONB, + expr, + .. + } = arg_expr + { + *node = match **expr { + Expr::Value(Value::SingleQuotedString(ref s)) => { + if let Ok(val) = serde_json::from_str::(s) { + FunctionArgExpr::Expr(json_to_expr(val)) + } else { + FunctionArgExpr::Expr((**expr).clone()) + } + } + _ => FunctionArgExpr::Expr((**expr).clone()), + }; } } diff --git a/nexus/server/src/main.rs b/nexus/server/src/main.rs index 51d2999890..176928a71b 100644 --- a/nexus/server/src/main.rs +++ b/nexus/server/src/main.rs @@ -572,7 +572,7 @@ impl NexusBackend { match qrep_config { Some(mut qrep_config) => { if query_string.is_some() { - qrep_config.query = query_string.as_ref().unwrap().clone(); + qrep_config.query.clone_from(query_string.as_ref().unwrap()); } qrep_config.dst_table_full_resync = true; diff --git a/nexus/server/tests/server_test.rs b/nexus/server/tests/server_test.rs index 49ec206e93..2ca7d02eb7 100644 --- a/nexus/server/tests/server_test.rs +++ b/nexus/server/tests/server_test.rs @@ -162,8 +162,8 @@ fn server_test() { output_file.flush().expect("Unable to flush output file"); } - let obtained_file = std::fs::read_to_string(&actual_output_path).unwrap(); - let expected_file = std::fs::read_to_string(&expected_output_path).unwrap(); + let obtained_file = std::fs::read_to_string(actual_output_path).unwrap(); + let expected_file = std::fs::read_to_string(expected_output_path).unwrap(); // if there is a mismatch, print the diff, along with the path. if obtained_file != expected_file { tracing::info!("failed: {file}"); From c1852cfe1a5236bc75c191aec04e3ffe680812a5 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Fri, 3 May 2024 02:56:53 +0530 Subject: [PATCH 12/41] BigQuery Add Column: Account for update limit (#1662) BigQuery has a [quota](https://cloud.google.com/bigquery/quotas#standard_tables) of 5 metadata update operations per table per 10 seconds. So currently if many columns are added in a PG to BQ CDC mirror, sync records fails with this error from BigQuery - even though we do `ADD COLUMN IF NOT EXISTS` (it still counts as an update operation) This PR accounts for this by manually checking the table metadata for the column before adding it. This will atleast ensure that on sync flow retry, the remaining columns get added Functionally tested --- flow/connectors/bigquery/bigquery.go | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/flow/connectors/bigquery/bigquery.go b/flow/connectors/bigquery/bigquery.go index 40682a5f75..ffaaf4490d 100644 --- a/flow/connectors/bigquery/bigquery.go +++ b/flow/connectors/bigquery/bigquery.go @@ -217,6 +217,21 @@ func (c *BigQueryConnector) ReplayTableSchemaDeltas( for _, addedColumn := range schemaDelta.AddedColumns { dstDatasetTable, _ := c.convertToDatasetTable(schemaDelta.DstTableName) + table := c.client.DatasetInProject(c.projectID, dstDatasetTable.dataset).Table(dstDatasetTable.table) + dstMetadata, metadataErr := table.Metadata(ctx) + if metadataErr != nil { + return fmt.Errorf("failed to get metadata for table %s: %w", schemaDelta.DstTableName, metadataErr) + } + + // check if the column already exists + for _, field := range dstMetadata.Schema { + if field.Name == addedColumn.Name { + c.logger.Info(fmt.Sprintf("[schema delta replay] column %s already exists in table %s", + addedColumn.Name, schemaDelta.DstTableName)) + continue + } + } + addedColumnBigQueryType := qValueKindToBigQueryTypeString(addedColumn.Type) query := c.client.Query(fmt.Sprintf( "ALTER TABLE %s ADD COLUMN IF NOT EXISTS `%s` %s", From 717b2999928f1f8e831f7ee45632aa85dc03c6df Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Thu, 2 May 2024 23:57:49 +0000 Subject: [PATCH 13/41] Fix #1662 (#1663) continue wasn't doing anything --- flow/connectors/bigquery/bigquery.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flow/connectors/bigquery/bigquery.go b/flow/connectors/bigquery/bigquery.go index ffaaf4490d..51812bdf4a 100644 --- a/flow/connectors/bigquery/bigquery.go +++ b/flow/connectors/bigquery/bigquery.go @@ -215,6 +215,7 @@ func (c *BigQueryConnector) ReplayTableSchemaDeltas( continue } + AddedColumnsLoop: for _, addedColumn := range schemaDelta.AddedColumns { dstDatasetTable, _ := c.convertToDatasetTable(schemaDelta.DstTableName) table := c.client.DatasetInProject(c.projectID, dstDatasetTable.dataset).Table(dstDatasetTable.table) @@ -228,7 +229,7 @@ func (c *BigQueryConnector) ReplayTableSchemaDeltas( if field.Name == addedColumn.Name { c.logger.Info(fmt.Sprintf("[schema delta replay] column %s already exists in table %s", addedColumn.Name, schemaDelta.DstTableName)) - continue + continue AddedColumnsLoop } } From 794e8ea87974e7cc184556a80ec3ef300254c243 Mon Sep 17 00:00:00 2001 From: Kevin Biju <52661649+heavycrystal@users.noreply.github.com> Date: Fri, 3 May 2024 19:37:23 +0530 Subject: [PATCH 14/41] ES tests: retry on search_phase_execution_exception (#1665) --- flow/e2e/elasticsearch/elasticsearch.go | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/flow/e2e/elasticsearch/elasticsearch.go b/flow/e2e/elasticsearch/elasticsearch.go index ed3389adb7..9a338c2969 100644 --- a/flow/e2e/elasticsearch/elasticsearch.go +++ b/flow/e2e/elasticsearch/elasticsearch.go @@ -83,8 +83,10 @@ func (s elasticsearchSuite) Peer() *protos.Peer { func (s elasticsearchSuite) countDocumentsInIndex(index string) int64 { res, err := s.esClient.Count().Index(index).Do(context.Background()) // index may not exist yet, don't error out for that - if err != nil && strings.Contains(err.Error(), "index_not_found_exception") { - return 0 + // search can occasionally fail, retry for that + if err != nil && (strings.Contains(err.Error(), "index_not_found_exception") || + strings.Contains(err.Error(), "search_phase_execution_exception")) { + return -1 } require.NoError(s.t, err, "failed to get count of documents in index") return res.Count From 95453f963203d0a3c2c65064e85031acd0a1929b Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Fri, 3 May 2024 20:34:28 +0530 Subject: [PATCH 15/41] Changes for Create Mirror (#1666) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - Fix XMIN validation - Change label to Create Watermark Table on Destination, and make it false by default - Catch putting existing mirror name in UI Screenshot 2024-05-03 at 8 10 21 PM --- flow/cmd/validate_mirror.go | 30 +++++++++++++++++++++++++ ui/app/mirrors/create/handlers.ts | 2 +- ui/app/mirrors/create/helpers/common.ts | 2 +- ui/app/mirrors/create/helpers/qrep.ts | 4 ++-- 4 files changed, 34 insertions(+), 4 deletions(-) diff --git a/flow/cmd/validate_mirror.go b/flow/cmd/validate_mirror.go index 59fdc37d0a..2c8998c4b2 100644 --- a/flow/cmd/validate_mirror.go +++ b/flow/cmd/validate_mirror.go @@ -6,6 +6,8 @@ import ( "fmt" "log/slog" + "github.com/jackc/pgx/v5/pgtype" + connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/generated/protos" @@ -15,6 +17,24 @@ import ( func (h *FlowRequestHandler) ValidateCDCMirror( ctx context.Context, req *protos.CreateCDCFlowRequest, ) (*protos.ValidateCDCMirrorResponse, error) { + mirrorExists, existCheckErr := h.CheckIfMirrorNameExists(ctx, req.ConnectionConfigs.FlowJobName) + if existCheckErr != nil { + slog.Error("/validatecdc failed to check if mirror name exists", slog.Any("error", existCheckErr)) + return &protos.ValidateCDCMirrorResponse{ + Ok: false, + }, existCheckErr + } + + if mirrorExists { + displayErr := fmt.Errorf("mirror with name %s already exists", req.ConnectionConfigs.FlowJobName) + h.alerter.LogNonFlowWarning(ctx, telemetry.CreateMirror, req.ConnectionConfigs.FlowJobName, + fmt.Sprint(displayErr), + ) + return &protos.ValidateCDCMirrorResponse{ + Ok: false, + }, displayErr + } + if req.ConnectionConfigs == nil { slog.Error("/validatecdc connection configs is nil") return &protos.ValidateCDCMirrorResponse{ @@ -98,3 +118,13 @@ func (h *FlowRequestHandler) ValidateCDCMirror( Ok: true, }, nil } + +func (h *FlowRequestHandler) CheckIfMirrorNameExists(ctx context.Context, mirrorName string) (bool, error) { + var nameExists pgtype.Bool + err := h.pool.QueryRow(ctx, "SELECT EXISTS(SELECT * FROM flows WHERE name = $1)", mirrorName).Scan(&nameExists) + if err != nil { + return true, fmt.Errorf("failed to check if mirror name exists: %v", err) + } + + return nameExists.Bool, nil +} diff --git a/ui/app/mirrors/create/handlers.ts b/ui/app/mirrors/create/handlers.ts index 6b808fedbc..d924dc2588 100644 --- a/ui/app/mirrors/create/handlers.ts +++ b/ui/app/mirrors/create/handlers.ts @@ -206,7 +206,7 @@ export const handleCreateQRep = async ( return; } - if (query === QRepQueryTemplate) { + if (query === QRepQueryTemplate && !xmin) { notifyErr('Please fill in the query box'); return; } diff --git a/ui/app/mirrors/create/helpers/common.ts b/ui/app/mirrors/create/helpers/common.ts index acbadadd5e..4237bc2176 100644 --- a/ui/app/mirrors/create/helpers/common.ts +++ b/ui/app/mirrors/create/helpers/common.ts @@ -50,6 +50,6 @@ export const blankQRepSetting = { writeMode: undefined, stagingPath: '', numRowsPerPartition: 100000, - setupWatermarkTableOnDestination: true, + setupWatermarkTableOnDestination: false, dstTableFullResync: false, }; diff --git a/ui/app/mirrors/create/helpers/qrep.ts b/ui/app/mirrors/create/helpers/qrep.ts index e4712459ac..392439ee77 100644 --- a/ui/app/mirrors/create/helpers/qrep.ts +++ b/ui/app/mirrors/create/helpers/qrep.ts @@ -28,7 +28,7 @@ export const qrepSettings: MirrorSetting[] = [ required: true, }, { - label: 'Create Destination Table', + label: 'Create Watermark Table On Destination', stateHandler: (value, setter) => setter((curr: QRepConfig) => ({ ...curr, @@ -36,7 +36,7 @@ export const qrepSettings: MirrorSetting[] = [ })), tips: 'Specify if you want to create the watermark table on the destination as-is, can be used for some queries.', type: 'switch', - default: true, + default: false, }, { label: 'Destination Table Name', From 05111666a80ce448bba8ace3bc16e06c3635a797 Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Fri, 3 May 2024 13:26:48 -0400 Subject: [PATCH 16/41] Improve default docker compose (#1669) use an explicit external network this removes environment related problems that can cause problems with quickstart. Also remove multiple flow workers in the default configuration --- dev-peerdb.sh | 6 ++++++ docker-compose-dev.yml | 39 +++++++++------------------------------ docker-compose.yml | 34 +++++++++------------------------- run-peerdb.sh | 6 ++++++ 4 files changed, 30 insertions(+), 55 deletions(-) diff --git a/dev-peerdb.sh b/dev-peerdb.sh index 796425f3dd..7c32151845 100755 --- a/dev-peerdb.sh +++ b/dev-peerdb.sh @@ -23,5 +23,11 @@ else EXTRA_ARGS="" fi +# check if peerdb_network exists if not create it +if ! $DOCKER network inspect peerdb_network &> /dev/null +then + $DOCKER network create peerdb_network +fi + export PEERDB_VERSION_SHA_SHORT=local-$(git rev-parse --short HEAD) exec $DOCKER -f docker-compose-dev.yml up --build $EXTRA_ARGS diff --git a/docker-compose-dev.yml b/docker-compose-dev.yml index c27ad6bda7..d7d93f6de9 100644 --- a/docker-compose-dev.yml +++ b/docker-compose-dev.yml @@ -1,5 +1,7 @@ version: "3.9" +name: peerdb-quickstart-dev + x-minio-config: &minio-config PEERDB_CLICKHOUSE_AWS_CREDENTIALS_AWS_ACCESS_KEY_ID: _peerdb_minioadmin PEERDB_CLICKHOUSE_AWS_CREDENTIALS_AWS_SECRET_ACCESS_KEY: _peerdb_minioadmin @@ -145,42 +147,14 @@ services: temporal-admin-tools: condition: service_healthy - flow-worker1: - container_name: flow-worker1 - build: - context: . - dockerfile: stacks/flow.Dockerfile - target: flow-worker - environment: - <<: [*catalog-config, *flow-worker-env, *minio-config] - depends_on: - temporal-admin-tools: - condition: service_healthy - - flow-worker2: - container_name: flow-worker2 + flow-worker: + container_name: flow-worker build: context: . dockerfile: stacks/flow.Dockerfile target: flow-worker environment: <<: [*catalog-config, *flow-worker-env, *minio-config] - profiles: - - multi - depends_on: - temporal-admin-tools: - condition: service_healthy - - flow-worker3: - container_name: flow-worker3 - build: - context: . - dockerfile: stacks/flow.Dockerfile - target: flow-worker - environment: - <<: [*catalog-config, *flow-worker-env, *minio-config] - profiles: - - multi depends_on: temporal-admin-tools: condition: service_healthy @@ -244,3 +218,8 @@ services: volumes: pgdata: minio-data: + +networks: + default: + external: true + name: peerdb_network diff --git a/docker-compose.yml b/docker-compose.yml index 75480ecdaf..42fa2f26fc 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -1,5 +1,7 @@ version: "3.9" +name: peerdb-quickstart + x-minio-config: &minio-config PEERDB_CLICKHOUSE_AWS_CREDENTIALS_AWS_ACCESS_KEY_ID: _peerdb_minioadmin PEERDB_CLICKHOUSE_AWS_CREDENTIALS_AWS_SECRET_ACCESS_KEY: _peerdb_minioadmin @@ -124,33 +126,11 @@ services: temporal-admin-tools: condition: service_healthy - flow-worker1: - container_name: flow-worker1 - image: ghcr.io/peerdb-io/flow-worker:latest-dev - environment: - <<: [*catalog-config, *flow-worker-env, *minio-config] - depends_on: - temporal-admin-tools: - condition: service_healthy - - flow-worker2: - container_name: flow-worker2 + flow-worker: + container_name: flow-worker image: ghcr.io/peerdb-io/flow-worker:latest-dev environment: <<: [*catalog-config, *flow-worker-env, *minio-config] - profiles: - - multi - depends_on: - temporal-admin-tools: - condition: service_healthy - - flow-worker3: - container_name: flow-worker3 - image: ghcr.io/peerdb-io/flow-worker:latest-dev - environment: - <<: [*catalog-config, *flow-worker-env, *minio-config] - profiles: - - multi depends_on: temporal-admin-tools: condition: service_healthy @@ -208,5 +188,9 @@ services: volumes: pgdata: - prometheusdata: minio-data: + +networks: + default: + external: true + name: peerdb_network diff --git a/run-peerdb.sh b/run-peerdb.sh index 20eb6dcf07..002bf93679 100755 --- a/run-peerdb.sh +++ b/run-peerdb.sh @@ -7,5 +7,11 @@ then exit 1 fi +# check if peerdb_network exists if not create it +if ! docker network inspect peerdb_network &> /dev/null +then + docker network create peerdb_network +fi + docker compose pull docker compose -f docker-compose.yml up --no-attach catalog --no-attach temporal --no-attach temporal-ui --no-attach temporal-admin-tools From 1769b3d4eeb2ef64889683d9b6c1cbd5231719da Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Sat, 4 May 2024 06:58:38 +0530 Subject: [PATCH 17/41] API: Fix mirror name validation for query layer, resync (#1668) Ensure mirror name validation is bypassed for query layer (where flow job entry happens before hand) and resync (where mirror name has to be the same) functionally tested --- flow/cmd/validate_mirror.go | 32 +++++++++++++++++--------------- 1 file changed, 17 insertions(+), 15 deletions(-) diff --git a/flow/cmd/validate_mirror.go b/flow/cmd/validate_mirror.go index 2c8998c4b2..89e34d8ee4 100644 --- a/flow/cmd/validate_mirror.go +++ b/flow/cmd/validate_mirror.go @@ -17,22 +17,24 @@ import ( func (h *FlowRequestHandler) ValidateCDCMirror( ctx context.Context, req *protos.CreateCDCFlowRequest, ) (*protos.ValidateCDCMirrorResponse, error) { - mirrorExists, existCheckErr := h.CheckIfMirrorNameExists(ctx, req.ConnectionConfigs.FlowJobName) - if existCheckErr != nil { - slog.Error("/validatecdc failed to check if mirror name exists", slog.Any("error", existCheckErr)) - return &protos.ValidateCDCMirrorResponse{ - Ok: false, - }, existCheckErr - } + if req.CreateCatalogEntry && !req.ConnectionConfigs.Resync { + mirrorExists, existCheckErr := h.CheckIfMirrorNameExists(ctx, req.ConnectionConfigs.FlowJobName) + if existCheckErr != nil { + slog.Error("/validatecdc failed to check if mirror name exists", slog.Any("error", existCheckErr)) + return &protos.ValidateCDCMirrorResponse{ + Ok: false, + }, existCheckErr + } - if mirrorExists { - displayErr := fmt.Errorf("mirror with name %s already exists", req.ConnectionConfigs.FlowJobName) - h.alerter.LogNonFlowWarning(ctx, telemetry.CreateMirror, req.ConnectionConfigs.FlowJobName, - fmt.Sprint(displayErr), - ) - return &protos.ValidateCDCMirrorResponse{ - Ok: false, - }, displayErr + if mirrorExists { + displayErr := fmt.Errorf("mirror with name %s already exists", req.ConnectionConfigs.FlowJobName) + h.alerter.LogNonFlowWarning(ctx, telemetry.CreateMirror, req.ConnectionConfigs.FlowJobName, + fmt.Sprint(displayErr), + ) + return &protos.ValidateCDCMirrorResponse{ + Ok: false, + }, displayErr + } } if req.ConnectionConfigs == nil { From bdbdfca6ac53da16af96cfe0b16fe0e0c3bf4338 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Sat, 4 May 2024 14:15:44 +0000 Subject: [PATCH 18/41] mysql: limit 0 describe (#1670) --- nexus/peer-mysql/src/lib.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/nexus/peer-mysql/src/lib.rs b/nexus/peer-mysql/src/lib.rs index d57e83b932..6868a53a18 100644 --- a/nexus/peer-mysql/src/lib.rs +++ b/nexus/peer-mysql/src/lib.rs @@ -9,7 +9,7 @@ use peer_cursor::{ }; use pgwire::error::{ErrorInfo, PgWireError, PgWireResult}; use pt::peerdb_peers::MySqlConfig; -use sqlparser::ast::{CloseCursor, Declare, FetchDirection, Statement}; +use sqlparser::ast::{CloseCursor, Declare, Expr, FetchDirection, Statement, Value}; use stream::MyRecordStream; pub struct MySqlQueryExecutor { @@ -202,6 +202,7 @@ impl QueryExecutor for MySqlQueryExecutor { Statement::Query(query) => { let mut query = query.clone(); ast::rewrite_query(&self.peer_name, &mut query); + query.limit = Some(Expr::Value(Value::Number(String::from("0"), false))); Ok(Some(self.query_schema(query.to_string()).await?)) } Statement::Declare { stmts } => { From 524d77e173377b2079911e3c435522a47948d21e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Mon, 6 May 2024 12:14:39 +0000 Subject: [PATCH 19/41] Lua decimal math (#1671) --- flow/pua/peerdb.go | 68 ++++++++++++++++++++++++++++++++++++++++++---- 1 file changed, 63 insertions(+), 5 deletions(-) diff --git a/flow/pua/peerdb.go b/flow/pua/peerdb.go index 540065970f..7330cb9d09 100644 --- a/flow/pua/peerdb.go +++ b/flow/pua/peerdb.go @@ -3,6 +3,7 @@ package pua import ( "bytes" "fmt" + "math/big" "time" "github.com/google/uuid" @@ -72,6 +73,7 @@ func RegisterTypes(ls *lua.LState) { mt.RawSetString("__eq", ls.NewFunction(LuaBigIntEq)) mt.RawSetString("__le", ls.NewFunction(LuaBigIntLe)) mt.RawSetString("__lt", ls.NewFunction(LuaBigIntLt)) + mt.RawSetString("__unm", ls.NewFunction(LuaBigIntUnm)) mt = shared.LuaDecimal.NewMetatable(ls) mt.RawSetString("__index", ls.NewFunction(LuaDecimalIndex)) @@ -79,6 +81,25 @@ func RegisterTypes(ls *lua.LState) { mt.RawSetString("__eq", ls.NewFunction(LuaDecimalEq)) mt.RawSetString("__le", ls.NewFunction(LuaDecimalLe)) mt.RawSetString("__lt", ls.NewFunction(LuaDecimalLt)) + mt.RawSetString("__unm", ls.NewFunction(LuaDecimalUnm)) + mt.RawSetString("__add", ls.NewFunction(decimalBinop(func(d1 decimal.Decimal, d2 decimal.Decimal) decimal.Decimal { + return d1.Add(d2) + }))) + mt.RawSetString("__sub", ls.NewFunction(decimalBinop(func(d1 decimal.Decimal, d2 decimal.Decimal) decimal.Decimal { + return d1.Sub(d2) + }))) + mt.RawSetString("__mul", ls.NewFunction(decimalBinop(func(d1 decimal.Decimal, d2 decimal.Decimal) decimal.Decimal { + return d1.Mul(d2) + }))) + mt.RawSetString("__div", ls.NewFunction(decimalBinop(func(d1 decimal.Decimal, d2 decimal.Decimal) decimal.Decimal { + return d1.Div(d2) + }))) + mt.RawSetString("__mod", ls.NewFunction(decimalBinop(func(d1 decimal.Decimal, d2 decimal.Decimal) decimal.Decimal { + return d1.Mod(d2) + }))) + mt.RawSetString("__pow", ls.NewFunction(decimalBinop(func(d1 decimal.Decimal, d2 decimal.Decimal) decimal.Decimal { + return d1.Pow(d2) + }))) mt.RawSetString("__msgpack", ls.NewFunction(LuaDecimalString)) peerdb := ls.NewTable() @@ -330,19 +351,37 @@ func LuaUUID(ls *lua.LState) int { return 1 } -func LuaParseDecimal(ls *lua.LState) int { - switch v := ls.Get(1).(type) { +func LVAsDecimal(ls *lua.LState, lv lua.LValue) decimal.Decimal { + switch v := lv.(type) { case lua.LNumber: - ls.Push(shared.LuaDecimal.New(ls, decimal.NewFromFloat(float64(v)))) + return decimal.NewFromFloat(float64(v)) case lua.LString: d, err := decimal.NewFromString(string(v)) if err != nil { ls.RaiseError(err.Error()) } - ls.Push(shared.LuaDecimal.New(ls, d)) + return d + case *lua.LUserData: + switch v := v.Value.(type) { + case int64: + return decimal.NewFromInt(v) + case uint64: + return decimal.NewFromUint64(v) + case *big.Int: + return decimal.NewFromBigInt(v, 0) + case decimal.Decimal: + return v + default: + ls.RaiseError("cannot create decimal from %T", v) + } default: - ls.RaiseError("cannot create decimal from " + v.Type().String()) + ls.RaiseError("cannot create decimal from %s", v.Type()) } + return decimal.Decimal{} +} + +func LuaParseDecimal(ls *lua.LState) int { + ls.Push(shared.LuaDecimal.New(ls, LVAsDecimal(ls, ls.Get(1)))) return 1 } @@ -449,6 +488,12 @@ func LuaBigIntString(ls *lua.LState) int { return 1 } +func LuaBigIntUnm(ls *lua.LState) int { + bi := shared.LuaBigInt.StartMethod(ls) + ls.Push(shared.LuaBigInt.New(ls, new(big.Int).Neg(bi))) + return 1 +} + func LuaBigIntEq(ls *lua.LState) int { t1 := shared.LuaBigInt.StartMethod(ls) _, t2 := shared.LuaBigInt.Check(ls, 2) @@ -497,6 +542,19 @@ func LuaDecimalString(ls *lua.LState) int { return 1 } +func LuaDecimalUnm(ls *lua.LState) int { + num := shared.LuaDecimal.StartMethod(ls) + ls.Push(shared.LuaDecimal.New(ls, num.Neg())) + return 1 +} + +func decimalBinop(f func(d1 decimal.Decimal, d2 decimal.Decimal) decimal.Decimal) func(ls *lua.LState) int { + return func(ls *lua.LState) int { + ls.Push(shared.LuaDecimal.New(ls, f(LVAsDecimal(ls, ls.Get(1)), LVAsDecimal(ls, ls.Get(2))))) + return 1 + } +} + func LuaDecimalEq(ls *lua.LState) int { t1 := shared.LuaDecimal.StartMethod(ls) _, t2 := shared.LuaDecimal.Check(ls, 2) From ea58d02d1301e81a081692a2ee7ea8163d9bf581 Mon Sep 17 00:00:00 2001 From: Kevin Biju <52661649+heavycrystal@users.noreply.github.com> Date: Mon, 6 May 2024 20:39:21 +0530 Subject: [PATCH 20/41] OpenTelemetry: initial metrics and export via otlpmetrichttp (#1664) `flow-worker` now exports `slotlag` and `open_connections` metrics (both are gauges) for each Postgres peer. Metrics enabled by the environment variable `ENABLE_OTEL_METRICS` and export endpoint set by the environment variable `OTEL_EXPORTER_OTLP_METRICS_ENDPOINT` on `flow-worker`. Metrics update at the frequency of the `RecordSlotSizesWorkflow`, which is currently 5 minutes. --- flow/activities/flowable.go | 30 ++++++- flow/cmd/api.go | 2 +- flow/cmd/cert.go | 2 +- flow/cmd/snapshot_worker.go | 2 +- flow/cmd/worker.go | 52 ++++++++++-- flow/connectors/core.go | 5 +- flow/connectors/postgres/postgres.go | 12 +++ flow/go.mod | 9 ++- flow/go.sum | 12 ++- flow/main.go | 14 +++- flow/otel_metrics/otel_manager.go | 54 +++++++++++++ flow/otel_metrics/sync_gauges.go | 114 +++++++++++++++++++++++++++ 12 files changed, 288 insertions(+), 20 deletions(-) create mode 100644 flow/otel_metrics/otel_manager.go create mode 100644 flow/otel_metrics/sync_gauges.go diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index 7c3a41ac58..9c03346734 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -11,6 +11,7 @@ import ( "github.com/jackc/pgx/v5" "github.com/jackc/pgx/v5/pgxpool" + "go.opentelemetry.io/otel/metric" "go.temporal.io/sdk/activity" "go.temporal.io/sdk/log" "go.temporal.io/sdk/temporal" @@ -26,6 +27,7 @@ import ( "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/otel_metrics" "github.com/PeerDB-io/peer-flow/peerdbenv" "github.com/PeerDB-io/peer-flow/shared" ) @@ -44,6 +46,7 @@ type FlowableActivity struct { CatalogPool *pgxpool.Pool Alerter *alerting.Alerter CdcCache map[string]CdcCacheEntry + OtelManager *otel_metrics.OtelManager CdcCacheRw sync.RWMutex } @@ -592,7 +595,32 @@ func (a *FlowableActivity) RecordSlotSizes(ctx context.Context) error { if ctx.Err() != nil { return } - err = srcConn.HandleSlotInfo(ctx, a.Alerter, a.CatalogPool, slotName, peerName) + + var slotLagGauge *otel_metrics.Float64Gauge + var openConnectionsGauge *otel_metrics.Int64Gauge + if a.OtelManager != nil { + slotLagGauge, err = otel_metrics.GetOrInitFloat64Gauge(a.OtelManager.Meter, + a.OtelManager.Float64GaugesCache, + "cdc_slot_lag", + metric.WithUnit("MB"), + metric.WithDescription("Postgres replication slot lag in MB")) + if err != nil { + logger.Error("Failed to get slot lag gauge", slog.Any("error", err)) + return + } + + openConnectionsGauge, err = otel_metrics.GetOrInitInt64Gauge(a.OtelManager.Meter, + a.OtelManager.Int64GaugesCache, + "open_connections", + metric.WithDescription("Current open connections for PeerDB user")) + if err != nil { + logger.Error("Failed to get open connections gauge", slog.Any("error", err)) + return + } + } + + err = srcConn.HandleSlotInfo(ctx, a.Alerter, a.CatalogPool, slotName, peerName, + slotLagGauge, openConnectionsGauge) if err != nil { logger.Error("Failed to handle slot info", slog.Any("error", err)) } diff --git a/flow/cmd/api.go b/flow/cmd/api.go index 8b8be80c6e..5b010916db 100644 --- a/flow/cmd/api.go +++ b/flow/cmd/api.go @@ -96,7 +96,7 @@ func APIMain(ctx context.Context, args *APIServerParams) error { if args.TemporalCert != "" && args.TemporalKey != "" { slog.Info("Using temporal certificate/key for authentication") - certs, err := Base64DecodeCertAndKey(args.TemporalCert, args.TemporalKey) + certs, err := base64DecodeCertAndKey(args.TemporalCert, args.TemporalKey) if err != nil { return fmt.Errorf("unable to base64 decode certificate and key: %w", err) } diff --git a/flow/cmd/cert.go b/flow/cmd/cert.go index 9031d55b6c..9537b0f1e7 100644 --- a/flow/cmd/cert.go +++ b/flow/cmd/cert.go @@ -7,7 +7,7 @@ import ( "strings" ) -func Base64DecodeCertAndKey(cert string, key string) ([]tls.Certificate, error) { +func base64DecodeCertAndKey(cert string, key string) ([]tls.Certificate, error) { temporalCert := strings.TrimSpace(cert) certBytes, err := base64.StdEncoding.DecodeString(temporalCert) if err != nil { diff --git a/flow/cmd/snapshot_worker.go b/flow/cmd/snapshot_worker.go index 128759a474..ee49dbc039 100644 --- a/flow/cmd/snapshot_worker.go +++ b/flow/cmd/snapshot_worker.go @@ -33,7 +33,7 @@ func SnapshotWorkerMain(opts *SnapshotWorkerOptions) (client.Client, worker.Work } if opts.TemporalCert != "" && opts.TemporalKey != "" { - certs, err := Base64DecodeCertAndKey(opts.TemporalCert, opts.TemporalKey) + certs, err := base64DecodeCertAndKey(opts.TemporalCert, opts.TemporalKey) if err != nil { return nil, nil, fmt.Errorf("unable to process certificate and key: %w", err) } diff --git a/flow/cmd/worker.go b/flow/cmd/worker.go index 654cfad163..ef4f0bbca2 100644 --- a/flow/cmd/worker.go +++ b/flow/cmd/worker.go @@ -10,18 +10,20 @@ import ( "runtime" "github.com/grafana/pyroscope-go" + sdkmetric "go.opentelemetry.io/otel/sdk/metric" "go.temporal.io/sdk/client" "go.temporal.io/sdk/worker" "github.com/PeerDB-io/peer-flow/activities" "github.com/PeerDB-io/peer-flow/alerting" "github.com/PeerDB-io/peer-flow/logger" + "github.com/PeerDB-io/peer-flow/otel_metrics" "github.com/PeerDB-io/peer-flow/peerdbenv" "github.com/PeerDB-io/peer-flow/shared" peerflow "github.com/PeerDB-io/peer-flow/workflows" ) -type WorkerOptions struct { +type WorkerSetupOptions struct { TemporalHostPort string PyroscopeServer string TemporalNamespace string @@ -30,9 +32,16 @@ type WorkerOptions struct { TemporalMaxConcurrentActivities int TemporalMaxConcurrentWorkflowTasks int EnableProfiling bool + EnableOtelMetrics bool } -func setupPyroscope(opts *WorkerOptions) { +type workerSetupResponse struct { + Client client.Client + Worker worker.Worker + Cleanup func() +} + +func setupPyroscope(opts *WorkerSetupOptions) { if opts.PyroscopeServer == "" { log.Fatal("pyroscope server address is not set but profiling is enabled") } @@ -73,7 +82,7 @@ func setupPyroscope(opts *WorkerOptions) { } } -func WorkerMain(opts *WorkerOptions) (client.Client, worker.Worker, error) { +func WorkerSetup(opts *WorkerSetupOptions) (*workerSetupResponse, error) { if opts.EnableProfiling { setupPyroscope(opts) } @@ -86,9 +95,9 @@ func WorkerMain(opts *WorkerOptions) (client.Client, worker.Worker, error) { if opts.TemporalCert != "" && opts.TemporalKey != "" { slog.Info("Using temporal certificate/key for authentication") - certs, err := Base64DecodeCertAndKey(opts.TemporalCert, opts.TemporalKey) + certs, err := base64DecodeCertAndKey(opts.TemporalCert, opts.TemporalKey) if err != nil { - return nil, nil, fmt.Errorf("unable to process certificate and key: %w", err) + return nil, fmt.Errorf("unable to process certificate and key: %w", err) } connOptions := client.ConnectionOptions{ TLS: &tls.Config{ @@ -101,12 +110,12 @@ func WorkerMain(opts *WorkerOptions) (client.Client, worker.Worker, error) { conn, err := peerdbenv.GetCatalogConnectionPoolFromEnv(context.Background()) if err != nil { - return nil, nil, fmt.Errorf("unable to create catalog connection pool: %w", err) + return nil, fmt.Errorf("unable to create catalog connection pool: %w", err) } c, err := client.Dial(clientOptions) if err != nil { - return nil, nil, fmt.Errorf("unable to create Temporal client: %w", err) + return nil, fmt.Errorf("unable to create Temporal client: %w", err) } slog.Info("Created temporal client") @@ -128,11 +137,38 @@ func WorkerMain(opts *WorkerOptions) (client.Client, worker.Worker, error) { }) peerflow.RegisterFlowWorkerWorkflows(w) + var metricsProvider *sdkmetric.MeterProvider + var otelManager *otel_metrics.OtelManager + if opts.EnableOtelMetrics { + metricsProvider, err = otel_metrics.SetupOtelMetricsExporter("flow-worker") + if err != nil { + return nil, err + } + otelManager = &otel_metrics.OtelManager{ + MetricsProvider: metricsProvider, + Meter: metricsProvider.Meter("io.peerdb.flow-worker"), + Float64GaugesCache: make(map[string]*otel_metrics.Float64Gauge), + Int64GaugesCache: make(map[string]*otel_metrics.Int64Gauge), + } + } w.RegisterActivity(&activities.FlowableActivity{ CatalogPool: conn, Alerter: alerting.NewAlerter(context.Background(), conn), CdcCache: make(map[string]activities.CdcCacheEntry), + OtelManager: otelManager, }) - return c, w, nil + return &workerSetupResponse{ + Client: c, + Worker: w, + Cleanup: func() { + if otelManager != nil { + err := otelManager.MetricsProvider.Shutdown(context.Background()) + if err != nil { + slog.Error("Failed to shutdown metrics provider", slog.Any("error", err)) + } + } + c.Close() + }, + }, nil } diff --git a/flow/connectors/core.go b/flow/connectors/core.go index 9a9a42f1a3..b6f75c4828 100644 --- a/flow/connectors/core.go +++ b/flow/connectors/core.go @@ -22,6 +22,7 @@ import ( "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/otel_metrics" ) var ErrUnsupportedFunctionality = errors.New("requested connector does not support functionality") @@ -73,7 +74,9 @@ type CDCPullConnectorCore interface { PullFlowCleanup(ctx context.Context, jobName string) error // HandleSlotInfo update monitoring info on slot size etc - HandleSlotInfo(ctx context.Context, alerter *alerting.Alerter, catalogPool *pgxpool.Pool, slotName string, peerName string) error + HandleSlotInfo(ctx context.Context, alerter *alerting.Alerter, + catalogPool *pgxpool.Pool, slotName string, peerName string, + slotLagGauge *otel_metrics.Float64Gauge, openConnectionsGauge *otel_metrics.Int64Gauge) error // GetSlotInfo returns the WAL (or equivalent) info of a slot for the connector. GetSlotInfo(ctx context.Context, slotName string) ([]*protos.SlotInfo, error) diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index 1f5a7545f4..03ee0c02b1 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -15,6 +15,7 @@ import ( "github.com/jackc/pgx/v5" "github.com/jackc/pgx/v5/pgtype" "github.com/jackc/pgx/v5/pgxpool" + "go.opentelemetry.io/otel/attribute" "go.temporal.io/sdk/activity" "go.temporal.io/sdk/log" "go.temporal.io/sdk/temporal" @@ -26,6 +27,8 @@ import ( "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/otel_metrics" + "github.com/PeerDB-io/peer-flow/peerdbenv" "github.com/PeerDB-io/peer-flow/shared" ) @@ -1109,6 +1112,8 @@ func (c *PostgresConnector) HandleSlotInfo( catalogPool *pgxpool.Pool, slotName string, peerName string, + slotLagGauge *otel_metrics.Float64Gauge, + openConnectionsGauge *otel_metrics.Int64Gauge, ) error { logger := logger.LoggerFromCtx(ctx) @@ -1125,6 +1130,10 @@ func (c *PostgresConnector) HandleSlotInfo( logger.Info(fmt.Sprintf("Checking %s lag for %s", slotName, peerName), slog.Float64("LagInMB", float64(slotInfo[0].LagInMb))) alerter.AlertIfSlotLag(ctx, peerName, slotInfo[0]) + slotLagGauge.Set(float64(slotInfo[0].LagInMb), attribute.NewSet( + attribute.String("peerName", peerName), + attribute.String("slotName", slotName), + attribute.String("deploymentUID", peerdbenv.PeerDBDeploymentUID()))) // Also handles alerts for PeerDB user connections exceeding a given limit here res, err := getOpenConnectionsForUser(ctx, c.conn, c.config.User) @@ -1133,6 +1142,9 @@ func (c *PostgresConnector) HandleSlotInfo( return err } alerter.AlertIfOpenConnections(ctx, peerName, res) + openConnectionsGauge.Set(res.CurrentOpenConnections, attribute.NewSet( + attribute.String("peerName", peerName), + attribute.String("deploymentUID", peerdbenv.PeerDBDeploymentUID()))) return monitoring.AppendSlotSizeInfo(ctx, catalogPool, peerName, slotInfo[0]) } diff --git a/flow/go.mod b/flow/go.mod index 71ab6c6557..af7033eae9 100644 --- a/flow/go.mod +++ b/flow/go.mod @@ -50,6 +50,11 @@ require ( github.com/urfave/cli/v3 v3.0.0-alpha9 github.com/youmark/pkcs8 v0.0.0-20240424034433-3c2c7870ae76 github.com/yuin/gopher-lua v1.1.1 + go.opentelemetry.io/otel v1.26.0 + go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetrichttp v1.26.0 + go.opentelemetry.io/otel/metric v1.26.0 + go.opentelemetry.io/otel/sdk v1.26.0 + go.opentelemetry.io/otel/sdk/metric v1.26.0 go.temporal.io/api v1.32.0 go.temporal.io/sdk v1.26.1 go.uber.org/automaxprocs v1.5.3 @@ -77,6 +82,7 @@ require ( github.com/aws/aws-sdk-go-v2/service/ssooidc v1.23.4 // indirect github.com/aws/aws-sdk-go-v2/service/sts v1.28.6 // indirect github.com/beorn7/perks v1.0.1 // indirect + github.com/cenkalti/backoff/v4 v4.3.0 // indirect github.com/cespare/xxhash/v2 v2.3.0 // indirect github.com/cockroachdb/errors v1.11.1 // indirect github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b // indirect @@ -110,9 +116,8 @@ require ( github.com/twmb/franz-go/pkg/kmsg v1.7.0 // indirect go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.51.0 // indirect go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.51.0 // indirect - go.opentelemetry.io/otel v1.26.0 // indirect - go.opentelemetry.io/otel/metric v1.26.0 // indirect go.opentelemetry.io/otel/trace v1.26.0 // indirect + go.opentelemetry.io/proto/otlp v1.2.0 // indirect golang.org/x/term v0.19.0 // indirect ) diff --git a/flow/go.sum b/flow/go.sum index 5857b2424b..150c58bafe 100644 --- a/flow/go.sum +++ b/flow/go.sum @@ -127,6 +127,8 @@ github.com/aws/smithy-go v1.20.2/go.mod h1:krry+ya/rV9RDcV/Q16kpu6ypI4K2czasz0NC github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= +github.com/cenkalti/backoff/v4 v4.3.0 h1:MyRJ/UdXutAwSAT+s3wNd7MfTIcy71VQueUuFK343L8= +github.com/cenkalti/backoff/v4 v4.3.0/go.mod h1:Y3VNntkOUPxTVeUxJ/G5vcM//AlwfmyYozVcomhLiZE= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs= github.com/cespare/xxhash/v2 v2.3.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= @@ -427,12 +429,18 @@ go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.51.0 h1:Xs2Ncz0 go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.51.0/go.mod h1:vy+2G/6NvVMpwGX/NyLqcC41fxepnuKHk16E6IZUcJc= go.opentelemetry.io/otel v1.26.0 h1:LQwgL5s/1W7YiiRwxf03QGnWLb2HW4pLiAhaA5cZXBs= go.opentelemetry.io/otel v1.26.0/go.mod h1:UmLkJHUAidDval2EICqBMbnAd0/m2vmpf/dAM+fvFs4= +go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetrichttp v1.26.0 h1:HGZWGmCVRCVyAs2GQaiHQPbDHo+ObFWeUEOd+zDnp64= +go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetrichttp v1.26.0/go.mod h1:SaH+v38LSCHddyk7RGlU9uZyQoRrKao6IBnJw6Kbn+c= go.opentelemetry.io/otel/metric v1.26.0 h1:7S39CLuY5Jgg9CrnA9HHiEjGMF/X2VHvoXGgSllRz30= go.opentelemetry.io/otel/metric v1.26.0/go.mod h1:SY+rHOI4cEawI9a7N1A4nIg/nTQXe1ccCNWYOJUrpX4= -go.opentelemetry.io/otel/sdk v1.24.0 h1:YMPPDNymmQN3ZgczicBY3B6sf9n62Dlj9pWD3ucgoDw= -go.opentelemetry.io/otel/sdk v1.24.0/go.mod h1:KVrIYw6tEubO9E96HQpcmpTKDVn9gdv35HoYiQWGDFg= +go.opentelemetry.io/otel/sdk v1.26.0 h1:Y7bumHf5tAiDlRYFmGqetNcLaVUZmh4iYfmGxtmz7F8= +go.opentelemetry.io/otel/sdk v1.26.0/go.mod h1:0p8MXpqLeJ0pzcszQQN4F0S5FVjBLgypeGSngLsmirs= +go.opentelemetry.io/otel/sdk/metric v1.26.0 h1:cWSks5tfriHPdWFnl+qpX3P681aAYqlZHcAyHw5aU9Y= +go.opentelemetry.io/otel/sdk/metric v1.26.0/go.mod h1:ClMFFknnThJCksebJwz7KIyEDHO+nTB6gK8obLy8RyE= go.opentelemetry.io/otel/trace v1.26.0 h1:1ieeAUb4y0TE26jUFrCIXKpTuVK7uJGN9/Z/2LP5sQA= go.opentelemetry.io/otel/trace v1.26.0/go.mod h1:4iDxvGDQuUkHve82hJJ8UqrwswHYsZuWCBllGV2U2y0= +go.opentelemetry.io/proto/otlp v1.2.0 h1:pVeZGk7nXDC9O2hncA6nHldxEjm6LByfA2aN8IOkz94= +go.opentelemetry.io/proto/otlp v1.2.0/go.mod h1:gGpR8txAl5M03pDhMC79G6SdqNV26naRm/KDsgaHD8A= 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= diff --git a/flow/main.go b/flow/main.go index 06843b9254..3b1696161b 100644 --- a/flow/main.go +++ b/flow/main.go @@ -47,6 +47,12 @@ func main() { Usage: "Enable profiling for the application", Sources: cli.EnvVars("ENABLE_PROFILING"), } + otelMetricsFlag := &cli.BoolFlag{ + Name: "enable-otel-metrics", + Value: false, // Default is off + Usage: "Enable OpenTelemetry metrics for the application", + Sources: cli.EnvVars("ENABLE_OTEL_METRICS"), + } pyroscopeServerFlag := &cli.StringFlag{ Name: "pyroscope-server-address", @@ -83,9 +89,10 @@ func main() { Name: "worker", Action: func(ctx context.Context, clicmd *cli.Command) error { temporalHostPort := clicmd.String("temporal-host-port") - c, w, err := cmd.WorkerMain(&cmd.WorkerOptions{ + res, err := cmd.WorkerSetup(&cmd.WorkerSetupOptions{ TemporalHostPort: temporalHostPort, EnableProfiling: clicmd.Bool("enable-profiling"), + EnableOtelMetrics: clicmd.Bool("enable-otel-metrics"), PyroscopeServer: clicmd.String("pyroscope-server-address"), TemporalNamespace: clicmd.String("temporal-namespace"), TemporalCert: clicmd.String("temporal-cert"), @@ -96,12 +103,13 @@ func main() { if err != nil { return err } - defer c.Close() - return w.Run(worker.InterruptCh()) + defer res.Cleanup() + return res.Worker.Run(worker.InterruptCh()) }, Flags: []cli.Flag{ temporalHostPortFlag, profilingFlag, + otelMetricsFlag, pyroscopeServerFlag, temporalNamespaceFlag, &temporalCertFlag, diff --git a/flow/otel_metrics/otel_manager.go b/flow/otel_metrics/otel_manager.go new file mode 100644 index 0000000000..ceb7a511db --- /dev/null +++ b/flow/otel_metrics/otel_manager.go @@ -0,0 +1,54 @@ +package otel_metrics + +import ( + "context" + "fmt" + "time" + + "go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetrichttp" + "go.opentelemetry.io/otel/metric" + sdkmetric "go.opentelemetry.io/otel/sdk/metric" + "go.opentelemetry.io/otel/sdk/resource" + semconv "go.opentelemetry.io/otel/semconv/v1.24.0" +) + +type OtelManager struct { + MetricsProvider *sdkmetric.MeterProvider + Meter metric.Meter + Float64GaugesCache map[string]*Float64Gauge + Int64GaugesCache map[string]*Int64Gauge +} + +// newOtelResource returns a resource describing this application. +func newOtelResource(otelServiceName string) (*resource.Resource, error) { + r, err := resource.Merge( + resource.Default(), + resource.NewWithAttributes( + semconv.SchemaURL, + semconv.ServiceNameKey.String(otelServiceName), + ), + ) + + return r, err +} + +func SetupOtelMetricsExporter(otelServiceName string) (*sdkmetric.MeterProvider, error) { + metricExporter, err := otlpmetrichttp.New(context.Background(), + otlpmetrichttp.WithCompression(otlpmetrichttp.GzipCompression), + ) + if err != nil { + return nil, fmt.Errorf("failed to create OpenTelemetry metrics exporter: %w", err) + } + + resource, err := newOtelResource(otelServiceName) + if err != nil { + return nil, fmt.Errorf("failed to create OpenTelemetry resource: %w", err) + } + + meterProvider := sdkmetric.NewMeterProvider( + sdkmetric.WithReader(sdkmetric.NewPeriodicReader(metricExporter, + sdkmetric.WithInterval(3*time.Second))), + sdkmetric.WithResource(resource), + ) + return meterProvider, nil +} diff --git a/flow/otel_metrics/sync_gauges.go b/flow/otel_metrics/sync_gauges.go new file mode 100644 index 0000000000..14a7058301 --- /dev/null +++ b/flow/otel_metrics/sync_gauges.go @@ -0,0 +1,114 @@ +package otel_metrics + +import ( + "context" + "fmt" + "math" + "sync/atomic" + + "go.opentelemetry.io/otel/attribute" + "go.opentelemetry.io/otel/metric" +) + +// synchronous gauges are what we want, so we can control when the value is updated +// but they are "experimental", so we resort to using the asynchronous gauges +// but the callback is just a wrapper around the current value, so we can control by calling Set() +type Int64Gauge struct { + observableGauge metric.Int64ObservableGauge + observations map[attribute.Set]*atomic.Int64 +} + +func NewInt64SyncGauge(meter metric.Meter, gaugeName string, opts ...metric.Int64ObservableGaugeOption) (*Int64Gauge, error) { + syncGauge := &Int64Gauge{} + observableGauge, err := meter.Int64ObservableGauge(gaugeName, append(opts, metric.WithInt64Callback(syncGauge.callback))...) + if err != nil { + return nil, fmt.Errorf("failed to create Int64SyncGauge: %w", err) + } + syncGauge.observableGauge = observableGauge + syncGauge.observations = make(map[attribute.Set]*atomic.Int64) + return syncGauge, nil +} + +func (g *Int64Gauge) callback(ctx context.Context, o metric.Int64Observer) error { + for attrs, val := range g.observations { + o.Observe(val.Load(), metric.WithAttributeSet(attrs)) + } + return nil +} + +func (g *Int64Gauge) Set(input int64, attrs attribute.Set) { + if g == nil { + return + } + val, ok := g.observations[attrs] + if !ok { + val = &atomic.Int64{} + g.observations[attrs] = val + } + val.Store(input) +} + +type Float64Gauge struct { + observableGauge metric.Float64ObservableGauge + observationsAsUint64 map[attribute.Set]*atomic.Uint64 +} + +func NewFloat64SyncGauge(meter metric.Meter, gaugeName string, opts ...metric.Float64ObservableGaugeOption) (*Float64Gauge, error) { + syncGauge := &Float64Gauge{} + observableGauge, err := meter.Float64ObservableGauge(gaugeName, append(opts, metric.WithFloat64Callback(syncGauge.callback))...) + if err != nil { + return nil, fmt.Errorf("failed to create Int64SyncGauge: %w", err) + } + syncGauge.observableGauge = observableGauge + syncGauge.observationsAsUint64 = make(map[attribute.Set]*atomic.Uint64) + return syncGauge, nil +} + +func (g *Float64Gauge) callback(ctx context.Context, o metric.Float64Observer) error { + for attrs, val := range g.observationsAsUint64 { + o.Observe(math.Float64frombits(val.Load()), metric.WithAttributeSet(attrs)) + } + return nil +} + +func (g *Float64Gauge) Set(input float64, attrs attribute.Set) { + if g == nil { + return + } + val, ok := g.observationsAsUint64[attrs] + if !ok { + val = &atomic.Uint64{} + g.observationsAsUint64[attrs] = val + } + val.Store(math.Float64bits(input)) +} + +func GetOrInitInt64Gauge(meter metric.Meter, cache map[string]*Int64Gauge, + name string, opts ...metric.Int64ObservableGaugeOption, +) (*Int64Gauge, error) { + gauge, ok := cache[name] + if !ok { + var err error + gauge, err = NewInt64SyncGauge(meter, name, opts...) + if err != nil { + return nil, err + } + cache[name] = gauge + } + return gauge, nil +} + +func GetOrInitFloat64Gauge(meter metric.Meter, cache map[string]*Float64Gauge, + name string, opts ...metric.Float64ObservableGaugeOption, +) (*Float64Gauge, error) { + gauge, ok := cache[name] + if !ok { + var err error + gauge, err = NewFloat64SyncGauge(meter, name, opts...) + if err != nil { + return nil, err + } + cache[name] = gauge + } + return gauge, nil +} From 9a857f743124cd258ea9a807ae5b437fd5082d90 Mon Sep 17 00:00:00 2001 From: Kevin Biju <52661649+heavycrystal@users.noreply.github.com> Date: Mon, 6 May 2024 22:57:34 +0530 Subject: [PATCH 21/41] swapping out custom error for errors.ErrUnsupported (#1674) turns out reading release notes once in a while helps --- flow/activities/flowable.go | 10 +++++----- flow/connectors/core.go | 6 ++---- 2 files changed, 7 insertions(+), 9 deletions(-) diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index 9c03346734..2291400b03 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -154,7 +154,7 @@ func (a *FlowableActivity) CreateNormalizedTable( ctx = context.WithValue(ctx, shared.FlowNameKey, config.FlowName) conn, err := connectors.GetAs[connectors.NormalizedTablesConnector](ctx, config.PeerConnectionConfig) if err != nil { - if err == connectors.ErrUnsupportedFunctionality { + if errors.Is(err, errors.ErrUnsupported) { logger.Info("Connector does not implement normalized tables") return nil, nil } @@ -298,7 +298,7 @@ func (a *FlowableActivity) StartNormalize( logger := activity.GetLogger(ctx) dstConn, err := connectors.GetCDCNormalizeConnector(ctx, conn.Destination) - if errors.Is(err, connectors.ErrUnsupportedFunctionality) { + if errors.Is(err, errors.ErrUnsupported) { err = monitoring.UpdateEndTimeForCDCBatch(ctx, a.CatalogPool, input.FlowConnectionConfigs.FlowJobName, input.SyncBatchID) return nil, err @@ -441,7 +441,7 @@ func (a *FlowableActivity) ConsolidateQRepPartitions(ctx context.Context, config ) error { ctx = context.WithValue(ctx, shared.FlowNameKey, config.FlowJobName) dstConn, err := connectors.GetQRepConsolidateConnector(ctx, config.DestinationPeer) - if errors.Is(err, connectors.ErrUnsupportedFunctionality) { + if errors.Is(err, errors.ErrUnsupported) { return monitoring.UpdateEndTimeForQRepRun(ctx, a.CatalogPool, runUUID) } else if err != nil { return err @@ -465,7 +465,7 @@ func (a *FlowableActivity) ConsolidateQRepPartitions(ctx context.Context, config func (a *FlowableActivity) CleanupQRepFlow(ctx context.Context, config *protos.QRepConfig) error { ctx = context.WithValue(ctx, shared.FlowNameKey, config.FlowJobName) dst, err := connectors.GetQRepConsolidateConnector(ctx, config.DestinationPeer) - if errors.Is(err, connectors.ErrUnsupportedFunctionality) { + if errors.Is(err, errors.ErrUnsupported) { return nil } else if err != nil { a.Alerter.LogFlowError(ctx, config.FlowJobName, err) @@ -578,7 +578,7 @@ func (a *FlowableActivity) RecordSlotSizes(ctx context.Context) error { func() { srcConn, err := connectors.GetCDCPullConnector(ctx, config.Source) if err != nil { - if err != connectors.ErrUnsupportedFunctionality { + if !errors.Is(err, errors.ErrUnsupported) { logger.Error("Failed to create connector to handle slot info", slog.Any("error", err)) } return diff --git a/flow/connectors/core.go b/flow/connectors/core.go index b6f75c4828..dc7afe0495 100644 --- a/flow/connectors/core.go +++ b/flow/connectors/core.go @@ -25,8 +25,6 @@ import ( "github.com/PeerDB-io/peer-flow/otel_metrics" ) -var ErrUnsupportedFunctionality = errors.New("requested connector does not support functionality") - type Connector interface { Close() error ConnectionActive(context.Context) error @@ -245,7 +243,7 @@ func GetConnector(ctx context.Context, config *protos.Peer) (Connector, error) { case *protos.Peer_ElasticsearchConfig: return connelasticsearch.NewElasticsearchConnector(ctx, inner.ElasticsearchConfig) default: - return nil, ErrUnsupportedFunctionality + return nil, errors.ErrUnsupported } } @@ -259,7 +257,7 @@ func GetAs[T Connector](ctx context.Context, config *protos.Peer) (T, error) { if conn, ok := conn.(T); ok { return conn, nil } else { - return none, ErrUnsupportedFunctionality + return none, errors.ErrUnsupported } } From cad7eb194a6f4ab0e34d5f5add34d766377c9f06 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Mon, 6 May 2024 23:58:40 +0530 Subject: [PATCH 22/41] PeerDB: For queues, move sync interval to advanced, default 10 mins (#1672) For queue destination peers: Moves sync interval field in create cdc mirror UI to advanced settings and defaults to 10 minutes. Functionally tested --- ui/app/mirrors/create/cdc/cdc.tsx | 46 ++++++++++++++++++------- ui/app/mirrors/create/helpers/cdc.ts | 23 +++++++------ ui/app/mirrors/create/helpers/common.ts | 8 ++++- 3 files changed, 52 insertions(+), 25 deletions(-) diff --git a/ui/app/mirrors/create/cdc/cdc.tsx b/ui/app/mirrors/create/cdc/cdc.tsx index b4581b9cf0..2ef76be5be 100644 --- a/ui/app/mirrors/create/cdc/cdc.tsx +++ b/ui/app/mirrors/create/cdc/cdc.tsx @@ -5,7 +5,7 @@ import { Icon } from '@/lib/Icon'; import { Dispatch, SetStateAction, useEffect, useMemo, useState } from 'react'; import { CDCConfig, MirrorSetter, TableMapRow } from '../../../dto/MirrorsDTO'; import { IsQueuePeer, fetchPublications } from '../handlers'; -import { MirrorSetting } from '../helpers/common'; +import { AdvancedSettingType, MirrorSetting } from '../helpers/common'; import CDCField from './fields'; import TableMapping from './tablemapping'; @@ -46,12 +46,32 @@ export default function CDCConfigForm({ }; const normalSettings = useMemo(() => { - return settings.filter((setting) => setting.advanced != true); - }, [settings]); + return settings!.filter( + (setting) => + !( + (IsQueuePeer(mirrorConfig.destination?.type) && + setting.advanced === AdvancedSettingType.QUEUE) || + setting.advanced === AdvancedSettingType.ALL + ) + ); + }, [settings, mirrorConfig.destination?.type]); const advancedSettings = useMemo(() => { - return settings.filter((setting) => setting.advanced == true); - }, [settings]); + return settings! + .map((setting) => { + if ( + IsQueuePeer(mirrorConfig.destination?.type) && + setting.advanced === AdvancedSettingType.QUEUE + ) { + setting.stateHandler(600, setter); + return { ...setting, default: 600 }; + } + if (setting.advanced === AdvancedSettingType.ALL) { + return setting; + } + }) + .filter((setting) => setting !== undefined); + }, [settings, mirrorConfig.destination?.type, setter]); const paramDisplayCondition = (setting: MirrorSetting) => { const label = setting.label.toLowerCase(); @@ -91,15 +111,15 @@ export default function CDCConfigForm({ if (mirrorConfig.source != undefined && mirrorConfig.destination != undefined) return ( <> - {normalSettings.map((setting, id) => { + {normalSettings!.map((setting, id) => { return ( - paramDisplayCondition(setting) && ( + paramDisplayCondition(setting!) && ( {show && - advancedSettings.map((setting, id) => { + advancedSettings!.map((setting, id) => { return ( - paramDisplayCondition(setting) && ( + paramDisplayCondition(setting!) && ( ) ); diff --git a/ui/app/mirrors/create/helpers/cdc.ts b/ui/app/mirrors/create/helpers/cdc.ts index 1d190f78a7..a480840047 100644 --- a/ui/app/mirrors/create/helpers/cdc.ts +++ b/ui/app/mirrors/create/helpers/cdc.ts @@ -1,6 +1,6 @@ import { TypeSystem } from '@/grpc_generated/flow'; import { CDCConfig } from '../../../dto/MirrorsDTO'; -import { MirrorSetting } from './common'; +import { AdvancedSettingType, MirrorSetting } from './common'; export const cdcSettings: MirrorSetting[] = [ { label: 'Initial Copy', @@ -24,7 +24,7 @@ export const cdcSettings: MirrorSetting[] = [ tips: 'The number of rows PeerDB will pull from source at a time. If left empty, the default value is 1,000,000 rows.', type: 'number', default: '1000000', - advanced: true, + advanced: AdvancedSettingType.ALL, }, { label: 'Sync Interval (Seconds)', @@ -38,6 +38,7 @@ export const cdcSettings: MirrorSetting[] = [ type: 'number', default: '60', required: true, + advanced: AdvancedSettingType.QUEUE, }, { label: 'Publication Name', @@ -71,7 +72,7 @@ export const cdcSettings: MirrorSetting[] = [ tips: 'PeerDB splits up table data into partitions for increased performance. This setting controls the number of rows per partition. The default value is 1000000.', default: '1000000', type: 'number', - advanced: true, + advanced: AdvancedSettingType.ALL, }, { label: 'Parallelism for Initial Load', @@ -95,7 +96,7 @@ export const cdcSettings: MirrorSetting[] = [ tips: 'Specify the number of tables to sync perform initial load for, in parallel. The default value is 1.', default: '1', type: 'number', - advanced: true, + advanced: AdvancedSettingType.ALL, }, { label: 'Snapshot Staging Path', @@ -105,7 +106,7 @@ export const cdcSettings: MirrorSetting[] = [ snapshotStagingPath: value as string | '', })), tips: 'You can specify staging path for Snapshot sync mode AVRO. For Snowflake as destination peer, this must be either empty or an S3 bucket URL. For BigQuery, this must be either empty or an existing GCS bucket name. In both cases, if empty, the local filesystem will be used.', - advanced: true, + advanced: AdvancedSettingType.ALL, }, { label: 'CDC Staging Path', @@ -115,7 +116,7 @@ export const cdcSettings: MirrorSetting[] = [ cdcStagingPath: (value as string) || '', })), tips: 'You can specify staging path for CDC sync mode AVRO. For Snowflake as destination peer, this must be either empty or an S3 bucket URL. For BigQuery, this must be either empty or an existing GCS bucket name. In both cases, if empty, the local filesystem will be used.', - advanced: true, + advanced: AdvancedSettingType.ALL, }, { label: 'Soft Delete', @@ -138,7 +139,7 @@ export const cdcSettings: MirrorSetting[] = [ })), tips: 'If set, PeerDB will only perform initial load and will not perform CDC sync.', type: 'switch', - advanced: true, + advanced: AdvancedSettingType.ALL, }, { label: 'Script', @@ -148,7 +149,7 @@ export const cdcSettings: MirrorSetting[] = [ script: (value as string) || '', })), tips: 'Associate PeerDB script with this mirror.', - advanced: true, + advanced: AdvancedSettingType.ALL, }, { label: 'Use Postgres type system', @@ -160,7 +161,7 @@ export const cdcSettings: MirrorSetting[] = [ type: 'switch', default: false, tips: 'Decide if PeerDB should use native Postgres types directly', - advanced: true, + advanced: AdvancedSettingType.ALL, }, { label: 'Synced-At Column Name', @@ -170,7 +171,7 @@ export const cdcSettings: MirrorSetting[] = [ syncedAtColName: value as string | '', })), tips: 'A field to set the name of PeerDBs synced_at column. If not set, a default name will be set', - advanced: true, + advanced: AdvancedSettingType.ALL, }, { label: 'Soft Delete Column Name', @@ -180,6 +181,6 @@ export const cdcSettings: MirrorSetting[] = [ softDeleteColName: value as string | '', })), tips: 'A field to set the name of PeerDBs soft delete column.', - advanced: true, + advanced: AdvancedSettingType.ALL, }, ]; diff --git a/ui/app/mirrors/create/helpers/common.ts b/ui/app/mirrors/create/helpers/common.ts index 4237bc2176..c63278a4a9 100644 --- a/ui/app/mirrors/create/helpers/common.ts +++ b/ui/app/mirrors/create/helpers/common.ts @@ -1,5 +1,11 @@ import { FlowConnectionConfigs, TypeSystem } from '@/grpc_generated/flow'; +export enum AdvancedSettingType { + QUEUE = 'queue', + ALL = 'all', + NONE = 'none', +} + export interface MirrorSetting { label: string; stateHandler: (value: any, setter: any) => void; @@ -8,7 +14,7 @@ export interface MirrorSetting { tips?: string; helpfulLink?: string; default?: string | number | boolean; - advanced?: boolean; // whether it should come under an 'Advanced' section + advanced?: AdvancedSettingType; // whether it should come under an 'Advanced' section command?: string; } From fcd15de6353266a7c670e5d676096d7b4a9a2efc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Mon, 6 May 2024 19:07:42 +0000 Subject: [PATCH 23/41] chore: update dependencies (#1673) --- flow/go.mod | 22 ++++----- flow/go.sum | 46 +++++++++---------- nexus/Cargo.lock | 106 +++++++++++++++++++++++-------------------- ui/package-lock.json | 70 ++++++++++++++-------------- ui/package.json | 6 +-- 5 files changed, 127 insertions(+), 123 deletions(-) diff --git a/flow/go.mod b/flow/go.mod index af7033eae9..d6d94975d4 100644 --- a/flow/go.mod +++ b/flow/go.mod @@ -37,7 +37,7 @@ require ( github.com/joho/godotenv v1.5.1 github.com/klauspost/compress v1.17.8 github.com/lib/pq v1.10.9 - github.com/linkedin/goavro/v2 v2.12.0 + github.com/linkedin/goavro/v2 v2.13.0 github.com/microsoft/go-mssqldb v1.7.1 github.com/orcaman/concurrent-map/v2 v2.0.1 github.com/shopspring/decimal v1.4.0 @@ -58,13 +58,13 @@ require ( 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/crypto v0.23.0 golang.org/x/mod v0.17.0 golang.org/x/sync v0.7.0 google.golang.org/api v0.177.0 google.golang.org/genproto/googleapis/api v0.0.0-20240429193739-8cf5692501f6 google.golang.org/grpc v1.63.2 - google.golang.org/protobuf v1.34.0 + google.golang.org/protobuf v1.34.1 ) require ( @@ -118,14 +118,14 @@ require ( go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.51.0 // indirect go.opentelemetry.io/otel/trace v1.26.0 // indirect go.opentelemetry.io/proto/otlp v1.2.0 // indirect - golang.org/x/term v0.19.0 // indirect + golang.org/x/term v0.20.0 // indirect ) require ( cloud.google.com/go/compute/metadata v0.3.0 // indirect - cloud.google.com/go/iam v1.1.7 // indirect + cloud.google.com/go/iam v1.1.8 // indirect github.com/Azure/azure-sdk-for-go/sdk/azcore v1.11.1 // indirect - github.com/Azure/azure-sdk-for-go/sdk/internal v1.6.0 // indirect + github.com/Azure/azure-sdk-for-go/sdk/internal v1.7.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 @@ -155,7 +155,7 @@ require ( github.com/google/flatbuffers v24.3.25+incompatible // indirect github.com/google/s2a-go v0.1.7 // indirect github.com/googleapis/enterprise-certificate-proxy v0.3.2 // indirect - github.com/googleapis/gax-go/v2 v2.12.3 // indirect + github.com/googleapis/gax-go/v2 v2.12.4 // indirect github.com/grafana/pyroscope-go/godeltaprof v0.1.7 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/jackc/pgio v1.0.0 // indirect @@ -175,10 +175,10 @@ require ( github.com/zeebo/xxh3 v1.0.2 // indirect go.opencensus.io v0.24.0 // indirect golang.org/x/exp v0.0.0-20240416160154-fe59bbe5cc7f - golang.org/x/net v0.24.0 // indirect - golang.org/x/oauth2 v0.19.0 // indirect - golang.org/x/sys v0.19.0 // indirect - golang.org/x/text v0.14.0 // indirect + golang.org/x/net v0.25.0 // indirect + golang.org/x/oauth2 v0.20.0 // indirect + golang.org/x/sys v0.20.0 // indirect + golang.org/x/text v0.15.0 // indirect 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 diff --git a/flow/go.sum b/flow/go.sum index 150c58bafe..6e1813dc05 100644 --- a/flow/go.sum +++ b/flow/go.sum @@ -11,8 +11,8 @@ cloud.google.com/go/compute/metadata v0.3.0 h1:Tz+eQXMEqDIKRsmY3cHTL6FVaynIjX2Qx 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= -cloud.google.com/go/iam v1.1.7/go.mod h1:J4PMPg8TtyurAUvSmPj8FF3EDgY1SPRZxcUGrn7WXGA= +cloud.google.com/go/iam v1.1.8 h1:r7umDwhj+BQyz0ScZMp4QrGXjSTI3ZINnpgU2nlB/K0= +cloud.google.com/go/iam v1.1.8/go.mod h1:GvE6lyMmfxXauzNq8NbgJbeVQNspG+tcdL/W8QO1+zE= cloud.google.com/go/kms v1.15.8 h1:szIeDCowID8th2i8XE4uRev5PMxQFqW+JjwYxL9h6xs= cloud.google.com/go/kms v1.15.8/go.mod h1:WoUHcDjD9pluCg7pNds131awnH429QGvRM3N/4MyoVs= cloud.google.com/go/longrunning v0.5.6 h1:xAe8+0YaWoCKr9t1+aWe+OeQgN/iJK1fEgZSXmjuEaE= @@ -31,8 +31,8 @@ github.com/Azure/azure-sdk-for-go/sdk/azcore v1.11.1 h1:E+OJmp2tPvt1W+amx48v1eqb 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.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/internal v1.7.0 h1:rTfKOCZGy5ViVrlA74ZPE99a+SgoEE2K/yg3RyW9dFA= +github.com/Azure/azure-sdk-for-go/sdk/internal v1.7.0/go.mod h1:4OG6tQ9EOP/MT0NMjDlRzWoVFxfu9rN9B2X+tlSVktg= 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= @@ -157,8 +157,6 @@ github.com/djherbis/buffer v1.2.0 h1:PH5Dd2ss0C7CRRhQCZ2u7MssF+No9ide8Ye71nPHcrQ github.com/djherbis/buffer v1.2.0/go.mod h1:fjnebbZjCUpPinBRD+TDwXSOeNQ7fPQWLfGQqiAiUyE= 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.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= @@ -257,8 +255,8 @@ github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/enterprise-certificate-proxy v0.3.2 h1:Vie5ybvEvT75RniqhfFxPRy3Bf7vr3h0cechB90XaQs= github.com/googleapis/enterprise-certificate-proxy v0.3.2/go.mod h1:VLSiSSBs/ksPL8kq3OBOQ6WRI2QnaFynd1DCjZ62+V0= -github.com/googleapis/gax-go/v2 v2.12.3 h1:5/zPPDvw8Q1SuXjrqrZslrqT7dL/uJT2CQii/cLCKqA= -github.com/googleapis/gax-go/v2 v2.12.3/go.mod h1:AKloxT6GtNbaLm8QTNSidHUVsHYcBHwWRvkNFJUQcS4= +github.com/googleapis/gax-go/v2 v2.12.4 h1:9gWcmF85Wvq4ryPFvGFaOgPIs1AQX0d0bcbGw4Z96qg= +github.com/googleapis/gax-go/v2 v2.12.4/go.mod h1:KYEYLorsnIGDi/rPC8b5TdlB9kbKoFubselGIoBMCwI= github.com/gorilla/websocket v1.4.2/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= github.com/gorilla/websocket v1.5.1 h1:gmztn0JnHVt9JZquRuzLw3g4wouNVzKL15iLr/zn/QY= github.com/gorilla/websocket v1.5.1/go.mod h1:x3kM2JMyaluk02fnUJpQuwD2dCS5NDG2ZHL0uE0tcaY= @@ -316,8 +314,8 @@ github.com/kylelemons/godebug v1.1.0 h1:RPNrshWIDI6G2gRW9EHilWtl7Z6Sb1BR0xunSBf0 github.com/kylelemons/godebug v1.1.0/go.mod h1:9/0rRGxNHcop5bhtWyNeEfOS8JIWk580+fNqagV/RAw= github.com/lib/pq v1.10.9 h1:YXG7RB+JIjhP29X+OtkiDnYaXQwpS4JEWq7dtCCRUEw= github.com/lib/pq v1.10.9/go.mod h1:AlVN5x4E4T544tWzH6hKfbfQvm3HdbOxrmggDNAPY9o= -github.com/linkedin/goavro/v2 v2.12.0 h1:rIQQSj8jdAUlKQh6DttK8wCRv4t4QO09g1C4aBWXslg= -github.com/linkedin/goavro/v2 v2.12.0/go.mod h1:KXx+erlq+RPlGSPmLF7xGo6SAbh8sCQ53x064+ioxhk= +github.com/linkedin/goavro/v2 v2.13.0 h1:L8eI8GcuciwUkt41Ej62joSZS4kKaYIUdze+6for9NU= +github.com/linkedin/goavro/v2 v2.13.0/go.mod h1:KXx+erlq+RPlGSPmLF7xGo6SAbh8sCQ53x064+ioxhk= github.com/mattn/go-sqlite3 v1.14.22 h1:2gZY6PC6kBnID23Tichd1K+Z0oS6nE/XwU+Vz/5o4kU= github.com/mattn/go-sqlite3 v1.14.22/go.mod h1:Uh1q+B4BYcTPb+yiD3kU8Ct7aC0hY9fxUwlHK0RXw+Y= github.com/microsoft/go-mssqldb v1.7.1 h1:KU/g8aWeM3Hx7IMOFpiwYiUkU+9zeISb4+tx3ScVfsM= @@ -455,8 +453,8 @@ golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACk golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= -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/crypto v0.23.0 h1:dIJU/v2J8Mdglj/8rJ6UUOM3Zc9zLZxVZwwxMooUSAI= +golang.org/x/crypto v0.23.0/go.mod h1:CKFgDieR+mRhux2Lsu27y0fO304Db0wZe70UKqHu0v8= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20240416160154-fe59bbe5cc7f h1:99ci1mjWVBWwJiEKYY6jWa4d2nTQVIEhZIptnrVb1XY= golang.org/x/exp v0.0.0-20240416160154-fe59bbe5cc7f/go.mod h1:/lliqkxwWAhPjf5oSOIJup2XcqJaw8RGS6k3TGEc7GI= @@ -480,11 +478,11 @@ golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwY golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.24.0 h1:1PcaxkF854Fu3+lvBIx5SYn9wRlBzzcnHZSiaFFAb0w= -golang.org/x/net v0.24.0/go.mod h1:2Q7sJY5mzlzWjKtYUEXSlBWCdyaioyXzRB2RtU8KVE8= +golang.org/x/net v0.25.0 h1:d/OCCoBEUq33pjydKrGQhw7IlUPI2Oylr+8qLx49kac= +golang.org/x/net v0.25.0/go.mod h1:JkAGAh7GEvH74S6FOH42FLoXpXbE/aqXSrIQjXgsiwM= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= -golang.org/x/oauth2 v0.19.0 h1:9+E/EZBCbTLNrbN35fHv/a/d/mOBatymz1zbtQrXpIg= -golang.org/x/oauth2 v0.19.0/go.mod h1:vYi7skDa1x015PmRRYZ7+s1cWyPgrPiSYRe4rnsexc8= +golang.org/x/oauth2 v0.20.0 h1:4mQdhULixXKP1rwYBW0vAijoXnkTG0BLCDRzfe1idMo= +golang.org/x/oauth2 v0.20.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -507,17 +505,17 @@ golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.1.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.19.0 h1:q5f1RH2jigJ1MoAWp2KTp3gm5zAGFUTarQZ5U386+4o= -golang.org/x/sys v0.19.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.20.0 h1:Od9JTbYCk261bKm4M/mw7AklTlFYIa0bIp9BgSm1S8Y= +golang.org/x/sys v0.20.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= -golang.org/x/term v0.19.0 h1:+ThwsDv+tYfnJFhF4L8jITxu1tdTWRTZpdsWgEgjL6Q= -golang.org/x/term v0.19.0/go.mod h1:2CuTdWZ7KHSQwUzKva0cbMg6q2DMI3Mmxp+gKJbskEk= +golang.org/x/term v0.20.0 h1:VnkxpohqXaOBYJtBmEppKUG6mXpi+4O6purfc2+sMhw= +golang.org/x/term v0.20.0/go.mod h1:8UkIAJTvZgivsXaD6/pH6U9ecQzZ45awqEOzuCvwpFY= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= -golang.org/x/text v0.14.0 h1:ScX5w1eTa3QqT8oi6+ziP7dTV1S2+ALU0bI+0zXKWiQ= -golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= +golang.org/x/text v0.15.0 h1:h1V/4gjBv8v9cjcR6+AR5+/cIYK5N/WAgiv4xlsEtAk= +golang.org/x/text v0.15.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= golang.org/x/time v0.5.0 h1:o7cqy6amK/52YcAKIPlM3a+Fpj35zvRj2TP+e1xFSfk= golang.org/x/time v0.5.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= @@ -573,8 +571,8 @@ google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpAD google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.34.0 h1:Qo/qEd2RZPCf2nKuorzksSknv0d3ERwp1vFG38gSmH4= -google.golang.org/protobuf v1.34.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= +google.golang.org/protobuf v1.34.1 h1:9ddQBjfCyZPOHPUiPxpYESBLc+T8P3E+Vo4IbKZgFWg= +google.golang.org/protobuf v1.34.1/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/nexus/Cargo.lock b/nexus/Cargo.lock index 400bdd24ca..d5985ed3b9 100644 --- a/nexus/Cargo.lock +++ b/nexus/Cargo.lock @@ -97,47 +97,48 @@ dependencies = [ [[package]] name = "anstream" -version = "0.6.13" +version = "0.6.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d96bd03f33fe50a863e394ee9718a706f988b9079b20c3784fb726e7678b62fb" +checksum = "418c75fa768af9c03be99d17643f93f79bbba589895012a80e3452a19ddda15b" dependencies = [ "anstyle", "anstyle-parse", "anstyle-query", "anstyle-wincon", "colorchoice", + "is_terminal_polyfill", "utf8parse", ] [[package]] name = "anstyle" -version = "1.0.6" +version = "1.0.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8901269c6307e8d93993578286ac0edf7f195079ffff5ebdeea6a59ffb7e36bc" +checksum = "038dfcf04a5feb68e9c60b21c9625a54c2c0616e79b72b0fd87075a056ae1d1b" [[package]] name = "anstyle-parse" -version = "0.2.3" +version = "0.2.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c75ac65da39e5fe5ab759307499ddad880d724eed2f6ce5b5e8a26f4f387928c" +checksum = "c03a11a9034d92058ceb6ee011ce58af4a9bf61491aa7e1e59ecd24bd40d22d4" dependencies = [ "utf8parse", ] [[package]] name = "anstyle-query" -version = "1.0.2" +version = "1.0.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e28923312444cdd728e4738b3f9c9cac739500909bb3d3c94b43551b16517648" +checksum = "a64c907d4e79225ac72e2a354c9ce84d50ebb4586dee56c82b3ee73004f537f5" dependencies = [ "windows-sys 0.52.0", ] [[package]] name = "anstyle-wincon" -version = "3.0.2" +version = "3.0.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1cd54b81ec8d6180e24654d0b371ad22fc3dd083b6ff8ba325b72e00c87660a7" +checksum = "61a38449feb7068f52bb06c12759005cf459ee52bb4adc1d5a7c4322d716fb19" dependencies = [ "anstyle", "windows-sys 0.52.0", @@ -226,9 +227,9 @@ dependencies = [ [[package]] name = "autocfg" -version = "1.2.0" +version = "1.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f1fdabc7756949593fe60f30ec81974b613357de856987752631dea1e3394c80" +checksum = "0c4b4d0bd25bd0b74681c0ad21497610ce1b7c91b1022cd21c80c6fbdd9476b0" [[package]] name = "axum" @@ -455,9 +456,9 @@ checksum = "514de17de45fdb8dc022b1a7975556c53c86f9f0aa5f534b98977b171857c2c9" [[package]] name = "cargo-deb" -version = "2.1.1" +version = "2.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "769d6bfa0f70c0e584f46d6bfc78d094f708fa249f2b7b524ab6bed62eb3df36" +checksum = "310298143a2181609b853b663ac6616e56fd78a92eb6efca04eed3dceae43600" dependencies = [ "ar", "cargo_toml", @@ -467,12 +468,12 @@ dependencies = [ "glob", "itertools 0.12.1", "log", - "md5", "quick-error", "rayon", "regex", "serde", "serde_json", + "sha2", "tar", "tempfile", "toml", @@ -521,9 +522,9 @@ dependencies = [ [[package]] name = "cc" -version = "1.0.96" +version = "1.0.97" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "065a29261d53ba54260972629f9ca6bffa69bac13cd1fed61420f7fa68b9f8bd" +checksum = "099a5357d84c4c61eb35fc8eafa9a79a902c2f76911e5747ced4e032edd8d9b4" dependencies = [ "jobserver", "libc", @@ -638,9 +639,9 @@ dependencies = [ [[package]] name = "colorchoice" -version = "1.0.0" +version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "acbf1af155f9b9ef647e42cdc158db4b64a1b61f743629225fde6f3e0be2a7c7" +checksum = "0b6a852b24ab71dffc585bcb46eaf7959d175cb865a7152e35b348d1b2960422" [[package]] name = "console-api" @@ -800,9 +801,9 @@ dependencies = [ [[package]] name = "deadpool" -version = "0.11.2" +version = "0.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ff0fc28638c21092aba483136debc6e177fff3dace8c835d715866923b03323e" +checksum = "28d3c3845002cd8dec1e045bc3fc076d1e467a123794cf56326d804489df1896" dependencies = [ "deadpool-runtime", "num_cpus", @@ -811,9 +812,9 @@ dependencies = [ [[package]] name = "deadpool-postgres" -version = "0.13.0" +version = "0.13.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4aa08f5c838496cbabb672e3614534444145fc6632995f102e13d30a29a25a13" +checksum = "10566d25f606cc5f2dee63311af0eb2dc861cba4a5a7dee2daa4c2b181a42862" dependencies = [ "deadpool", "tokio", @@ -823,9 +824,9 @@ dependencies = [ [[package]] name = "deadpool-runtime" -version = "0.1.3" +version = "0.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "63dfa964fe2a66f3fde91fc70b267fe193d822c7e603e2a675a49a7f46ad3f49" +checksum = "092966b41edc516079bdf31ec78a2e0588d1d0c08f78b91d8307215928642b2b" dependencies = [ "tokio", ] @@ -1150,9 +1151,9 @@ dependencies = [ [[package]] name = "getrandom" -version = "0.2.14" +version = "0.2.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "94b22e06ecb0110981051723910cbf0b5f5e09a2062dd7663334ee79a9d1286c" +checksum = "c4567c8db10ae91089c99af84c68c38da3ec2f087c3f82960bcdbf3656b6f4d7" dependencies = [ "cfg-if", "js-sys", @@ -1509,6 +1510,12 @@ version = "2.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8f518f335dce6725a761382244631d86cf0ccb2863413590b31338feb467f9c3" +[[package]] +name = "is_terminal_polyfill" +version = "1.70.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f8478577c03552c21db0e2724ffb8986a5ce7af88107e6be5d2ee6e158c12800" + [[package]] name = "itertools" version = "0.11.0" @@ -1867,9 +1874,9 @@ dependencies = [ [[package]] name = "num-iter" -version = "0.1.44" +version = "0.1.45" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d869c01cc0c455284163fd0092f1f93835385ccab5a98a0dcc497b2f8bf055a9" +checksum = "1429034a0490724d0075ebb2bc9e875d6503c3cf69e235a8941aa757d83ef5bf" dependencies = [ "autocfg", "num-integer", @@ -1878,9 +1885,9 @@ dependencies = [ [[package]] name = "num-traits" -version = "0.2.18" +version = "0.2.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "da0df0e5185db44f69b44f26786fe401b6c293d1907744beaa7fa62b2e5a517a" +checksum = "071dfc062690e90b734c0b2273ce72ad0ffa95f0c74596bc250dcfd960262841" dependencies = [ "autocfg", "libm", @@ -3067,11 +3074,11 @@ dependencies = [ [[package]] name = "security-framework" -version = "2.10.0" +version = "2.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "770452e37cad93e0a50d5abc3990d2bc351c36d0328f86cefec2f2fb206eaef6" +checksum = "c627723fd09706bacdb5cf41499e95098555af3c3c29d014dc3c458ef6be11c0" dependencies = [ - "bitflags 1.3.2", + "bitflags 2.5.0", "core-foundation", "core-foundation-sys", "libc", @@ -3080,9 +3087,9 @@ dependencies = [ [[package]] name = "security-framework-sys" -version = "2.10.0" +version = "2.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "41f3cc463c0ef97e11c3461a9d3787412d30e8e7eb907c79180c4a57bf7c04ef" +checksum = "317936bbbd05227752583946b9e66d7ce3b489f84e11a94a510b4437fef407d7" dependencies = [ "core-foundation-sys", "libc", @@ -3090,9 +3097,9 @@ dependencies = [ [[package]] name = "serde" -version = "1.0.199" +version = "1.0.200" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c9f6e76df036c77cd94996771fb40db98187f096dd0b9af39c6c6e452ba966a" +checksum = "ddc6f9cc94d67c0e21aaf7eda3a010fd3af78ebf6e096aa6e2e13c79749cce4f" dependencies = [ "serde_derive", ] @@ -3108,9 +3115,9 @@ dependencies = [ [[package]] name = "serde_derive" -version = "1.0.199" +version = "1.0.200" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "11bd257a6541e141e42ca6d24ae26f7714887b47e89aa739099104c7e4d3b7fc" +checksum = "856f046b9400cee3c8c94ed572ecdb752444c24528c035cd35882aad6f492bcb" dependencies = [ "proc-macro2", "quote", @@ -3614,16 +3621,15 @@ dependencies = [ [[package]] name = "tokio-util" -version = "0.7.10" +version = "0.7.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5419f34732d9eb6ee4c3578b7989078579b7f039cbbb9ca2c4da015749371e15" +checksum = "9cf6b47b3771c49ac75ad09a6162f53ad4b8088b76ac60e8ec1455b31a189fe1" dependencies = [ "bytes", "futures-core", "futures-sink", "pin-project-lite", "tokio", - "tracing", ] [[package]] @@ -3668,7 +3674,7 @@ dependencies = [ "serde", "serde_spanned", "toml_datetime", - "winnow 0.6.7", + "winnow 0.6.8", ] [[package]] @@ -4329,9 +4335,9 @@ dependencies = [ [[package]] name = "winnow" -version = "0.6.7" +version = "0.6.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "14b9415ee827af173ebb3f15f9083df5a122eb93572ec28741fb153356ea2578" +checksum = "c3c52e9c97a68071b23e836c9380edae937f17b9c4667bd021973efc689f618d" dependencies = [ "memchr", ] @@ -4412,18 +4418,18 @@ dependencies = [ [[package]] name = "zerocopy" -version = "0.7.32" +version = "0.7.33" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "74d4d3961e53fa4c9a25a8637fc2bfaf2595b3d3ae34875568a5cf64787716be" +checksum = "087eca3c1eaf8c47b94d02790dd086cd594b912d2043d4de4bfdd466b3befb7c" dependencies = [ "zerocopy-derive", ] [[package]] name = "zerocopy-derive" -version = "0.7.32" +version = "0.7.33" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ce1b18ccd8e73a9321186f97e46f9f04b778851177567b1975109d26a08d2a6" +checksum = "6f4b6c273f496d8fd4eaf18853e6b448760225dc030ff2c485a786859aea6393" dependencies = [ "proc-macro2", "quote", diff --git a/ui/package-lock.json b/ui/package-lock.json index 93a22cff58..9be71e968c 100644 --- a/ui/package-lock.json +++ b/ui/package-lock.json @@ -8,7 +8,7 @@ "name": "peerdb-ui", "version": "0.1.0", "dependencies": { - "@grpc/grpc-js": "^1.10.6", + "@grpc/grpc-js": "^1.10.7", "@monaco-editor/react": "^4.6.0", "@prisma/client": "^5.13.0", "@radix-ui/react-checkbox": "^1.0.4", @@ -27,7 +27,7 @@ "@radix-ui/react-toggle-group": "^1.0.4", "@radix-ui/react-tooltip": "^1.0.7", "@tremor/react": "^3.16.2", - "@types/node": "^20.12.7", + "@types/node": "^20.12.8", "@types/react": "^18.3.1", "@types/react-dom": "^18.3.0", "classnames": "^2.5.1", @@ -48,7 +48,7 @@ "styled-components": "^6.1.9", "swr": "^2.2.5", "usehooks-ts": "^3.1.0", - "zod": "^3.23.5" + "zod": "^3.23.6" }, "devDependencies": { "autoprefixer": "^10.4.19", @@ -405,9 +405,9 @@ } }, "node_modules/@floating-ui/dom": { - "version": "1.6.4", - "resolved": "https://registry.npmjs.org/@floating-ui/dom/-/dom-1.6.4.tgz", - "integrity": "sha512-0G8R+zOvQsAG1pg2Q99P21jiqxqGBW1iRe/iXHsBRBxnpXKFI8QwbB4x5KmYLggNO5m34IQgOIu9SCRfR/WWiQ==", + "version": "1.6.5", + "resolved": "https://registry.npmjs.org/@floating-ui/dom/-/dom-1.6.5.tgz", + "integrity": "sha512-Nsdud2X65Dz+1RHjAIP0t8z5e2ff/IRbei6BqFrl1urT8sDVzM1HMQ+R0XcU5ceRfyO3I6ayeqIfh+6Wb8LGTw==", "dependencies": { "@floating-ui/core": "^1.0.0", "@floating-ui/utils": "^0.2.0" @@ -457,11 +457,11 @@ "integrity": "sha512-J4yDIIthosAsRZ5CPYP/jQvUAQtlZTTD/4suA08/FEnlxqW3sKS9iAhgsa9VYLZ6vDHn/ixJgIqRQPotoBjxIw==" }, "node_modules/@grpc/grpc-js": { - "version": "1.10.6", - "resolved": "https://registry.npmjs.org/@grpc/grpc-js/-/grpc-js-1.10.6.tgz", - "integrity": "sha512-xP58G7wDQ4TCmN/cMUHh00DS7SRDv/+lC+xFLrTkMIN8h55X5NhZMLYbvy7dSELP15qlI6hPhNCRWVMtZMwqLA==", + "version": "1.10.7", + "resolved": "https://registry.npmjs.org/@grpc/grpc-js/-/grpc-js-1.10.7.tgz", + "integrity": "sha512-ZMBVjSeDAz3tFSehyO6Pd08xZT1HfIwq3opbeM4cDlBh52gmwp0wVIPcQur53NN0ac68HMZ/7SF2rGRD5KmVmg==", "dependencies": { - "@grpc/proto-loader": "^0.7.10", + "@grpc/proto-loader": "^0.7.13", "@js-sdsl/ordered-map": "^4.4.2" }, "engines": { @@ -469,13 +469,13 @@ } }, "node_modules/@grpc/proto-loader": { - "version": "0.7.12", - "resolved": "https://registry.npmjs.org/@grpc/proto-loader/-/proto-loader-0.7.12.tgz", - "integrity": "sha512-DCVwMxqYzpUCiDMl7hQ384FqP4T3DbNpXU8pt681l3UWCip1WUiD5JrkImUwCB9a7f2cq4CUTmi5r/xIMRPY1Q==", + "version": "0.7.13", + "resolved": "https://registry.npmjs.org/@grpc/proto-loader/-/proto-loader-0.7.13.tgz", + "integrity": "sha512-AiXO/bfe9bmxBjxxtYxFAXGZvMaN5s8kO+jBHAJCON8rJoB5YS/D6X7ZNc6XQkuHNmyl4CYaMI1fJ/Gn27RGGw==", "dependencies": { "lodash.camelcase": "^4.3.0", "long": "^5.0.0", - "protobufjs": "^7.2.4", + "protobufjs": "^7.2.5", "yargs": "^17.7.2" }, "bin": { @@ -2169,9 +2169,9 @@ "dev": true }, "node_modules/@types/node": { - "version": "20.12.7", - "resolved": "https://registry.npmjs.org/@types/node/-/node-20.12.7.tgz", - "integrity": "sha512-wq0cICSkRLVaf3UGLMGItu/PtdY7oaXaI/RVU+xliKVOtRna3PRY57ZDfztpDL0n11vfymMUnXv8QwYCO7L1wg==", + "version": "20.12.8", + "resolved": "https://registry.npmjs.org/@types/node/-/node-20.12.8.tgz", + "integrity": "sha512-NU0rJLJnshZWdE/097cdCBbyW1h4hEg0xpovcoAQYHl8dnEyp/NAOiE45pvc+Bd1Dt+2r94v2eGFpQJ4R7g+2w==", "dependencies": { "undici-types": "~5.26.4" } @@ -3103,9 +3103,9 @@ } }, "node_modules/caniuse-lite": { - "version": "1.0.30001614", - "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001614.tgz", - "integrity": "sha512-jmZQ1VpmlRwHgdP1/uiKzgiAuGOfLEJsYFP4+GBou/QQ4U6IOJCB4NP1c+1p9RGLpwObcT94jA5/uO+F1vBbog==", + "version": "1.0.30001616", + "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001616.tgz", + "integrity": "sha512-RHVYKov7IcdNjVHJFNY/78RdG4oGVjbayxv8u5IO74Wv7Hlq4PnJE6mo/OjFijjVFNy5ijnCt6H3IIo4t+wfEw==", "funding": [ { "type": "opencollective", @@ -3717,9 +3717,9 @@ "integrity": "sha512-I88TYZWc9XiYHRQ4/3c5rjjfgkjhLyW2luGIheGERbNQ6OY7yTybanSpDXZa8y7VUP9YmDcYa+eyq4ca7iLqWA==" }, "node_modules/electron-to-chromium": { - "version": "1.4.752", - "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.752.tgz", - "integrity": "sha512-P3QJreYI/AUTcfBVrC4zy9KvnZWekViThgQMX/VpJ+IsOBbcX5JFpORM4qWapwWQ+agb2nYAOyn/4PMXOk0m2Q==", + "version": "1.4.756", + "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.756.tgz", + "integrity": "sha512-RJKZ9+vEBMeiPAvKNWyZjuYyUqMndcP1f335oHqn3BEQbs2NFtVrnK5+6Xg5wSM9TknNNpWghGDUCKGYF+xWXw==", "dev": true }, "node_modules/email-addresses": { @@ -4725,9 +4725,9 @@ } }, "node_modules/get-tsconfig": { - "version": "4.7.3", - "resolved": "https://registry.npmjs.org/get-tsconfig/-/get-tsconfig-4.7.3.tgz", - "integrity": "sha512-ZvkrzoUA0PQZM6fy6+/Hce561s+faD1rsNwhnO5FelNjyy7EMGJ3Rz1AQ8GYDWjhRs/7dBLOEJvhK8MiEJOAFg==", + "version": "4.7.4", + "resolved": "https://registry.npmjs.org/get-tsconfig/-/get-tsconfig-4.7.4.tgz", + "integrity": "sha512-ofbkKj+0pjXjhejr007J/fLf+sW+8H7K5GCm+msC8q3IpvgjobpyPqSRFemNyIMxklC0zeJpi7VDFna19FacvQ==", "dev": true, "dependencies": { "resolve-pkg-maps": "^1.0.0" @@ -5979,9 +5979,9 @@ } }, "node_modules/minipass": { - "version": "7.0.4", - "resolved": "https://registry.npmjs.org/minipass/-/minipass-7.0.4.tgz", - "integrity": "sha512-jYofLM5Dam9279rdkWzqHozUo4ybjdZmCsDHePy5V/PbBcVMiSZR97gmAy45aqi8CK1lG2ECd356FU86avfwUQ==", + "version": "7.1.0", + "resolved": "https://registry.npmjs.org/minipass/-/minipass-7.1.0.tgz", + "integrity": "sha512-oGZRv2OT1lO2UF1zUcwdTb3wqUwI0kBGTgt/T7OdSj6M6N5m3o5uPf0AIW6lVxGGoiWUR7e2AwTE+xiwK8WQig==", "engines": { "node": ">=16 || 14 >=14.17" } @@ -8430,9 +8430,9 @@ } }, "node_modules/update-browserslist-db": { - "version": "1.0.14", - "resolved": "https://registry.npmjs.org/update-browserslist-db/-/update-browserslist-db-1.0.14.tgz", - "integrity": "sha512-JixKH8GR2pWYshIPUg/NujK3JO7JiqEEUiNArE86NQyrgUuZeTlZQN3xuS/yiV5Kb48ev9K6RqNkaJjXsdg7Jw==", + "version": "1.0.15", + "resolved": "https://registry.npmjs.org/update-browserslist-db/-/update-browserslist-db-1.0.15.tgz", + "integrity": "sha512-K9HWH62x3/EalU1U6sjSZiylm9C8tgq2mSvshZpqc7QE69RaA2qjhkW2HlNA0tFpEbtyFz7HTqbSdN4MSwUodA==", "dev": true, "funding": [ { @@ -8975,9 +8975,9 @@ } }, "node_modules/zod": { - "version": "3.23.5", - "resolved": "https://registry.npmjs.org/zod/-/zod-3.23.5.tgz", - "integrity": "sha512-fkwiq0VIQTksNNA131rDOsVJcns0pfVUjHzLrNBiF/O/Xxb5lQyEXkhZWcJ7npWsYlvs+h0jFWXXy4X46Em1JA==", + "version": "3.23.6", + "resolved": "https://registry.npmjs.org/zod/-/zod-3.23.6.tgz", + "integrity": "sha512-RTHJlZhsRbuA8Hmp/iNL7jnfc4nZishjsanDAfEY1QpDQZCahUp3xDzl+zfweE9BklxMUcgBgS1b7Lvie/ZVwA==", "funding": { "url": "https://github.com/sponsors/colinhacks" } diff --git a/ui/package.json b/ui/package.json index 65d07edb40..d66bb51b8c 100644 --- a/ui/package.json +++ b/ui/package.json @@ -10,7 +10,7 @@ "format": "prettier --write ." }, "dependencies": { - "@grpc/grpc-js": "^1.10.6", + "@grpc/grpc-js": "^1.10.7", "@monaco-editor/react": "^4.6.0", "@prisma/client": "^5.13.0", "@radix-ui/react-checkbox": "^1.0.4", @@ -29,7 +29,7 @@ "@radix-ui/react-toggle-group": "^1.0.4", "@radix-ui/react-tooltip": "^1.0.7", "@tremor/react": "^3.16.2", - "@types/node": "^20.12.7", + "@types/node": "^20.12.8", "@types/react": "^18.3.1", "@types/react-dom": "^18.3.0", "classnames": "^2.5.1", @@ -50,7 +50,7 @@ "styled-components": "^6.1.9", "swr": "^2.2.5", "usehooks-ts": "^3.1.0", - "zod": "^3.23.5" + "zod": "^3.23.6" }, "devDependencies": { "autoprefixer": "^10.4.19", From e8ebd3c0dca3d72c965e58c1b00e18a7a8f2197e Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Tue, 7 May 2024 00:48:07 +0530 Subject: [PATCH 24/41] Patch clickhouse SQL parsing (#1676) Patches clickhouse peer creation from query layer ```sql CREATE PEER FROM CLICKHOUSE WITH ( host='', port=, user='', password='', database='', -- disable_tls = true ); ``` Functionally tested: ``` amogh=> create peer mych from CLICKHOUSE with(host='host.docker.internal', port=9000, user='default', password='clickhouse', database='clickhouse', disable_tls = true); OK ``` ``` amogh=> create peer mych from CLICKHOUSE with(host='host.docker.internal', port=9000, user='default', password='clickhoufse', database='clickhouse', disable_tls = true); ERROR: User provided error: ErrorInfo { severity: "ERROR", code: "internal_error", message: "[peer]: invalid configuration: CLICKHOUSE peer mych was invalidated: failed to open connection to Clickhouse peer: failed to ping to Clickhouse peer: code: 516, message: default: Authentication failed: password is incorrect, or there is no user with such name. ``` --- nexus/Cargo.lock | 4 ++-- nexus/analyzer/src/lib.rs | 19 +++++++++---------- nexus/pt/src/lib.rs | 1 + 3 files changed, 12 insertions(+), 12 deletions(-) diff --git a/nexus/Cargo.lock b/nexus/Cargo.lock index d5985ed3b9..ab9e0554db 100644 --- a/nexus/Cargo.lock +++ b/nexus/Cargo.lock @@ -3304,7 +3304,7 @@ dependencies = [ [[package]] name = "sqlparser" version = "0.45.0" -source = "git+https://github.com/peerdb-io/sqlparser-rs.git?branch=main#d76527bd14ceb5f0b739feb37a8912a9ad7f864c" +source = "git+https://github.com/peerdb-io/sqlparser-rs.git?branch=main#8c341b80ff24a3e9172c23d75bfa8e1a54e385e5" dependencies = [ "log", "sqlparser_derive", @@ -3313,7 +3313,7 @@ dependencies = [ [[package]] name = "sqlparser_derive" version = "0.2.2" -source = "git+https://github.com/peerdb-io/sqlparser-rs.git?branch=main#d76527bd14ceb5f0b739feb37a8912a9ad7f864c" +source = "git+https://github.com/peerdb-io/sqlparser-rs.git?branch=main#8c341b80ff24a3e9172c23d75bfa8e1a54e385e5" dependencies = [ "proc-macro2", "quote", diff --git a/nexus/analyzer/src/lib.rs b/nexus/analyzer/src/lib.rs index 83bea8d59c..b0ff741612 100644 --- a/nexus/analyzer/src/lib.rs +++ b/nexus/analyzer/src/lib.rs @@ -728,24 +728,23 @@ fn parse_db_options(db_type: DbType, with_options: &[SqlOption]) -> anyhow::Resu .to_string(), s3_path: opts .get("s3_path") - .context("no s3 path specified")? - .to_string(), + .map(|s| s.to_string()) + .unwrap_or_default(), access_key_id: opts .get("access_key_id") - .context("no access key id specified")? - .to_string(), + .map(|s| s.to_string()) + .unwrap_or_default(), secret_access_key: opts .get("secret_access_key") - .context("no secret access key specified")? - .to_string(), + .map(|s| s.to_string()) + .unwrap_or_default(), region: opts .get("region") - .context("no region specified")? - .to_string(), + .map(|s| s.to_string()) + .unwrap_or_default(), disable_tls: opts .get("disable_tls") - .and_then(|s| s.parse::().ok()) - .unwrap_or_default(), + .map(|s| s.parse::().unwrap_or_default()).unwrap_or_default(), endpoint: opts.get("endpoint").map(|s| s.to_string()), }; Config::ClickhouseConfig(clickhouse_config) diff --git a/nexus/pt/src/lib.rs b/nexus/pt/src/lib.rs index 137cb3909f..e14144efd0 100644 --- a/nexus/pt/src/lib.rs +++ b/nexus/pt/src/lib.rs @@ -31,6 +31,7 @@ impl From for DbType { PeerType::Eventhubs => DbType::Eventhubs, PeerType::PubSub => DbType::Pubsub, PeerType::Elasticsearch => DbType::Elasticsearch, + PeerType::Clickhouse => DbType::Clickhouse, } } } From 23a3ec6954ad2043273397dcfd09c6f4e3abd7ad Mon Sep 17 00:00:00 2001 From: Yasin Zaehringer Date: Tue, 7 May 2024 14:36:30 +0200 Subject: [PATCH 25/41] qrep: if the last sync is nil, check the table if there are new rows. (#1667) this bug can prevent tables which are empty to be synced once they get data. in general one has to be very careful here with nil pointer exceptions, I tried to navigate them to the best of my ability but this code path makes lots of assumptions about the state of the world. --- flow/activities/flowable.go | 2 +- flow/connectors/postgres/qrep.go | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index 2291400b03..7571c74b55 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -643,7 +643,7 @@ func (a *FlowableActivity) QRepHasNewRows(ctx context.Context, ctx = context.WithValue(ctx, shared.FlowNameKey, config.FlowJobName) logger := log.With(activity.GetLogger(ctx), slog.String(string(shared.FlowNameKey), config.FlowJobName)) - if config.SourcePeer.Type != protos.DBType_POSTGRES || last.Range == nil { + if config.SourcePeer.Type != protos.DBType_POSTGRES { return QRepWaitUntilNewRowsResult{Found: true}, nil } diff --git a/flow/connectors/postgres/qrep.go b/flow/connectors/postgres/qrep.go index 8ec936d2ea..0d298a193f 100644 --- a/flow/connectors/postgres/qrep.go +++ b/flow/connectors/postgres/qrep.go @@ -287,6 +287,10 @@ func (c *PostgresConnector) CheckForUpdatedMaxValue( return false, fmt.Errorf("error while getting min and max values: %w", err) } + if last == nil || last.Range == nil { + return maxValue != nil, nil + } + switch x := last.Range.Range.(type) { case *protos.PartitionRange_IntRange: if maxValue.(int64) > x.IntRange.End { From 5104b00a6042998c8a3012854af8e9d575a3b42c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Tue, 7 May 2024 13:25:52 +0000 Subject: [PATCH 26/41] fix ci (#1678) regression in #1667 that wasn't caught because CI doesn't work for external PRs --- flow/workflows/qrep_flow.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flow/workflows/qrep_flow.go b/flow/workflows/qrep_flow.go index a988e1f96c..8be2619866 100644 --- a/flow/workflows/qrep_flow.go +++ b/flow/workflows/qrep_flow.go @@ -570,7 +570,7 @@ func QRepFlowWorkflow( return err } - if state.LastPartition != nil { + if !config.InitialCopyOnly && state.LastPartition != nil { if err := q.waitForNewRows(ctx, signalChan, state.LastPartition); err != nil { return err } From eae0f3203afad3701c259734418c08cb024ac26b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Tue, 7 May 2024 14:09:08 +0000 Subject: [PATCH 27/41] initial load for queues (#1675) Turns out waitgroup unnecessary since kgo flush handles wait, so removed Meanwhile, potential send-after-close found in pubsub, so fix EventHubs is complicated, so skipped. Can be followup --- flow/connectors/core.go | 3 +- flow/connectors/kafka/kafka.go | 57 ++++----- flow/connectors/kafka/qrep.go | 111 +++++++++++++++++ flow/connectors/pubsub/pubsub.go | 116 ++++++++++-------- flow/connectors/pubsub/qrep.go | 128 ++++++++++++++++++++ flow/connectors/snowflake/qrep_avro_sync.go | 2 +- flow/e2e/kafka/kafka_test.go | 56 +++++++++ flow/e2e/pubsub/pubsub_test.go | 70 +++++++++++ flow/workflows/snapshot_flow.go | 1 + protos/flow.proto | 1 + 10 files changed, 459 insertions(+), 86 deletions(-) create mode 100644 flow/connectors/kafka/qrep.go create mode 100644 flow/connectors/pubsub/qrep.go diff --git a/flow/connectors/core.go b/flow/connectors/core.go index dc7afe0495..0814feabd9 100644 --- a/flow/connectors/core.go +++ b/flow/connectors/core.go @@ -329,8 +329,9 @@ var ( _ QRepSyncConnector = &connpostgres.PostgresConnector{} _ QRepSyncConnector = &connbigquery.BigQueryConnector{} _ QRepSyncConnector = &connsnowflake.SnowflakeConnector{} - _ QRepSyncConnector = &connclickhouse.ClickhouseConnector{} + _ QRepSyncConnector = &connkafka.KafkaConnector{} _ QRepSyncConnector = &conns3.S3Connector{} + _ QRepSyncConnector = &connclickhouse.ClickhouseConnector{} _ QRepSyncConnector = &connelasticsearch.ElasticsearchConnector{} _ QRepConsolidateConnector = &connsnowflake.SnowflakeConnector{} diff --git a/flow/connectors/kafka/kafka.go b/flow/connectors/kafka/kafka.go index 30b370e51e..cfe4652598 100644 --- a/flow/connectors/kafka/kafka.go +++ b/flow/connectors/kafka/kafka.go @@ -6,7 +6,6 @@ import ( "fmt" "log/slog" "strings" - "sync" "sync/atomic" "time" @@ -166,42 +165,48 @@ 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.RecordItems]) (*model.SyncResponse, error) { - var wg sync.WaitGroup - wgCtx, wgErr := context.WithCancelCause(ctx) +func (c *KafkaConnector) createPool( + ctx context.Context, + script string, + flowJobName string, + queueErr func(error), +) (*utils.LPool[[]*kgo.Record], error) { produceCb := func(_ *kgo.Record, err error) { if err != nil { - wgErr(err) + queueErr(err) } - wg.Done() } - numRecords := atomic.Int64{} - tableNameRowsMapping := utils.InitialiseTableRowsMap(req.TableMappings) - - pool, err := utils.LuaPool(func() (*lua.LState, error) { - ls, err := utils.LoadScript(wgCtx, req.Script, func(ls *lua.LState) int { + return utils.LuaPool(func() (*lua.LState, error) { + ls, err := utils.LoadScript(ctx, 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")) + _ = c.LogFlowInfo(ctx, flowJobName, strings.Join(ss, "\t")) return 0 }) if err != nil { return nil, err } - if req.Script == "" { + if 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.client.Produce(ctx, kr, produceCb) } }) +} + +func (c *KafkaConnector) SyncRecords(ctx context.Context, req *model.SyncRecordsRequest[model.RecordItems]) (*model.SyncResponse, error) { + numRecords := atomic.Int64{} + tableNameRowsMapping := utils.InitialiseTableRowsMap(req.TableMappings) + + queueCtx, queueErr := context.WithCancelCause(ctx) + pool, err := c.createPool(queueCtx, req.Script, req.FlowJobName, queueErr) if err != nil { return nil, err } @@ -250,7 +255,7 @@ Loop: lfn := ls.Env.RawGetString("onRecord") fn, ok := lfn.(*lua.LFunction) if !ok { - wgErr(fmt.Errorf("script should define `onRecord` as function, not %s", lfn)) + queueErr(fmt.Errorf("script should define `onRecord` as function, not %s", lfn)) return nil } @@ -258,7 +263,7 @@ Loop: ls.Push(pua.LuaRecord.New(ls, record)) err := ls.PCall(1, -1, nil) if err != nil { - wgErr(fmt.Errorf("script failed: %w", err)) + queueErr(fmt.Errorf("script failed: %w", err)) return nil } @@ -267,7 +272,7 @@ Loop: for i := range args { kr, err := lvalueToKafkaRecord(ls, ls.Get(i-args)) if err != nil { - wgErr(err) + queueErr(err) return nil } if kr != nil { @@ -284,28 +289,18 @@ Loop: return results }) - case <-wgCtx.Done(): + case <-queueCtx.Done(): break Loop } } close(flushLoopDone) - if err := pool.Wait(wgCtx); err != nil { + if err := pool.Wait(queueCtx); err != nil { return nil, err } - if err := c.client.Flush(wgCtx); err != nil { + if err := c.client.Flush(queueCtx); err != nil { return nil, fmt.Errorf("[kafka] final flush error: %w", err) } - waitChan := make(chan struct{}) - go func() { - wg.Wait() - close(waitChan) - }() - select { - case <-wgCtx.Done(): - return nil, wgCtx.Err() - case <-waitChan: - } lastCheckpoint := req.Records.GetLastCheckpoint() if err := c.FinishBatch(ctx, req.FlowJobName, req.SyncBatchID, lastCheckpoint); err != nil { diff --git a/flow/connectors/kafka/qrep.go b/flow/connectors/kafka/qrep.go new file mode 100644 index 0000000000..a856ad1ccf --- /dev/null +++ b/flow/connectors/kafka/qrep.go @@ -0,0 +1,111 @@ +package connkafka + +import ( + "context" + "fmt" + "sync/atomic" + "time" + + "github.com/twmb/franz-go/pkg/kgo" + lua "github.com/yuin/gopher-lua" + + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/model" + "github.com/PeerDB-io/peer-flow/pua" +) + +func (*KafkaConnector) SetupQRepMetadataTables(_ context.Context, _ *protos.QRepConfig) error { + return nil +} + +func (c *KafkaConnector) SyncQRepRecords( + ctx context.Context, + config *protos.QRepConfig, + partition *protos.QRepPartition, + stream *model.QRecordStream, +) (int, error) { + startTime := time.Now() + numRecords := atomic.Int64{} + schema := stream.Schema() + + queueCtx, queueErr := context.WithCancelCause(ctx) + pool, err := c.createPool(queueCtx, config.Script, config.FlowJobName, queueErr) + if err != nil { + return 0, err + } + defer pool.Close() + +Loop: + for { + select { + case qrecord, ok := <-stream.Records: + if !ok { + c.logger.Info("flushing batches because no more records") + break Loop + } + + pool.Run(func(ls *lua.LState) []*kgo.Record { + items := model.NewRecordItems(len(qrecord)) + for i, val := range qrecord { + items.AddColumn(schema.Fields[i].Name, val) + } + record := &model.InsertRecord[model.RecordItems]{ + BaseRecord: model.BaseRecord{}, + Items: items, + SourceTableName: config.WatermarkTable, + DestinationTableName: config.DestinationTableIdentifier, + CommitID: 0, + } + + lfn := ls.Env.RawGetString("onRecord") + fn, ok := lfn.(*lua.LFunction) + if !ok { + queueErr(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 { + queueErr(fmt.Errorf("script failed: %w", err)) + return nil + } + + args := ls.GetTop() + results := make([]*kgo.Record, 0, args) + for i := range args { + kr, err := lvalueToKafkaRecord(ls, ls.Get(i-args)) + if err != nil { + queueErr(err) + return nil + } + if kr != nil { + if kr.Topic == "" { + kr.Topic = record.GetDestinationTableName() + } + results = append(results, kr) + } + } + ls.SetTop(0) + numRecords.Add(1) + return results + }) + + case <-queueCtx.Done(): + break Loop + } + } + + if err := pool.Wait(queueCtx); err != nil { + return 0, err + } + if err := c.client.Flush(queueCtx); err != nil { + return 0, fmt.Errorf("[kafka] final flush error: %w", err) + } + + if err := c.FinishQRepPartition(ctx, partition, config.FlowJobName, startTime); err != nil { + return 0, err + } + return int(numRecords.Load()), nil +} diff --git a/flow/connectors/pubsub/pubsub.go b/flow/connectors/pubsub/pubsub.go index b08150960c..54031f016d 100644 --- a/flow/connectors/pubsub/pubsub.go +++ b/flow/connectors/pubsub/pubsub.go @@ -121,6 +121,57 @@ func lvalueToPubSubMessage(ls *lua.LState, value lua.LValue) (PubSubMessage, err }, nil } +func (c *PubSubConnector) createPool( + ctx context.Context, + script string, + flowJobName string, + topiccache *topicCache, + publish chan<- *pubsub.PublishResult, + queueErr func(error), +) (*utils.LPool[[]PubSubMessage], error) { + return utils.LuaPool(func() (*lua.LState, error) { + ls, err := utils.LoadScript(ctx, 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")) + return 0 + }) + if err != nil { + return nil, err + } + if 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(ctx) + if err != nil { + return nil, fmt.Errorf("error checking if topic exists: %w", err) + } + if !exists { + topicClient, err = c.client.CreateTopic(ctx, message.Topic) + if err != nil { + return nil, fmt.Errorf("error creating topic: %w", err) + } + } + return topicClient, nil + }) + if err != nil { + queueErr(err) + return + } + + publish <- topicClient.Publish(ctx, message.Message) + } + }) +} + type topicCache struct { cache map[string]*pubsub.Topic lock sync.RWMutex @@ -175,51 +226,10 @@ func (c *PubSubConnector) SyncRecords(ctx context.Context, req *model.SyncRecord tableNameRowsMapping := utils.InitialiseTableRowsMap(req.TableMappings) topiccache := topicCache{cache: make(map[string]*pubsub.Topic)} publish := make(chan *pubsub.PublishResult, 32) - 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) - } - }) + queueCtx, queueErr := context.WithCancelCause(ctx) + pool, err := c.createPool(queueCtx, req.Script, req.FlowJobName, &topiccache, publish, queueErr) if err != nil { return nil, err } @@ -228,7 +238,7 @@ func (c *PubSubConnector) SyncRecords(ctx context.Context, req *model.SyncRecord go func() { for curpub := range publish { if _, err := curpub.Get(ctx); err != nil { - wgErr(err) + queueErr(err) break } } @@ -275,7 +285,7 @@ Loop: lfn := ls.Env.RawGetString("onRecord") fn, ok := lfn.(*lua.LFunction) if !ok { - wgErr(fmt.Errorf("script should define `onRecord` as function, not %s", lfn)) + queueErr(fmt.Errorf("script should define `onRecord` as function, not %s", lfn)) return nil } @@ -283,7 +293,7 @@ Loop: ls.Push(pua.LuaRecord.New(ls, record)) err := ls.PCall(1, -1, nil) if err != nil { - wgErr(fmt.Errorf("script failed: %w", err)) + queueErr(fmt.Errorf("script failed: %w", err)) return nil } @@ -292,7 +302,7 @@ Loop: for i := range args { msg, err := lvalueToPubSubMessage(ls, ls.Get(i-args)) if err != nil { - wgErr(err) + queueErr(err) return nil } if msg.Message != nil { @@ -309,20 +319,20 @@ Loop: return results }) - case <-wgCtx.Done(): + case <-queueCtx.Done(): break Loop } } close(flushLoopDone) - close(publish) - if err := pool.Wait(wgCtx); err != nil { + if err := pool.Wait(queueCtx); err != nil { return nil, err } - topiccache.Stop(wgCtx) + close(publish) + topiccache.Stop(queueCtx) select { - case <-wgCtx.Done(): - return nil, wgCtx.Err() + case <-queueCtx.Done(): + return nil, queueCtx.Err() case <-waitChan: } diff --git a/flow/connectors/pubsub/qrep.go b/flow/connectors/pubsub/qrep.go new file mode 100644 index 0000000000..c1f21edc4a --- /dev/null +++ b/flow/connectors/pubsub/qrep.go @@ -0,0 +1,128 @@ +package connpubsub + +import ( + "context" + "fmt" + "sync/atomic" + "time" + + "cloud.google.com/go/pubsub" + lua "github.com/yuin/gopher-lua" + + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/model" + "github.com/PeerDB-io/peer-flow/pua" +) + +func (*PubSubConnector) SetupQRepMetadataTables(_ context.Context, _ *protos.QRepConfig) error { + return nil +} + +func (c *PubSubConnector) SyncQRepRecords( + ctx context.Context, + config *protos.QRepConfig, + partition *protos.QRepPartition, + stream *model.QRecordStream, +) (int, error) { + startTime := time.Now() + numRecords := atomic.Int64{} + schema := stream.Schema() + topiccache := topicCache{cache: make(map[string]*pubsub.Topic)} + publish := make(chan *pubsub.PublishResult, 32) + waitChan := make(chan struct{}) + + queueCtx, queueErr := context.WithCancelCause(ctx) + pool, err := c.createPool(queueCtx, config.Script, config.FlowJobName, &topiccache, publish, queueErr) + if err != nil { + return 0, err + } + defer pool.Close() + + go func() { + for curpub := range publish { + if _, err := curpub.Get(ctx); err != nil { + queueErr(err) + break + } + } + close(waitChan) + }() + +Loop: + for { + select { + case qrecord, ok := <-stream.Records: + if !ok { + c.logger.Info("flushing batches because no more records") + break Loop + } + + pool.Run(func(ls *lua.LState) []PubSubMessage { + items := model.NewRecordItems(len(qrecord)) + for i, val := range qrecord { + items.AddColumn(schema.Fields[i].Name, val) + } + record := &model.InsertRecord[model.RecordItems]{ + BaseRecord: model.BaseRecord{}, + Items: items, + SourceTableName: config.WatermarkTable, + DestinationTableName: config.DestinationTableIdentifier, + CommitID: 0, + } + + lfn := ls.Env.RawGetString("onRecord") + fn, ok := lfn.(*lua.LFunction) + if !ok { + queueErr(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 { + queueErr(fmt.Errorf("script failed: %w", err)) + return nil + } + + args := ls.GetTop() + results := make([]PubSubMessage, 0, args) + for i := range args { + msg, err := lvalueToPubSubMessage(ls, ls.Get(i-args)) + if err != nil { + queueErr(err) + return nil + } + if msg.Message != nil { + if msg.Topic == "" { + msg.Topic = record.GetDestinationTableName() + } + results = append(results, msg) + } + } + ls.SetTop(0) + numRecords.Add(1) + return results + }) + + case <-queueCtx.Done(): + break Loop + } + } + + if err := pool.Wait(queueCtx); err != nil { + return 0, err + } + close(publish) + topiccache.Stop(queueCtx) + select { + case <-queueCtx.Done(): + return 0, queueCtx.Err() + case <-waitChan: + } + + if err := c.FinishQRepPartition(ctx, partition, config.FlowJobName, startTime); err != nil { + return 0, err + } + return int(numRecords.Load()), nil +} diff --git a/flow/connectors/snowflake/qrep_avro_sync.go b/flow/connectors/snowflake/qrep_avro_sync.go index f42c9f4cd8..fadd09deab 100644 --- a/flow/connectors/snowflake/qrep_avro_sync.go +++ b/flow/connectors/snowflake/qrep_avro_sync.go @@ -125,7 +125,7 @@ func (s *SnowflakeAvroSyncHandler) SyncQRepRecords( err = s.connector.FinishQRepPartition(ctx, partition, config.FlowJobName, startTime) if err != nil { - return -1, err + return 0, err } activity.RecordHeartbeat(ctx, "finished syncing records") diff --git a/flow/e2e/kafka/kafka_test.go b/flow/e2e/kafka/kafka_test.go index 83d4a42dd2..180383bd73 100644 --- a/flow/e2e/kafka/kafka_test.go +++ b/flow/e2e/kafka/kafka_test.go @@ -192,3 +192,59 @@ func (s KafkaSuite) TestDefault() { env.Cancel() e2e.RequireEnvCanceled(s.t, env) } + +func (s KafkaSuite) TestInitialLoad() { + srcTableName := e2e.AttachSchema(s, "kainitial") + + _, err := s.Conn().Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id SERIAL PRIMARY KEY, + val text + ); + `, srcTableName)) + require.NoError(s.t, err) + + flowName := e2e.AddSuffix(s, "kainitial") + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: flowName, + TableNameMapping: map[string]string{srcTableName: flowName}, + Destination: s.Peer(), + } + flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() + flowConnConfig.DoInitialSnapshot = true + + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s (id, val) VALUES (1, 'testval') + `, srcTableName)) + require.NoError(s.t, err) + + tc := e2e.NewTemporalClient(s.t) + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + + e2e.EnvWaitFor(s.t, env, 3*time.Minute, "normalize insert", func() bool { + kafka, err := kgo.NewClient( + kgo.SeedBrokers("localhost:9092"), + kgo.ConsumeTopics(flowName), + ) + if err != nil { + return false + } + defer kafka.Close() + + ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + defer cancel() + fetches := kafka.PollFetches(ctx) + fetches.EachTopic(func(ft kgo.FetchTopic) { + require.Equal(s.t, flowName, ft.Topic) + ft.EachRecord(func(r *kgo.Record) { + require.Contains(s.t, string(r.Value), "\"testval\"") + require.Equal(s.t, byte('{'), r.Value[0]) + require.Equal(s.t, byte('}'), r.Value[len(r.Value)-1]) + }) + }) + return true + }) + env.Cancel() + e2e.RequireEnvCanceled(s.t, env) +} diff --git a/flow/e2e/pubsub/pubsub_test.go b/flow/e2e/pubsub/pubsub_test.go index ec03fbb111..df6809b819 100644 --- a/flow/e2e/pubsub/pubsub_test.go +++ b/flow/e2e/pubsub/pubsub_test.go @@ -237,3 +237,73 @@ func (s PubSubSuite) TestSimple() { env.Cancel() e2e.RequireEnvCanceled(s.t, env) } + +func (s PubSubSuite) TestInitialLoad() { + srcTableName := e2e.AttachSchema(s, "psinitial") + + _, err := s.Conn().Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id SERIAL PRIMARY KEY, + val text + ); + `, srcTableName)) + require.NoError(s.t, err) + + sa, err := ServiceAccount() + require.NoError(s.t, err) + + _, err = s.Conn().Exec(context.Background(), `insert into public.scripts (name, lang, source) values + ('e2e_psinitial', 'lua', 'function onRecord(r) return r.row and r.row.val end') on conflict do nothing`) + require.NoError(s.t, err) + + flowName := e2e.AddSuffix(s, "e2epsinitial") + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: flowName, + TableNameMapping: map[string]string{srcTableName: flowName}, + Destination: s.Peer(sa), + } + flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() + flowConnConfig.Script = "e2e_psinitial" + flowConnConfig.DoInitialSnapshot = true + + psclient, err := sa.CreatePubSubClient(context.Background()) + require.NoError(s.t, err) + defer psclient.Close() + topic, err := psclient.CreateTopic(context.Background(), flowName) + require.NoError(s.t, err) + sub, err := psclient.CreateSubscription(context.Background(), flowName, pubsub.SubscriptionConfig{ + Topic: topic, + RetentionDuration: 10 * time.Minute, + ExpirationPolicy: 24 * time.Hour, + }) + require.NoError(s.t, err) + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s (id, val) VALUES (1, 'testval') + `, srcTableName)) + require.NoError(s.t, err) + + tc := e2e.NewTemporalClient(s.t) + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, connectionGen) + + ctx, cancel := context.WithTimeout(context.Background(), 3*time.Minute) + defer cancel() + + msgs := make(chan *pubsub.Message) + go func() { + _ = sub.Receive(ctx, func(_ context.Context, m *pubsub.Message) { + msgs <- m + }) + }() + select { + case msg := <-msgs: + require.NotNil(s.t, msg) + require.Equal(s.t, "testval", string(msg.Data)) + case <-ctx.Done(): + s.t.Log("UNEXPECTED TIMEOUT PubSub subscription waiting on message") + s.t.Fail() + } + + env.Cancel() + e2e.RequireEnvCanceled(s.t, env) +} diff --git a/flow/workflows/snapshot_flow.go b/flow/workflows/snapshot_flow.go index fdec9d15c6..10757774d6 100644 --- a/flow/workflows/snapshot_flow.go +++ b/flow/workflows/snapshot_flow.go @@ -199,6 +199,7 @@ func (s *SnapshotFlowExecution) cloneTable( SoftDeleteColName: s.config.SoftDeleteColName, WriteMode: snapshotWriteMode, System: s.config.System, + Script: s.config.Script, } state := NewQRepFlowState() diff --git a/protos/flow.proto b/protos/flow.proto index 46516fea29..4d4343f71a 100644 --- a/protos/flow.proto +++ b/protos/flow.proto @@ -297,6 +297,7 @@ message QRepConfig { string soft_delete_col_name = 17; TypeSystem system = 18; + string script = 19; } message QRepPartition { From 9967be5985317dbb50bb9ec18f4aa09e44b61291 Mon Sep 17 00:00:00 2001 From: exitxhu <59249127+exitxhu@users.noreply.github.com> Date: Tue, 7 May 2024 17:48:52 +0330 Subject: [PATCH 28/41] fix GetColumns script (#1681) fix this issue: https://github.com/PeerDB-io/peerdb/issues/1680 --- flow/cmd/peer_data.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/flow/cmd/peer_data.go b/flow/cmd/peer_data.go index ee514e2dfc..c88cb0c868 100644 --- a/flow/cmd/peer_data.go +++ b/flow/cmd/peer_data.go @@ -205,11 +205,13 @@ func (h *FlowRequestHandler) GetColumns( pg_attribute JOIN pg_class ON pg_attribute.attrelid = pg_class.oid + JOIN + pg_namespace on pg_class.relnamespace = pg_namespace.oid LEFT JOIN pg_constraint ON pg_attribute.attrelid = pg_constraint.conrelid AND pg_attribute.attnum = ANY(pg_constraint.conkey) WHERE - relnamespace::regnamespace::text = $1 + pg_namespace.nspname = $1 AND relname = $2 AND pg_attribute.attnum > 0 From 756679ebdbca68e26b1dbbb3918307134e5a9bf5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Tue, 7 May 2024 18:19:38 +0000 Subject: [PATCH 29/41] PopulateCountMap: use atomics (#1682) Race existed with parallel queue processing --- flow/connectors/utils/monitoring/monitoring.go | 8 +++++--- flow/connectors/utils/stream.go | 6 +----- flow/model/qrecord_stream.go | 8 +++++--- flow/model/record.go | 6 +++--- 4 files changed, 14 insertions(+), 14 deletions(-) diff --git a/flow/connectors/utils/monitoring/monitoring.go b/flow/connectors/utils/monitoring/monitoring.go index 7ecf248c0a..16f65655c5 100644 --- a/flow/connectors/utils/monitoring/monitoring.go +++ b/flow/connectors/utils/monitoring/monitoring.go @@ -123,14 +123,16 @@ func AddCDCBatchTablesForFlow(ctx context.Context, pool *pgxpool.Pool, flowJobNa }() for destinationTableName, rowCounts := range tableNameRowsMapping { - numRows := rowCounts.InsertCount + rowCounts.UpdateCount + rowCounts.DeleteCount + inserts := rowCounts.InsertCount.Load() + updates := rowCounts.UpdateCount.Load() + deletes := rowCounts.DeleteCount.Load() _, err = insertBatchTablesTx.Exec(ctx, `INSERT INTO peerdb_stats.cdc_batch_table (flow_name,batch_id,destination_table_name,num_rows, insert_count,update_count,delete_count) VALUES($1,$2,$3,$4,$5,$6,$7) ON CONFLICT DO NOTHING`, - flowJobName, batchID, destinationTableName, numRows, - rowCounts.InsertCount, rowCounts.UpdateCount, rowCounts.DeleteCount) + flowJobName, batchID, destinationTableName, + inserts+updates+deletes, inserts, updates, deletes) if err != nil { return fmt.Errorf("error while inserting statistics into cdc_batch_table: %w", err) } diff --git a/flow/connectors/utils/stream.go b/flow/connectors/utils/stream.go index 483d0f58ee..00688cc2ce 100644 --- a/flow/connectors/utils/stream.go +++ b/flow/connectors/utils/stream.go @@ -130,11 +130,7 @@ func recordToQRecordOrError[Items model.Items](batchID int64, record model.Recor func InitialiseTableRowsMap(tableMaps []*protos.TableMapping) map[string]*model.RecordTypeCounts { tableNameRowsMapping := make(map[string]*model.RecordTypeCounts, len(tableMaps)) for _, mapping := range tableMaps { - tableNameRowsMapping[mapping.DestinationTableIdentifier] = &model.RecordTypeCounts{ - InsertCount: 0, - UpdateCount: 0, - DeleteCount: 0, - } + tableNameRowsMapping[mapping.DestinationTableIdentifier] = &model.RecordTypeCounts{} } return tableNameRowsMapping diff --git a/flow/model/qrecord_stream.go b/flow/model/qrecord_stream.go index 0a0c26d2e4..576074db3b 100644 --- a/flow/model/qrecord_stream.go +++ b/flow/model/qrecord_stream.go @@ -1,13 +1,15 @@ package model import ( + "sync/atomic" + "github.com/PeerDB-io/peer-flow/model/qvalue" ) type RecordTypeCounts struct { - InsertCount int - UpdateCount int - DeleteCount int + InsertCount atomic.Int32 + UpdateCount atomic.Int32 + DeleteCount atomic.Int32 } type QRecordStream struct { diff --git a/flow/model/record.go b/flow/model/record.go index 30f3493cf4..9b728ff705 100644 --- a/flow/model/record.go +++ b/flow/model/record.go @@ -58,7 +58,7 @@ func (r *InsertRecord[T]) GetItems() T { func (r *InsertRecord[T]) PopulateCountMap(mapOfCounts map[string]*RecordTypeCounts) { recordCount, ok := mapOfCounts[r.DestinationTableName] if ok { - recordCount.InsertCount++ + recordCount.InsertCount.Add(1) } } @@ -91,7 +91,7 @@ func (r *UpdateRecord[T]) GetItems() T { func (r *UpdateRecord[T]) PopulateCountMap(mapOfCounts map[string]*RecordTypeCounts) { recordCount, ok := mapOfCounts[r.DestinationTableName] if ok { - recordCount.UpdateCount++ + recordCount.UpdateCount.Add(1) } } @@ -122,7 +122,7 @@ func (r *DeleteRecord[T]) GetItems() T { func (r *DeleteRecord[T]) PopulateCountMap(mapOfCounts map[string]*RecordTypeCounts) { recordCount, ok := mapOfCounts[r.DestinationTableName] if ok { - recordCount.DeleteCount++ + recordCount.DeleteCount.Add(1) } } From a11c3d4bf627d9949708204db97c9241aefcde01 Mon Sep 17 00:00:00 2001 From: Kevin Biju <52661649+heavycrystal@users.noreply.github.com> Date: Wed, 8 May 2024 01:37:05 +0530 Subject: [PATCH 30/41] Docker + run scripts minor fixes (#1686) 1. `run-peerdb.sh` should now support Podman too (needs to be tested) 2. implict network creation, check not needed 3. misc things to script --- dev-peerdb.sh | 36 +++++++++++++++--------------------- docker-compose-dev.yml | 1 - docker-compose.yml | 1 - run-peerdb.sh | 25 ++++++++++++++++--------- 4 files changed, 31 insertions(+), 32 deletions(-) diff --git a/dev-peerdb.sh b/dev-peerdb.sh index 7c32151845..fb5da35f1b 100755 --- a/dev-peerdb.sh +++ b/dev-peerdb.sh @@ -1,33 +1,27 @@ #!/bin/sh -if test -z "$USE_PODMAN" +set -Eeu + +DOCKER="docker" +EXTRA_ARGS="--no-attach temporal --no-attach pyroscope --no-attach temporal-ui" + +if test -n "${USE_PODMAN:=}" then - if ! command -v docker &> /dev/null - then - if command -v podman-compose - then - echo "docker could not be found on PATH, using podman-compose" + # 0 is found, checking for not found so we check for podman then + if $(docker compose &>/dev/null) && [ $? -ne 0 ]; then + if $(podman compose &>/dev/null) && [ $? -eq 0 ]; then + echo "docker could not be found on PATH, using podman compose" USE_PODMAN=1 else - echo "docker could not be found on PATH" + echo "docker compose could not be found on PATH" exit 1 fi fi fi -if test -z "$USE_PODMAN" -then - DOCKER="docker compose" - EXTRA_ARGS="--no-attach temporal --no-attach pyroscope --no-attach temporal-ui" -else - DOCKER="podman-compose --podman-run-args=--replace" - EXTRA_ARGS="" -fi - -# check if peerdb_network exists if not create it -if ! $DOCKER network inspect peerdb_network &> /dev/null -then - $DOCKER network create peerdb_network +if test -n "$USE_PODMAN"; then + DOCKER="podman" + EXTRA_ARGS="--podman-run-args=--replace" fi export PEERDB_VERSION_SHA_SHORT=local-$(git rev-parse --short HEAD) -exec $DOCKER -f docker-compose-dev.yml up --build $EXTRA_ARGS +exec $DOCKER compose -f docker-compose-dev.yml up --build $EXTRA_ARGS diff --git a/docker-compose-dev.yml b/docker-compose-dev.yml index d7d93f6de9..1868c755bf 100644 --- a/docker-compose-dev.yml +++ b/docker-compose-dev.yml @@ -221,5 +221,4 @@ volumes: networks: default: - external: true name: peerdb_network diff --git a/docker-compose.yml b/docker-compose.yml index 42fa2f26fc..fb144173aa 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -192,5 +192,4 @@ volumes: networks: default: - external: true name: peerdb_network diff --git a/run-peerdb.sh b/run-peerdb.sh index 002bf93679..91bce75a02 100755 --- a/run-peerdb.sh +++ b/run-peerdb.sh @@ -1,17 +1,24 @@ #!/bin/sh set -Eeu -if ! command -v docker &> /dev/null +DOCKER="docker" + +if test -n "${USE_PODMAN:=}" then - echo "docker could not be found on PATH" - exit 1 + if ! (command -v docker &> /dev/null); then + if (command -v podman &> /dev/null); then + echo "docker could not be found on PATH, using podman" + USE_PODMAN=1 + else + echo "docker could not be found on PATH" + exit 1 + fi + fi fi -# check if peerdb_network exists if not create it -if ! docker network inspect peerdb_network &> /dev/null -then - docker network create peerdb_network +if test -n "$USE_PODMAN"; then + DOCKER="podman" fi -docker compose pull -docker compose -f docker-compose.yml up --no-attach catalog --no-attach temporal --no-attach temporal-ui --no-attach temporal-admin-tools +$DOCKER compose pull +exec $DOCKER compose -f docker-compose.yml up --no-attach catalog --no-attach temporal --no-attach temporal-ui --no-attach temporal-admin-tools From 829e7538c79c9e8aeb81dc8c7a07130cddeca0fd Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Wed, 8 May 2024 17:50:23 +0530 Subject: [PATCH 31/41] UI: Use redpanda logo for peer component's kafka (#1690) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Uses redpanda logo for peer component Screenshot 2024-05-08 at 2 16 56 PM Screenshot 2024-05-08 at 2 17 09 PM --- ui/components/PeerComponent.tsx | 2 +- ui/public/svgs/redpanda.svg | 31 +++++++++++++++++++++++++++++++ 2 files changed, 32 insertions(+), 1 deletion(-) create mode 100644 ui/public/svgs/redpanda.svg diff --git a/ui/components/PeerComponent.tsx b/ui/components/PeerComponent.tsx index 6527a20247..bedbf41531 100644 --- a/ui/components/PeerComponent.tsx +++ b/ui/components/PeerComponent.tsx @@ -37,7 +37,7 @@ export const DBTypeToImageMapping = (peerType: DBType | string) => { return '/svgs/ms.svg'; case DBType.KAFKA: case 'KAFKA': - return '/svgs/kafka.svg'; + return '/svgs/redpanda.svg'; case DBType.PUBSUB: case 'PUBSUB': return '/svgs/pubsub.svg'; diff --git a/ui/public/svgs/redpanda.svg b/ui/public/svgs/redpanda.svg new file mode 100644 index 0000000000..ba9d666066 --- /dev/null +++ b/ui/public/svgs/redpanda.svg @@ -0,0 +1,31 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + From d9bdb107a7d13da1474e7d05c63495ed4af1d1f3 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Wed, 8 May 2024 19:04:51 +0530 Subject: [PATCH 32/41] CH UI: Help for user bucket setup (#1691) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - Adds placeholder for s3 path field - Link to doc for setting up s3 bucket Screenshot 2024-05-08 at 6 20 46 PM --- ui/app/peers/create/[peerType]/helpers/ch.ts | 1 + ui/components/PeerForms/ClickhouseConfig.tsx | 10 ++++++++++ 2 files changed, 11 insertions(+) diff --git a/ui/app/peers/create/[peerType]/helpers/ch.ts b/ui/app/peers/create/[peerType]/helpers/ch.ts index d45029a4e3..09626d73c0 100644 --- a/ui/app/peers/create/[peerType]/helpers/ch.ts +++ b/ui/app/peers/create/[peerType]/helpers/ch.ts @@ -52,6 +52,7 @@ export const clickhouseSetting: PeerSetting[] = [ stateHandler: (value, setter) => setter((curr) => ({ ...curr, s3Path: value as string })), tips: `This is an S3 bucket/object URL field. This bucket will be used as our intermediate stage for CDC`, + placeholder: 's3://', }, { label: 'Access Key ID', diff --git a/ui/components/PeerForms/ClickhouseConfig.tsx b/ui/components/PeerForms/ClickhouseConfig.tsx index f2ea31eb0c..405dd9af77 100644 --- a/ui/components/PeerForms/ClickhouseConfig.tsx +++ b/ui/components/PeerForms/ClickhouseConfig.tsx @@ -8,6 +8,7 @@ import { RowWithSwitch, RowWithTextField } from '@/lib/Layout'; import { Switch } from '@/lib/Switch'; import { TextField } from '@/lib/TextField'; import { Tooltip } from '@/lib/Tooltip'; +import Link from 'next/link'; import { useState } from 'react'; import { InfoPopover } from '../InfoPopover'; interface ConfigProps { @@ -122,6 +123,14 @@ export default function ClickhouseForm({ settings, setter }: ConfigProps) {

If you want this stage to belong to you, you can configure a bucket below. +

+ + Setting up an S3 bucket +