From e910b2580280fcde2cc4f7a0314e1f1312fdb222 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Fri, 31 May 2024 19:58:52 +0530 Subject: [PATCH 1/4] Add heartbeat routine in ReplicateQRepPartitions (#1774) Add heartbeat routine in ReplicateQRepPartitions activity --- flow/activities/flowable.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index 901ab5016..844c60c08 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -448,6 +448,11 @@ func (a *FlowableActivity) ReplicateQRepPartitions(ctx context.Context, partitions *protos.QRepPartitionBatch, runUUID string, ) error { + shutdown := heartbeatRoutine(ctx, func() string { + return "replicating partitions for job" + }) + defer shutdown() + ctx = context.WithValue(ctx, shared.FlowNameKey, config.FlowJobName) logger := log.With(activity.GetLogger(ctx), slog.String(string(shared.FlowNameKey), config.FlowJobName)) err := monitoring.UpdateStartTimeForQRepRun(ctx, a.CatalogPool, runUUID) From 2446e8ed0483f1c346fb3bcc69efe476722074cd Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Fri, 31 May 2024 16:13:11 -0400 Subject: [PATCH 2/4] Add query tags for Snowflake connector to track most expensive queries (#1776) This commit adds query tags to the Snowflake connector for the most frequently executed and expensive queries that contribute significantly to the overall cost. The tags are added using the `withMirrorNameQueryTag` method, which sets a tag with the format `peerdb-mirror-` for each query. Query tags have been added to key methods: - `SyncRecords` - `NormalizeRecords` - `CreateRawTable` - `SyncFlowCleanup` - `SyncQRepRecords` - `SetupQRepMetadataTables` - `ConsolidateQRepPartitions` These methods cover core functionality of syncing, normalizing, and managing QRep tasks. The tags will help identify and track the most expensive queries in the Snowflake web interface or using the `QUERY_HISTORY` table. Some minor queries have been ignored to focus on the most impactful areas. The query tags will aid in monitoring, optimizing, and controlling costs associated with the Snowflake connector. --- flow/connectors/snowflake/qrep.go | 6 ++++++ flow/connectors/snowflake/snowflake.go | 10 ++++++++++ 2 files changed, 16 insertions(+) diff --git a/flow/connectors/snowflake/qrep.go b/flow/connectors/snowflake/qrep.go index daf3e330d..ee3d12c5c 100644 --- a/flow/connectors/snowflake/qrep.go +++ b/flow/connectors/snowflake/qrep.go @@ -30,6 +30,8 @@ func (c *SnowflakeConnector) SyncQRepRecords( partition *protos.QRepPartition, stream *model.QRecordStream, ) (int, error) { + ctx = c.withMirrorNameQueryTag(ctx, config.FlowJobName) + // Ensure the destination table is available. destTable := config.DestinationTableIdentifier flowLog := slog.Group("sync_metadata", @@ -71,6 +73,8 @@ func (c *SnowflakeConnector) getTableSchema(ctx context.Context, tableName strin } func (c *SnowflakeConnector) SetupQRepMetadataTables(ctx context.Context, config *protos.QRepConfig) error { + ctx = c.withMirrorNameQueryTag(ctx, config.FlowJobName) + var schemaExists sql.NullBool err := c.database.QueryRowContext(ctx, checkIfSchemaExistsSQL, c.rawSchema).Scan(&schemaExists) if err != nil { @@ -169,6 +173,8 @@ func (c *SnowflakeConnector) createExternalStage(ctx context.Context, stageName } func (c *SnowflakeConnector) ConsolidateQRepPartitions(ctx context.Context, config *protos.QRepConfig) error { + ctx = c.withMirrorNameQueryTag(ctx, config.FlowJobName) + destTable := config.DestinationTableIdentifier stageName := c.getStageNameForJob(config.FlowJobName) diff --git a/flow/connectors/snowflake/snowflake.go b/flow/connectors/snowflake/snowflake.go index 74ce0942e..a0887ebfd 100644 --- a/flow/connectors/snowflake/snowflake.go +++ b/flow/connectors/snowflake/snowflake.go @@ -405,7 +405,13 @@ func (c *SnowflakeConnector) ReplayTableSchemaDeltas( return nil } +func (c *SnowflakeConnector) withMirrorNameQueryTag(ctx context.Context, mirrorName string) context.Context { + return gosnowflake.WithQueryTag(ctx, "peerdb-mirror-"+mirrorName) +} + func (c *SnowflakeConnector) SyncRecords(ctx context.Context, req *model.SyncRecordsRequest[model.RecordItems]) (*model.SyncResponse, error) { + ctx = c.withMirrorNameQueryTag(ctx, req.FlowJobName) + rawTableIdentifier := getRawTableIdentifier(req.FlowJobName) c.logger.Info("pushing records to Snowflake table " + rawTableIdentifier) @@ -468,6 +474,7 @@ func (c *SnowflakeConnector) syncRecordsViaAvro( // NormalizeRecords normalizes raw table to destination table. func (c *SnowflakeConnector) NormalizeRecords(ctx context.Context, req *model.NormalizeRecordsRequest) (*model.NormalizeResponse, error) { + ctx = c.withMirrorNameQueryTag(ctx, req.FlowJobName) normBatchID, err := c.GetLastNormalizeBatchID(ctx, req.FlowJobName) if err != nil { return nil, err @@ -583,6 +590,8 @@ func (c *SnowflakeConnector) mergeTablesForBatch( } func (c *SnowflakeConnector) CreateRawTable(ctx context.Context, req *protos.CreateRawTableInput) (*protos.CreateRawTableOutput, error) { + ctx = c.withMirrorNameQueryTag(ctx, req.FlowJobName) + var schemaExists sql.NullBool err := c.database.QueryRowContext(ctx, checkIfSchemaExistsSQL, c.rawSchema).Scan(&schemaExists) if err != nil { @@ -625,6 +634,7 @@ func (c *SnowflakeConnector) CreateRawTable(ctx context.Context, req *protos.Cre } func (c *SnowflakeConnector) SyncFlowCleanup(ctx context.Context, jobName string) error { + ctx = c.withMirrorNameQueryTag(ctx, jobName) err := c.PostgresMetadata.SyncFlowCleanup(ctx, jobName) if err != nil { return fmt.Errorf("[snowflake drop mirror] unable to clear metadata for sync flow cleanup: %w", err) From 53cee5a0e961d588c9cc0621e3ac0ea25ac7d8a6 Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Sun, 2 Jun 2024 16:41:01 -0400 Subject: [PATCH 3/4] ClickHouse Enhancement: Faster replication slot flush (#1780) Move bulk of the operations to normalize, `syncRecords` will just push to S3, and free up PostgreSQL replication slot faster. --- flow/connectors/clickhouse/cdc.go | 25 +++--- flow/connectors/clickhouse/clickhouse.go | 2 + flow/connectors/clickhouse/normalize.go | 32 +++++++ flow/connectors/clickhouse/qrep_avro_sync.go | 6 +- flow/connectors/clickhouse/s3_stage.go | 83 +++++++++++++++++++ flow/connectors/utils/avro/avro_writer.go | 6 +- nexus/catalog/migrations/V27__ch_s3_stage.sql | 11 +++ .../V28__ch_s3_stage_unique_constraint.sql | 3 + 8 files changed, 149 insertions(+), 19 deletions(-) create mode 100644 flow/connectors/clickhouse/s3_stage.go create mode 100644 nexus/catalog/migrations/V27__ch_s3_stage.sql create mode 100644 nexus/catalog/migrations/V28__ch_s3_stage_unique_constraint.sql diff --git a/flow/connectors/clickhouse/cdc.go b/flow/connectors/clickhouse/cdc.go index 63fd7bd47..6bf0854b2 100644 --- a/flow/connectors/clickhouse/cdc.go +++ b/flow/connectors/clickhouse/cdc.go @@ -6,7 +6,6 @@ import ( "errors" "fmt" "log/slog" - "strings" _ "github.com/ClickHouse/clickhouse-go/v2" _ "github.com/ClickHouse/clickhouse-go/v2/lib/driver" @@ -66,10 +65,18 @@ func (c *ClickhouseConnector) CreateRawTable(ctx context.Context, req *protos.Cr }, nil } +func (c *ClickhouseConnector) avroSyncMethod(flowJobName string) *ClickhouseAvroSyncMethod { + qrepConfig := &protos.QRepConfig{ + StagingPath: c.credsProvider.BucketPath, + FlowJobName: flowJobName, + DestinationTableIdentifier: c.getRawTableName(flowJobName), + } + return NewClickhouseAvroSyncMethod(qrepConfig, c) +} + func (c *ClickhouseConnector) syncRecordsViaAvro( ctx context.Context, req *model.SyncRecordsRequest[model.RecordItems], - rawTableIdentifier string, syncBatchID int64, ) (*model.SyncResponse, error) { tableNameRowsMapping := utils.InitialiseTableRowsMap(req.TableMappings) @@ -79,13 +86,8 @@ func (c *ClickhouseConnector) syncRecordsViaAvro( return nil, fmt.Errorf("failed to convert records to raw table stream: %w", err) } - qrepConfig := &protos.QRepConfig{ - StagingPath: c.credsProvider.BucketPath, - FlowJobName: req.FlowJobName, - DestinationTableIdentifier: strings.ToLower(rawTableIdentifier), - } - avroSyncer := NewClickhouseAvroSyncMethod(qrepConfig, c) - numRecords, err := avroSyncer.SyncRecords(ctx, stream, req.FlowJobName) + avroSyncer := c.avroSyncMethod(req.FlowJobName) + numRecords, err := avroSyncer.SyncRecords(ctx, stream, req.FlowJobName, syncBatchID) if err != nil { return nil, err } @@ -105,10 +107,7 @@ func (c *ClickhouseConnector) syncRecordsViaAvro( } func (c *ClickhouseConnector) SyncRecords(ctx context.Context, req *model.SyncRecordsRequest[model.RecordItems]) (*model.SyncResponse, error) { - rawTableName := c.getRawTableName(req.FlowJobName) - c.logger.Info("pushing records to Clickhouse table " + rawTableName) - - res, err := c.syncRecordsViaAvro(ctx, req, rawTableName, req.SyncBatchID) + res, err := c.syncRecordsViaAvro(ctx, req, req.SyncBatchID) if err != nil { return nil, err } diff --git a/flow/connectors/clickhouse/clickhouse.go b/flow/connectors/clickhouse/clickhouse.go index 1e299c59d..18dc75e64 100644 --- a/flow/connectors/clickhouse/clickhouse.go +++ b/flow/connectors/clickhouse/clickhouse.go @@ -31,6 +31,7 @@ type ClickhouseConnector struct { logger log.Logger config *protos.ClickhouseConfig credsProvider *utils.ClickHouseS3Credentials + s3Stage *ClickHouseS3Stage } func ValidateS3(ctx context.Context, creds *utils.ClickHouseS3Credentials) error { @@ -160,6 +161,7 @@ func NewClickhouseConnector( config: config, logger: logger, credsProvider: &clickHouseS3CredentialsNew, + s3Stage: NewClickHouseS3Stage(), }, nil } diff --git a/flow/connectors/clickhouse/normalize.go b/flow/connectors/clickhouse/normalize.go index 0813e59fa..e57cd8f71 100644 --- a/flow/connectors/clickhouse/normalize.go +++ b/flow/connectors/clickhouse/normalize.go @@ -136,6 +136,11 @@ func (c *ClickhouseConnector) NormalizeRecords(ctx context.Context, req *model.N }, nil } + err = c.copyAvroStagesToDestination(ctx, req.FlowJobName, normBatchID, req.SyncBatchID) + if err != nil { + return nil, fmt.Errorf("failed to copy avro stages to destination: %w", err) + } + destinationTableNames, err := c.getDistinctTableNamesInBatch( ctx, req.FlowJobName, @@ -282,3 +287,30 @@ func (c *ClickhouseConnector) getDistinctTableNamesInBatch( return tableNames, nil } + +func (c *ClickhouseConnector) copyAvroStageToDestination(ctx context.Context, flowJobName string, syncBatchID int64) error { + avroSynvMethod := c.avroSyncMethod(flowJobName) + avroFile, err := c.s3Stage.GetAvroStage(ctx, flowJobName, syncBatchID) + if err != nil { + return fmt.Errorf("failed to get avro stage: %w", err) + } + defer avroFile.Cleanup() + + err = avroSynvMethod.CopyStageToDestination(ctx, avroFile) + if err != nil { + return fmt.Errorf("failed to copy stage to destination: %w", err) + } + return nil +} + +func (c *ClickhouseConnector) copyAvroStagesToDestination( + ctx context.Context, flowJobName string, normBatchID, syncBatchID int64, +) error { + for s := normBatchID + 1; s <= syncBatchID; s++ { + err := c.copyAvroStageToDestination(ctx, flowJobName, s) + if err != nil { + return fmt.Errorf("failed to copy avro stage to destination: %w", err) + } + } + return nil +} diff --git a/flow/connectors/clickhouse/qrep_avro_sync.go b/flow/connectors/clickhouse/qrep_avro_sync.go index 011fe912d..f230c2459 100644 --- a/flow/connectors/clickhouse/qrep_avro_sync.go +++ b/flow/connectors/clickhouse/qrep_avro_sync.go @@ -64,6 +64,7 @@ func (s *ClickhouseAvroSyncMethod) SyncRecords( ctx context.Context, stream *model.QRecordStream, flowJobName string, + syncBatchID int64, ) (int, error) { tableLog := slog.String("destinationTable", s.config.DestinationTableIdentifier) dstTableName := s.config.DestinationTableIdentifier @@ -82,11 +83,10 @@ func (s *ClickhouseAvroSyncMethod) SyncRecords( return 0, err } - defer avroFile.Cleanup() s.connector.logger.Info(fmt.Sprintf("written %d records to Avro file", avroFile.NumRecords), tableLog) - err = s.CopyStageToDestination(ctx, avroFile) + err = s.connector.s3Stage.SetAvroStage(ctx, flowJobName, syncBatchID, avroFile) if err != nil { - return 0, err + return 0, fmt.Errorf("failed to set avro stage: %w", err) } return avroFile.NumRecords, nil diff --git a/flow/connectors/clickhouse/s3_stage.go b/flow/connectors/clickhouse/s3_stage.go new file mode 100644 index 000000000..b4ca7d71c --- /dev/null +++ b/flow/connectors/clickhouse/s3_stage.go @@ -0,0 +1,83 @@ +package connclickhouse + +import ( + "context" + "encoding/json" + "fmt" + + "github.com/jackc/pgx/v5" + "github.com/jackc/pgx/v5/pgxpool" + + utils "github.com/PeerDB-io/peer-flow/connectors/utils/avro" + "github.com/PeerDB-io/peer-flow/peerdbenv" +) + +type ClickHouseS3Stage struct{} + +func NewClickHouseS3Stage() *ClickHouseS3Stage { + return &ClickHouseS3Stage{} +} + +func (c *ClickHouseS3Stage) SetAvroStage( + ctx context.Context, + flowJobName string, + syncBatchID int64, + avroFile *utils.AvroFile, +) error { + avroFileJSON, err := json.Marshal(avroFile) + if err != nil { + return fmt.Errorf("failed to marshal avro file: %w", err) + } + + conn, err := c.getConn(ctx) + if err != nil { + return fmt.Errorf("failed to get connection: %w", err) + } + + _, err = conn.Exec(ctx, ` + INSERT INTO ch_s3_stage (flow_job_name, sync_batch_id, avro_file) + VALUES ($1, $2, $3) + ON CONFLICT (flow_job_name, sync_batch_id) + DO UPDATE SET avro_file = $3, created_at = CURRENT_TIMESTAMP + `, flowJobName, syncBatchID, avroFileJSON) + if err != nil { + return fmt.Errorf("failed to set avro stage: %w", err) + } + + return nil +} + +func (c *ClickHouseS3Stage) GetAvroStage(ctx context.Context, flowJobName string, syncBatchID int64) (*utils.AvroFile, error) { + conn, err := c.getConn(ctx) + if err != nil { + return nil, fmt.Errorf("failed to get connection: %w", err) + } + + var avroFileJSON []byte + err = conn.QueryRow(ctx, ` + SELECT avro_file FROM ch_s3_stage + WHERE flow_job_name = $1 AND sync_batch_id = $2 + `, flowJobName, syncBatchID).Scan(&avroFileJSON) + if err != nil { + if err == pgx.ErrNoRows { + return nil, fmt.Errorf("no avro stage found for flow job %s and sync batch %d", flowJobName, syncBatchID) + } + return nil, fmt.Errorf("failed to get avro stage: %w", err) + } + + var avroFile utils.AvroFile + if err := json.Unmarshal(avroFileJSON, &avroFile); err != nil { + return nil, fmt.Errorf("failed to unmarshal avro file: %w", err) + } + + return &avroFile, nil +} + +func (c *ClickHouseS3Stage) getConn(ctx context.Context) (*pgxpool.Pool, error) { + conn, err := peerdbenv.GetCatalogConnectionPoolFromEnv(ctx) + if err != nil { + return nil, fmt.Errorf("unable to create catalog connection pool: %w", err) + } + + return conn, nil +} diff --git a/flow/connectors/utils/avro/avro_writer.go b/flow/connectors/utils/avro/avro_writer.go index ad2597478..5abe9657b 100644 --- a/flow/connectors/utils/avro/avro_writer.go +++ b/flow/connectors/utils/avro/avro_writer.go @@ -51,9 +51,9 @@ type peerDBOCFWriter struct { } type AvroFile struct { - FilePath string - StorageLocation AvroStorageLocation - NumRecords int + FilePath string `json:"filePath"` + StorageLocation AvroStorageLocation `json:"storageLocation"` + NumRecords int `json:"numRecords"` } func (l *AvroFile) Cleanup() { diff --git a/nexus/catalog/migrations/V27__ch_s3_stage.sql b/nexus/catalog/migrations/V27__ch_s3_stage.sql new file mode 100644 index 000000000..61225700b --- /dev/null +++ b/nexus/catalog/migrations/V27__ch_s3_stage.sql @@ -0,0 +1,11 @@ +CREATE TABLE IF NOT EXISTS ch_s3_stage( + id SERIAL PRIMARY KEY, + flow_job_name TEXT NOT NULL, + sync_batch_id BIGINT NOT NULL, + avro_file JSONB NOT NULL, + created_at TIMESTAMPTZ NOT NULL DEFAULT CURRENT_TIMESTAMP +); + +CREATE INDEX IF NOT EXISTS ch_s3_stage_flow_job_name_idx ON ch_s3_stage (flow_job_name); + +CREATE INDEX IF NOT EXISTS ch_s3_stage_sync_batch_id_idx ON ch_s3_stage (sync_batch_id); diff --git a/nexus/catalog/migrations/V28__ch_s3_stage_unique_constraint.sql b/nexus/catalog/migrations/V28__ch_s3_stage_unique_constraint.sql new file mode 100644 index 000000000..e75b35a94 --- /dev/null +++ b/nexus/catalog/migrations/V28__ch_s3_stage_unique_constraint.sql @@ -0,0 +1,3 @@ +ALTER TABLE ch_s3_stage ADD CONSTRAINT +ch_s3_stage_flow_job_name_sync_batch_id_key +UNIQUE (flow_job_name, sync_batch_id); From 018918c84268ac0b392d56b6478f93913da15d86 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Mon, 3 Jun 2024 00:24:38 +0000 Subject: [PATCH 4/4] chore: update dependencies (#1782) --- flow/go.mod | 22 +++++----- flow/go.sum | 48 +++++++++++----------- nexus/Cargo.lock | 58 +++++++++++++------------- ui/package-lock.json | 96 ++++++++++++++++++++++---------------------- ui/package.json | 8 ++-- 5 files changed, 116 insertions(+), 116 deletions(-) diff --git a/flow/go.mod b/flow/go.mod index f416be22a..c7109374a 100644 --- a/flow/go.mod +++ b/flow/go.mod @@ -10,7 +10,7 @@ require ( github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.5.2 github.com/Azure/azure-sdk-for-go/sdk/messaging/azeventhubs v1.2.1 github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/eventhub/armeventhub v1.2.0 - github.com/ClickHouse/clickhouse-go/v2 v2.24.0 + github.com/ClickHouse/clickhouse-go/v2 v2.25.0 github.com/PeerDB-io/glua64 v1.0.1 github.com/PeerDB-io/gluabit32 v1.0.2 github.com/PeerDB-io/gluaflatbuffers v1.0.1 @@ -38,11 +38,11 @@ require ( github.com/klauspost/compress v1.17.8 github.com/lib/pq v1.10.9 github.com/linkedin/goavro/v2 v2.13.0 - github.com/microsoft/go-mssqldb v1.7.1 + github.com/microsoft/go-mssqldb v1.7.2 github.com/orcaman/concurrent-map/v2 v2.0.1 github.com/shopspring/decimal v1.4.0 github.com/slack-go/slack v0.13.0 - github.com/snowflakedb/gosnowflake v1.10.0 + github.com/snowflakedb/gosnowflake v1.10.1 github.com/stretchr/testify v1.9.0 github.com/twmb/franz-go v1.17.0 github.com/twmb/franz-go/plugin/kslog v1.0.0 @@ -62,14 +62,14 @@ require ( 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.181.0 - google.golang.org/genproto/googleapis/api v0.0.0-20240521202816-d264139d666e + google.golang.org/api v0.182.0 + google.golang.org/genproto/googleapis/api v0.0.0-20240528184218-531527333157 google.golang.org/grpc v1.64.0 google.golang.org/protobuf v1.34.1 ) require ( - cloud.google.com/go/auth v0.4.2 // indirect + cloud.google.com/go/auth v0.5.1 // 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 @@ -85,7 +85,7 @@ require ( 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.2 // indirect + github.com/cockroachdb/errors v1.11.3 // indirect github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b // indirect github.com/cockroachdb/redact v1.1.5 // indirect github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06 // indirect @@ -110,7 +110,7 @@ require ( github.com/prometheus/client_golang v1.19.1 // indirect github.com/prometheus/client_model v0.6.1 // indirect github.com/prometheus/common v0.53.0 // indirect - github.com/prometheus/procfs v0.15.0 // indirect + github.com/prometheus/procfs v0.15.1 // indirect github.com/rogpeppe/go-internal v1.12.0 // indirect github.com/segmentio/asm v1.2.0 // indirect github.com/sirupsen/logrus v1.9.3 // indirect @@ -174,7 +174,7 @@ require ( github.com/xrash/smetrics v0.0.0-20240521201337-686a1a2994c1 // indirect github.com/zeebo/xxh3 v1.0.2 // indirect go.opencensus.io v0.24.0 // indirect - golang.org/x/exp v0.0.0-20240525044651-4c93da0ed11d + golang.org/x/exp v0.0.0-20240531132922-fd00a4e0eefc golang.org/x/net v0.25.0 // indirect golang.org/x/oauth2 v0.20.0 // indirect golang.org/x/sys v0.20.0 // indirect @@ -182,7 +182,7 @@ require ( golang.org/x/time v0.5.0 // indirect golang.org/x/tools v0.21.0 // indirect golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 // indirect - google.golang.org/genproto v0.0.0-20240521202816-d264139d666e // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240521202816-d264139d666e // indirect + google.golang.org/genproto v0.0.0-20240528184218-531527333157 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240528184218-531527333157 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/flow/go.sum b/flow/go.sum index f2854a87c..6ec593ec4 100644 --- a/flow/go.sum +++ b/flow/go.sum @@ -1,8 +1,8 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.114.0 h1:OIPFAdfrFDFO2ve2U7r/H5SwSbBzEdrBdE7xkgwc+kY= cloud.google.com/go v0.114.0/go.mod h1:ZV9La5YYxctro1HTPug5lXH/GefROyW8PPD4T8n9J8E= -cloud.google.com/go/auth v0.4.2 h1:sb0eyLkhRtpq5jA+a8KWw0W70YcdVca7KJ8TM0AFYDg= -cloud.google.com/go/auth v0.4.2/go.mod h1:Kqvlz1cf1sNA0D+sYJnkPQOP+JMHkuHeIgVmCRtZOLc= +cloud.google.com/go/auth v0.5.1 h1:0QNO7VThG54LUzKiQxv8C6x1YX7lUrzlAa1nVLF8CIw= +cloud.google.com/go/auth v0.5.1/go.mod h1:vbZT8GjzDf3AVqCcQmqeeM32U9HBFc32vVVAbwDsa6s= 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= @@ -13,8 +13,8 @@ cloud.google.com/go/datacatalog v1.20.1 h1:czcba5mxwRM5V//jSadyig0y+8aOHmN7gUl9G cloud.google.com/go/datacatalog v1.20.1/go.mod h1:Jzc2CoHudhuZhpv78UBAjMEg3w7I9jHA11SbRshWUjk= 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.17.0 h1:5sFDtixjw5CivNoiiCbtWTQmi7bav3oMO6NeNg2IunQ= -cloud.google.com/go/kms v1.17.0/go.mod h1:DCMnCF/apA6fZk5Cj4XsD979OyHAqFasPuA5Sd0kGlQ= +cloud.google.com/go/kms v1.17.1 h1:5k0wXqkxL+YcXd4viQzTqCgzzVKKxzgrK+rCZJytEQs= +cloud.google.com/go/kms v1.17.1/go.mod h1:DCMnCF/apA6fZk5Cj4XsD979OyHAqFasPuA5Sd0kGlQ= cloud.google.com/go/longrunning v0.5.7 h1:WLbHekDbjK1fVFD3ibpFFVoyizlLRl73I7YKuAKilhU= cloud.google.com/go/longrunning v0.5.7/go.mod h1:8GClkudohy1Fxm3owmBGid8W0pSgodEMwEAztp38Xng= cloud.google.com/go/pubsub v1.38.0 h1:J1OT7h51ifATIedjqk/uBNPh+1hkvUaH4VKbz4UuAsc= @@ -56,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.24.0 h1:L/n/pVVpk95KtkHOiKuSnO7cu2ckeW4gICbbOh5qs74= -github.com/ClickHouse/clickhouse-go/v2 v2.24.0/go.mod h1:iDTViXk2Fgvf1jn2dbJd1ys+fBkdD1UMRnXlwmhijhQ= +github.com/ClickHouse/clickhouse-go/v2 v2.25.0 h1:rKscwqgQHzWBTZySZDcHKxgs0Ad+xFULfZvo26W5UlY= +github.com/ClickHouse/clickhouse-go/v2 v2.25.0/go.mod h1:iDTViXk2Fgvf1jn2dbJd1ys+fBkdD1UMRnXlwmhijhQ= 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= @@ -136,8 +136,8 @@ github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDk github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= github.com/cockroachdb/datadriven v1.0.3-0.20230413201302-be42291fc80f h1:otljaYPt5hWxV3MUfO5dFPFiOXg9CyG5/kCfayTqsJ4= github.com/cockroachdb/datadriven v1.0.3-0.20230413201302-be42291fc80f/go.mod h1:a9RdTaap04u637JoCzcUoIcDmvwSUtcUFtT/C3kJlTU= -github.com/cockroachdb/errors v1.11.2 h1:M5ABT/WDVqXk0FJwkfw/W+8d7dX55482qrq9dowO8Ks= -github.com/cockroachdb/errors v1.11.2/go.mod h1:g3dHDvKosgdwac1Yz8UPKvmk2yxXvk9M2bBfnwc9Oaw= +github.com/cockroachdb/errors v1.11.3 h1:5bA+k2Y6r+oz/6Z/RFlNeVCesGARKuC6YymtcDrbC/I= +github.com/cockroachdb/errors v1.11.3/go.mod h1:m4UIW4CDjx+R5cybPsNrRbreomiFqt8o1h1wUVazSd8= github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b h1:r6VH0faHjZeQy818SGhaone5OnYfxFR/+AzdY3sf5aE= github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b/go.mod h1:Vz9DsVWQQhf3vs21MhPMZpMGSht7O/2vFW2xusFUVOs= github.com/cockroachdb/pebble v1.1.0 h1:pcFh8CdCIt2kmEpK0OIatq67Ln9uGDYY3d5XnE0LJG4= @@ -316,8 +316,8 @@ github.com/linkedin/goavro/v2 v2.13.0 h1:L8eI8GcuciwUkt41Ej62joSZS4kKaYIUdze+6fo 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= -github.com/microsoft/go-mssqldb v1.7.1/go.mod h1:kOvZKUdrhhFQmxLZqbwUV0rHkNkZpthMITIb2Ko1IoA= +github.com/microsoft/go-mssqldb v1.7.2 h1:CHkFJiObW7ItKTJfHo1QX7QBBD1iV+mn1eOyRP3b/PA= +github.com/microsoft/go-mssqldb v1.7.2/go.mod h1:kOvZKUdrhhFQmxLZqbwUV0rHkNkZpthMITIb2Ko1IoA= github.com/montanaflynn/stats v0.0.0-20171201202039-1bf9dbcd8cbe/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= github.com/mtibben/percent v0.2.1 h1:5gssi8Nqo8QU/r2pynCm+hBQHpkB/uNK7BJCFogWdzs= github.com/mtibben/percent v0.2.1/go.mod h1:KG9uO+SZkUp+VkRHsCdYQV3XSZrrSpR3O9ibNBTZrns= @@ -351,8 +351,8 @@ github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= github.com/prometheus/common v0.53.0 h1:U2pL9w9nmJwJDa4qqLQ3ZaePJ6ZTwt7cMD3AG3+aLCE= github.com/prometheus/common v0.53.0/go.mod h1:BrxBKv3FWBIGXw89Mg1AeBq7FSyRzXWI3l3e7W3RN5U= -github.com/prometheus/procfs v0.15.0 h1:A82kmvXJq2jTu5YUhSGNlYoxh85zLnKgPz4bMZgI5Ek= -github.com/prometheus/procfs v0.15.0/go.mod h1:Y0RJ/Y5g5wJpkTisOtqwDSo4HwhGmLB4VQSw2sQJLHk= +github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= +github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= github.com/robfig/cron v1.2.0 h1:ZjScXvvxeQ63Dbyxy76Fj3AT3Ut0aKsyd2/tl3DTMuQ= github.com/robfig/cron v1.2.0/go.mod h1:JGuDeoQd7Z6yL4zQhZ3OPEVHB7fL6Ka6skscFHfmt2k= github.com/rogpeppe/go-internal v1.9.0/go.mod h1:WtVeX8xhTBvf0smdhujwtBcq4Qrzq/fJaraNFVN+nFs= @@ -367,8 +367,8 @@ github.com/sirupsen/logrus v1.9.3 h1:dueUQJ1C2q9oE3F7wvmSGAaVtTmUizReu6fjN8uqzbQ github.com/sirupsen/logrus v1.9.3/go.mod h1:naHLuLoDiP4jHNo9R0sCBMtWGeIprob74mVsIT4qYEQ= github.com/slack-go/slack v0.13.0 h1:7my/pR2ubZJ9912p9FtvALYpbt0cQPAqkRy2jaSI1PQ= github.com/slack-go/slack v0.13.0/go.mod h1:hlGi5oXA+Gt+yWTPP0plCdRKmjsDxecdHxYQdlMQKOw= -github.com/snowflakedb/gosnowflake v1.10.0 h1:5hBGKa/jJEhciokzgJcz5xmLNlJ8oUm8vhfu5tg82tM= -github.com/snowflakedb/gosnowflake v1.10.0/go.mod h1:WC4eGUOH3K9w3pLsdwZsdawIwtWgse4kZPPqNG0Ky/k= +github.com/snowflakedb/gosnowflake v1.10.1 h1:VGeQxsQj5s3hP0cRmtNYozhUvs2Y7Reu5Pk5pKuRGpI= +github.com/snowflakedb/gosnowflake v1.10.1/go.mod h1:hvc58mU03qg78mSz5z17/qnzI56hOdYYK2txWbM0hN0= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= @@ -456,8 +456,8 @@ golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0 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-20240525044651-4c93da0ed11d h1:N0hmiNbwsSNwHBAvR3QB5w25pUwH4tK0Y/RltD1j1h4= -golang.org/x/exp v0.0.0-20240525044651-4c93da0ed11d/go.mod h1:XtvwrStGgqGPLc4cjQfWqZHG1YFdYs6swckp8vpsjnc= +golang.org/x/exp v0.0.0-20240531132922-fd00a4e0eefc h1:O9NuF4s+E/PvMIy+9IUZB9znFwUIXEWSstNjek6VpVg= +golang.org/x/exp v0.0.0-20240531132922-fd00a4e0eefc/go.mod h1:XtvwrStGgqGPLc4cjQfWqZHG1YFdYs6swckp8vpsjnc= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= @@ -538,20 +538,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.181.0 h1:rPdjwnWgiPPOJx3IcSAQ2III5aX5tCer6wMpa/xmZi4= -google.golang.org/api v0.181.0/go.mod h1:MnQ+M0CFsfUwA5beZ+g/vCBCPXvtmZwRz2qzZk8ih1k= +google.golang.org/api v0.182.0 h1:if5fPvudRQ78GeRx3RayIoiuV7modtErPIZC/T2bIvE= +google.golang.org/api v0.182.0/go.mod h1:cGhjy4caqA5yXRzEhkHI8Y9mfyC2VLTlER2l08xaqtM= 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-20240521202816-d264139d666e h1:axIBUGXSVho2zB+3tJj8l9Qvm/El5vVYPYqhGA5PmJM= -google.golang.org/genproto v0.0.0-20240521202816-d264139d666e/go.mod h1:gOvX/2dWTqh+u3+IHjFeCxinlz5AZ5qhOufbQPub/dE= -google.golang.org/genproto/googleapis/api v0.0.0-20240521202816-d264139d666e h1:SkdGTrROJl2jRGT/Fxv5QUf9jtdKCQh4KQJXbXVLAi0= -google.golang.org/genproto/googleapis/api v0.0.0-20240521202816-d264139d666e/go.mod h1:LweJcLbyVij6rCex8YunD8DYR5VDonap/jYl3ZRxcIU= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240521202816-d264139d666e h1:Elxv5MwEkCI9f5SkoL6afed6NTdxaGoAo39eANBwHL8= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240521202816-d264139d666e/go.mod h1:EfXuqaE1J41VCDicxHzUDm+8rk+7ZdXzHV0IhO/I6s0= +google.golang.org/genproto v0.0.0-20240528184218-531527333157 h1:u7WMYrIrVvs0TF5yaKwKNbcJyySYf+HAIFXxWltJOXE= +google.golang.org/genproto v0.0.0-20240528184218-531527333157/go.mod h1:ubQlAQnzejB8uZzszhrTCU2Fyp6Vi7ZE5nn0c3W8+qQ= +google.golang.org/genproto/googleapis/api v0.0.0-20240528184218-531527333157 h1:7whR9kGa5LUwFtpLm2ArCEejtnxlGeLbAyjFY8sGNFw= +google.golang.org/genproto/googleapis/api v0.0.0-20240528184218-531527333157/go.mod h1:99sLkeliLXfdj2J75X3Ho+rrVCaJze0uwN7zDDkjPVU= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240528184218-531527333157 h1:Zy9XzmMEflZ/MAaA7vNcoebnRAld7FsPW1EeBB7V0m8= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240528184218-531527333157/go.mod h1:EfXuqaE1J41VCDicxHzUDm+8rk+7ZdXzHV0IhO/I6s0= 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= diff --git a/nexus/Cargo.lock b/nexus/Cargo.lock index bc3e85c52..576c5c284 100644 --- a/nexus/Cargo.lock +++ b/nexus/Cargo.lock @@ -4,9 +4,9 @@ version = 3 [[package]] name = "addr2line" -version = "0.21.0" +version = "0.22.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8a30b2e23b9e17a9f90641c7ab1549cd9b44f296d3ccbf309d2863cfe398a0cb" +checksum = "6e4503c46a5c0c7844e948c9a4d6acd9f50cccb4de1c48eb9e291ea17470c678" dependencies = [ "gimli", ] @@ -170,9 +170,9 @@ checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" [[package]] name = "async-compression" -version = "0.4.10" +version = "0.4.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c90a406b4495d129f00461241616194cb8a032c8d1c53c657f0961d5f8e0498" +checksum = "cd066d0b4ef8ecb03a55319dc13aa6910616d0f44008a045bb1835af830abff5" dependencies = [ "flate2", "futures-core", @@ -278,9 +278,9 @@ dependencies = [ [[package]] name = "backtrace" -version = "0.3.71" +version = "0.3.72" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "26b05800d2e817c8b3b4b54abd461726265fa9789ae34330622f2db9ee696f9d" +checksum = "17c6a35df3749d2e8bb1b7b21a976d82b15548788d2735b9d82f329268f71a11" dependencies = [ "addr2line", "cc", @@ -383,9 +383,9 @@ dependencies = [ [[package]] name = "borsh" -version = "1.5.0" +version = "1.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dbe5b10e214954177fb1dc9fbd20a1a2608fe99e6c832033bdc7cea287a20d77" +checksum = "a6362ed55def622cddc70a4746a68554d7b687713770de539e59a739b249f8ed" dependencies = [ "borsh-derive", "cfg_aliases", @@ -393,9 +393,9 @@ dependencies = [ [[package]] name = "borsh-derive" -version = "1.5.0" +version = "1.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d7a8646f94ab393e43e8b35a2558b1624bed28b97ee09c5d15456e3c9463f46d" +checksum = "c3ef8005764f53cd4dca619f5bf64cafd4664dada50ece25e4d81de54c80cc0b" dependencies = [ "once_cell", "proc-macro-crate", @@ -548,9 +548,9 @@ checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" [[package]] name = "cfg_aliases" -version = "0.1.1" +version = "0.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fd16c4719339c4530435d38e511904438d07cce7950afa3718a84ac36c10e89e" +checksum = "613afe47fcd5fac7ccf1db93babcb082c5994d996f20b8b159f2ad1658eb5724" [[package]] name = "chrono" @@ -579,9 +579,9 @@ dependencies = [ [[package]] name = "clang-sys" -version = "1.8.0" +version = "1.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a483f3cbf7cec2e153d424d0e92329d816becc6421389bd494375c6065921b9b" +checksum = "0b023947811758c97c59bf9d1c188fd619ad4718dcaa767947df1cadb14f39f4" dependencies = [ "glob", "libc", @@ -1159,9 +1159,9 @@ dependencies = [ [[package]] name = "gimli" -version = "0.28.1" +version = "0.29.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4271d37baee1b8c7e4b708028c57d816cf9d2434acb33a549475f78c181f6253" +checksum = "40ecd4077b5ae9fd2e9e169b102c6c330d0605168eb0e8bf79952b256dbefffd" [[package]] name = "glob" @@ -1418,9 +1418,9 @@ dependencies = [ [[package]] name = "hyper-util" -version = "0.1.4" +version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3d8d52be92d09acc2e01dddb7fde3ad983fc6489c7db4837e605bc3fca4cb63e" +checksum = "7b875924a60b96e5d7b9ae7b066540b1dd1cbd90d1828f54c92e02a283351c56" dependencies = [ "bytes", "futures-channel", @@ -1914,9 +1914,9 @@ dependencies = [ [[package]] name = "object" -version = "0.32.2" +version = "0.35.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a6a622008b6e321afc04970976f62ee297fdbaa6f95318ca343e3eebb9648441" +checksum = "b8ec7ab813848ba4522158d5517a6093db1ded27575b070f4177b8d12b41db5e" dependencies = [ "memchr", ] @@ -2487,9 +2487,9 @@ dependencies = [ [[package]] name = "proc-macro2" -version = "1.0.84" +version = "1.0.85" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ec96c6a92621310b51366f1e28d05ef11489516e93be030060e5fc12024a49d6" +checksum = "22244ce15aa966053a896d1accb3a6e68469b97c7f33f284b99f0d576879fc23" dependencies = [ "unicode-ident", ] @@ -3507,9 +3507,9 @@ checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.37.0" +version = "1.38.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1adbebffeca75fcfd058afa480fb6c0b81e165a0323f9c9d39c9697e37c46787" +checksum = "ba4f4a02a7a80d6f274636f0aa95c7e383b912d41fe721a31f29e29698585a4a" dependencies = [ "backtrace", "bytes", @@ -3537,9 +3537,9 @@ dependencies = [ [[package]] name = "tokio-macros" -version = "2.2.0" +version = "2.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b8a1e28f2deaa14e508979454cb3a223b10b938b45af148bc0986de36f1923b" +checksum = "5f5ae998a069d4b5aba8ee9dad856af7d520c3699e6159b185c2acd48155d39a" dependencies = [ "proc-macro2", "quote", @@ -3674,7 +3674,7 @@ dependencies = [ "serde", "serde_spanned", "toml_datetime", - "winnow 0.6.8", + "winnow 0.6.9", ] [[package]] @@ -4334,9 +4334,9 @@ dependencies = [ [[package]] name = "winnow" -version = "0.6.8" +version = "0.6.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c3c52e9c97a68071b23e836c9380edae937f17b9c4667bd021973efc689f618d" +checksum = "86c949fede1d13936a99f14fafd3e76fd642b556dd2ce96287fbe2e0151bfac6" dependencies = [ "memchr", ] diff --git a/ui/package-lock.json b/ui/package-lock.json index 09adbf000..dab414350 100644 --- a/ui/package-lock.json +++ b/ui/package-lock.json @@ -27,13 +27,13 @@ "@radix-ui/react-toggle-group": "^1.0.4", "@radix-ui/react-tooltip": "^1.0.7", "@tremor/react": "^3.17.2", - "@types/node": "^20.12.12", + "@types/node": "^20.14.0", "@types/react": "^18.3.3", "@types/react-dom": "^18.3.0", "classnames": "^2.5.1", "long": "^5.2.3", - "lucide-react": "^0.379.0", - "material-symbols": "^0.18.0", + "lucide-react": "^0.383.0", + "material-symbols": "^0.19.0", "moment": "^2.30.1", "moment-timezone": "^0.5.45", "next": "^14.2.3", @@ -59,7 +59,7 @@ "gh-pages": "^6.1.1", "less": "^4.2.0", "postcss": "^8.4.38", - "prettier": "^3.2.5", + "prettier": "^3.3.0", "prettier-plugin-organize-imports": "^3.2.4", "prisma": "^5.14.0", "string-width": "^7.1.0", @@ -543,9 +543,9 @@ } }, "node_modules/@headlessui/tailwindcss": { - "version": "0.2.0", - "resolved": "https://registry.npmjs.org/@headlessui/tailwindcss/-/tailwindcss-0.2.0.tgz", - "integrity": "sha512-fpL830Fln1SykOCboExsWr3JIVeQKieLJ3XytLe/tt1A0XzqUthOftDmjcCYLW62w7mQI7wXcoPXr3tZ9QfGxw==", + "version": "0.2.1", + "resolved": "https://registry.npmjs.org/@headlessui/tailwindcss/-/tailwindcss-0.2.1.tgz", + "integrity": "sha512-2+5+NZ+RzMyrVeCZOxdbvkUSssSxGvcUxphkIfSVLpRiKsj+/63T2TOL9dBYMXVfj/CGr6hMxSRInzXv6YY7sA==", "license": "MIT", "engines": { "node": ">=10" @@ -2322,9 +2322,9 @@ "license": "MIT" }, "node_modules/@types/node": { - "version": "20.12.12", - "resolved": "https://registry.npmjs.org/@types/node/-/node-20.12.12.tgz", - "integrity": "sha512-eWLDGF/FOSPtAvEqeRAQ4C8LSA7M1I7i0ky1I8U7kD1J5ITyW3AsRhQrKVoWf5pFKZ2kILsEGJhsI9r93PYnOw==", + "version": "20.14.0", + "resolved": "https://registry.npmjs.org/@types/node/-/node-20.14.0.tgz", + "integrity": "sha512-5cHBxFGJx6L4s56Bubp4fglrEpmyJypsqI6RgzMfBHWUJQGWAAi8cWcgetEbZXHYXo9C2Fa4EEds/uSyS4cxmA==", "license": "MIT", "dependencies": { "undici-types": "~5.26.4" @@ -3334,9 +3334,9 @@ } }, "node_modules/caniuse-lite": { - "version": "1.0.30001623", - "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001623.tgz", - "integrity": "sha512-X/XhAVKlpIxWPpgRTnlgZssJrF0m6YtRA0QDWgsBNT12uZM6LPRydR7ip405Y3t1LamD8cP2TZFEDZFBf5ApcA==", + "version": "1.0.30001626", + "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001626.tgz", + "integrity": "sha512-JRW7kAH8PFJzoPCJhLSHgDgKg5348hsQ68aqb+slnzuB5QFERv846oA/mRChmlLAOdEDeOkRn3ynb1gSFnjt3w==", "funding": [ { "type": "opencollective", @@ -3407,9 +3407,9 @@ } }, "node_modules/chrome-trace-event": { - "version": "1.0.3", - "resolved": "https://registry.npmjs.org/chrome-trace-event/-/chrome-trace-event-1.0.3.tgz", - "integrity": "sha512-p3KULyQg4S7NIHixdwbGX+nFHkoBiA4YQmyWtjb8XngSKV124nJmRysgAeujbUVb15vh+RvFUfCPqU7rXk+hZg==", + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/chrome-trace-event/-/chrome-trace-event-1.0.4.tgz", + "integrity": "sha512-rNjApaLzuwaOTjCiT8lSDdGN1APCiqkChLMJxJPWLunPAt5fy8xgU9/jNOchV84wfIxrA0lRQB7oCT8jrn/wrQ==", "dev": true, "license": "MIT", "engines": { @@ -3853,9 +3853,9 @@ } }, "node_modules/debug": { - "version": "4.3.4", - "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", - "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "version": "4.3.5", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.5.tgz", + "integrity": "sha512-pt0bNEmneDIvdL1Xsd9oDQ/wrQRkXDT4AUWlNZNPKvW5x/jyO9VFXkJUP07vQ2upmw5PlaITaPKc31jK13V+jg==", "dev": true, "license": "MIT", "dependencies": { @@ -4000,9 +4000,9 @@ "license": "MIT" }, "node_modules/electron-to-chromium": { - "version": "1.4.783", - "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.783.tgz", - "integrity": "sha512-bT0jEz/Xz1fahQpbZ1D7LgmPYZ3iHVY39NcWWro1+hA2IvjiPeaXtfSqrQ+nXjApMvQRE2ASt1itSLRrebHMRQ==", + "version": "1.4.788", + "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.788.tgz", + "integrity": "sha512-ubp5+Ev/VV8KuRoWnfP2QF2Bg+O2ZFdb49DiiNbz2VmgkIqrnyYaqIOqj8A6K/3p1xV0QcU5hBQ1+BmB6ot1OA==", "dev": true, "license": "ISC" }, @@ -4519,30 +4519,30 @@ } }, "node_modules/eslint-plugin-react": { - "version": "7.34.1", - "resolved": "https://registry.npmjs.org/eslint-plugin-react/-/eslint-plugin-react-7.34.1.tgz", - "integrity": "sha512-N97CxlouPT1AHt8Jn0mhhN2RrADlUAsk1/atcT2KyA/l9Q/E6ll7OIGwNumFmWfZ9skV3XXccYS19h80rHtgkw==", + "version": "7.34.2", + "resolved": "https://registry.npmjs.org/eslint-plugin-react/-/eslint-plugin-react-7.34.2.tgz", + "integrity": "sha512-2HCmrU+/JNigDN6tg55cRDKCQWicYAPB38JGSFDQt95jDm8rrvSUo7YPkOIm5l6ts1j1zCvysNcasvfTMQzUOw==", "dev": true, "license": "MIT", "dependencies": { - "array-includes": "^3.1.7", - "array.prototype.findlast": "^1.2.4", + "array-includes": "^3.1.8", + "array.prototype.findlast": "^1.2.5", "array.prototype.flatmap": "^1.3.2", "array.prototype.toreversed": "^1.1.2", "array.prototype.tosorted": "^1.1.3", "doctrine": "^2.1.0", - "es-iterator-helpers": "^1.0.17", + "es-iterator-helpers": "^1.0.19", "estraverse": "^5.3.0", "jsx-ast-utils": "^2.4.1 || ^3.0.0", "minimatch": "^3.1.2", - "object.entries": "^1.1.7", - "object.fromentries": "^2.0.7", - "object.hasown": "^1.1.3", - "object.values": "^1.1.7", + "object.entries": "^1.1.8", + "object.fromentries": "^2.0.8", + "object.hasown": "^1.1.4", + "object.values": "^1.2.0", "prop-types": "^15.8.1", "resolve": "^2.0.0-next.5", "semver": "^6.3.1", - "string.prototype.matchall": "^4.0.10" + "string.prototype.matchall": "^4.0.11" }, "engines": { "node": ">=4" @@ -6315,9 +6315,9 @@ } }, "node_modules/lucide-react": { - "version": "0.379.0", - "resolved": "https://registry.npmjs.org/lucide-react/-/lucide-react-0.379.0.tgz", - "integrity": "sha512-KcdeVPqmhRldldAAgptb8FjIunM2x2Zy26ZBh1RsEUcdLIvsEmbcw7KpzFYUy5BbpGeWhPu9Z9J5YXfStiXwhg==", + "version": "0.383.0", + "resolved": "https://registry.npmjs.org/lucide-react/-/lucide-react-0.383.0.tgz", + "integrity": "sha512-13xlG0CQCJtzjSQYwwJ3WRqMHtRj3EXmLlorrARt7y+IHnxUCp3XyFNL1DfaGySWxHObDvnu1u1dV+0VMKHUSg==", "license": "ISC", "peerDependencies": { "react": "^16.5.1 || ^17.0.0 || ^18.0.0" @@ -6350,9 +6350,9 @@ } }, "node_modules/material-symbols": { - "version": "0.18.0", - "resolved": "https://registry.npmjs.org/material-symbols/-/material-symbols-0.18.0.tgz", - "integrity": "sha512-U+b96KpXsN+sA9Y3b/50t6Z1rJPfh2UvHCvwsZRJtB6IlbYMhZw6E6kvfelhlOhm32qX43SSlVIemqDFkPjh/Q==", + "version": "0.19.0", + "resolved": "https://registry.npmjs.org/material-symbols/-/material-symbols-0.19.0.tgz", + "integrity": "sha512-/iWffKJzk8QZw2Wg6QjSS8f4oRbnesbU1Eodje1+jzBCtNYJupJwArjnXpiFeWBANkW0gvOx6AdQpnj9wsgAmA==", "license": "Apache-2.0" }, "node_modules/memoize-one": { @@ -7292,9 +7292,9 @@ } }, "node_modules/postcss-load-config/node_modules/yaml": { - "version": "2.4.2", - "resolved": "https://registry.npmjs.org/yaml/-/yaml-2.4.2.tgz", - "integrity": "sha512-B3VqDZ+JAg1nZpaEmWtTXUlBneoGx6CPM9b0TENK6aoSu5t73dItudwdgmi6tHlIZZId4dZ9skcAQ2UbcyAeVA==", + "version": "2.4.3", + "resolved": "https://registry.npmjs.org/yaml/-/yaml-2.4.3.tgz", + "integrity": "sha512-sntgmxj8o7DE7g/Qi60cqpLBA3HG3STcDA0kO+WfB05jEKhZMbY7umNm2rBpQvsmZ16/lPXCJGW2672dgOUkrg==", "license": "ISC", "bin": { "yaml": "bin.mjs" @@ -7374,9 +7374,9 @@ } }, "node_modules/prettier": { - "version": "3.2.5", - "resolved": "https://registry.npmjs.org/prettier/-/prettier-3.2.5.tgz", - "integrity": "sha512-3/GWa9aOC0YeD7LUfvOG2NiDyhOWRvt1k+rcKhOuYnMY24iiCphgneUfJDyFXd6rZCAnuLBv6UeAULtrhT/F4A==", + "version": "3.3.0", + "resolved": "https://registry.npmjs.org/prettier/-/prettier-3.3.0.tgz", + "integrity": "sha512-J9odKxERhCQ10OC2yb93583f6UnYutOeiV5i0zEDS7UGTdUt0u+y8erxl3lBKvwo/JHyyoEdXjwp4dke9oyZ/g==", "dev": true, "license": "MIT", "bin": { @@ -8010,9 +8010,9 @@ "optional": true }, "node_modules/sax": { - "version": "1.3.0", - "resolved": "https://registry.npmjs.org/sax/-/sax-1.3.0.tgz", - "integrity": "sha512-0s+oAmw9zLl1V1cS9BtZN7JAd0cW5e0QH4W3LWEK6a4LaLEA2OTpGYWDY+6XasBLtz6wkm3u1xRw95mRuJ59WA==", + "version": "1.4.1", + "resolved": "https://registry.npmjs.org/sax/-/sax-1.4.1.tgz", + "integrity": "sha512-+aWOz7yVScEGoKNd4PA10LZ8sk0A/z5+nXQG5giUO5rprX9jgYsTdov9qCchZiPIZezbZH+jRut8nPodFAX4Jg==", "dev": true, "license": "ISC", "optional": true diff --git a/ui/package.json b/ui/package.json index 7b85a1597..ed76a188a 100644 --- a/ui/package.json +++ b/ui/package.json @@ -29,13 +29,13 @@ "@radix-ui/react-toggle-group": "^1.0.4", "@radix-ui/react-tooltip": "^1.0.7", "@tremor/react": "^3.17.2", - "@types/node": "^20.12.12", + "@types/node": "^20.14.0", "@types/react": "^18.3.3", "@types/react-dom": "^18.3.0", "classnames": "^2.5.1", "long": "^5.2.3", - "lucide-react": "^0.379.0", - "material-symbols": "^0.18.0", + "lucide-react": "^0.383.0", + "material-symbols": "^0.19.0", "moment": "^2.30.1", "moment-timezone": "^0.5.45", "next": "^14.2.3", @@ -61,7 +61,7 @@ "gh-pages": "^6.1.1", "less": "^4.2.0", "postcss": "^8.4.38", - "prettier": "^3.2.5", + "prettier": "^3.3.0", "prettier-plugin-organize-imports": "^3.2.4", "prisma": "^5.14.0", "string-width": "^7.1.0",