From e8ec146bc475aa3534b0c4a66999685983db60d1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Mon, 30 Sep 2024 19:33:18 +0000 Subject: [PATCH 01/13] remove clickhouse qrep metadata logic (#2115) external catalog handles this --- flow/activities/flowable.go | 3 +- flow/connectors/clickhouse/qrep.go | 82 +------------------- flow/connectors/clickhouse/qrep_avro_sync.go | 28 ------- 3 files changed, 4 insertions(+), 109 deletions(-) diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index b8c0480df9..65502da0bf 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -596,8 +596,7 @@ func (a *FlowableActivity) ConsolidateQRepPartitions(ctx context.Context, config }) defer shutdown() - err = dstConn.ConsolidateQRepPartitions(ctx, config) - if err != nil { + if err := dstConn.ConsolidateQRepPartitions(ctx, config); err != nil { a.Alerter.LogFlowError(ctx, config.FlowJobName, err) return err } diff --git a/flow/connectors/clickhouse/qrep.go b/flow/connectors/clickhouse/qrep.go index f6e25becad..ad6c9d6e01 100644 --- a/flow/connectors/clickhouse/qrep.go +++ b/flow/connectors/clickhouse/qrep.go @@ -5,12 +5,10 @@ import ( "fmt" "log/slog" "strings" - "time" "github.com/ClickHouse/clickhouse-go/v2/lib/driver" "github.com/aws/aws-sdk-go-v2/aws" "github.com/aws/aws-sdk-go-v2/service/s3" - "google.golang.org/protobuf/encoding/protojson" "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/generated/protos" @@ -18,7 +16,9 @@ import ( "github.com/PeerDB-io/peer-flow/shared" ) -const qRepMetadataTableName = "_peerdb_query_replication_metadata" +func (*ClickHouseConnector) SetupQRepMetadataTables(_ context.Context, _ *protos.QRepConfig) error { + return nil +} func (c *ClickHouseConnector) SyncQRepRecords( ctx context.Context, @@ -44,30 +44,6 @@ func (c *ClickHouseConnector) SyncQRepRecords( return avroSync.SyncQRepRecords(ctx, config, partition, tblSchema, stream) } -func (c *ClickHouseConnector) createMetadataInsertStatement( - partition *protos.QRepPartition, - jobName string, - startTime time.Time, -) (string, error) { - // marshal the partition to json using protojson - pbytes, err := protojson.Marshal(partition) - if err != nil { - return "", fmt.Errorf("failed to marshal partition to json: %v", err) - } - - // convert the bytes to string - partitionJSON := string(pbytes) - - insertMetadataStmt := fmt.Sprintf( - `INSERT INTO %s - (flowJobName, partitionID, syncPartition, syncStartTime, syncFinishTime) - VALUES ('%s', '%s', '%s', '%s', NOW());`, - qRepMetadataTableName, jobName, partition.PartitionId, - partitionJSON, startTime.Format("2006-01-02 15:04:05.000000")) - - return insertMetadataStmt, nil -} - func (c *ClickHouseConnector) getTableSchema(ctx context.Context, tableName string) ([]driver.ColumnType, error) { queryString := fmt.Sprintf(`SELECT * FROM %s LIMIT 0`, tableName) rows, err := c.database.Query(ctx, queryString) @@ -79,58 +55,6 @@ func (c *ClickHouseConnector) getTableSchema(ctx context.Context, tableName stri return rows.ColumnTypes(), nil } -func (c *ClickHouseConnector) IsQRepPartitionSynced(ctx context.Context, - req *protos.IsQRepPartitionSyncedInput, -) (bool, error) { - queryString := fmt.Sprintf(`SELECT COUNT(*) FROM %s WHERE partitionID = '%s'`, qRepMetadataTableName, req.PartitionId) - - var count uint64 - if err := c.database.QueryRow(ctx, queryString).Scan(&count); err != nil { - return false, fmt.Errorf("failed to execute query: %w", err) - } - return count > 0, nil -} - -func (c *ClickHouseConnector) SetupQRepMetadataTables(ctx context.Context, config *protos.QRepConfig) error { - err := c.createQRepMetadataTable(ctx) - if err != nil { - return err - } - - if config.WriteMode.WriteType == protos.QRepWriteType_QREP_WRITE_MODE_OVERWRITE { - err = c.execWithLogging(ctx, "TRUNCATE TABLE "+config.DestinationTableIdentifier) - if err != nil { - return fmt.Errorf("failed to TRUNCATE table before query replication: %w", err) - } - } - - return nil -} - -func (c *ClickHouseConnector) createQRepMetadataTable(ctx context.Context) error { - // Define the schema - schemaStatement := ` - CREATE TABLE IF NOT EXISTS %s ( - flowJobName String, - partitionID String, - syncPartition String, - syncStartTime DateTime64, - syncFinishTime DateTime64 - ) ENGINE = MergeTree() - ORDER BY partitionID; - ` - queryString := fmt.Sprintf(schemaStatement, qRepMetadataTableName) - err := c.execWithLogging(ctx, queryString) - if err != nil { - c.logger.Error("failed to create table "+qRepMetadataTableName, - slog.Any("error", err)) - - return fmt.Errorf("failed to create table %s: %w", qRepMetadataTableName, err) - } - c.logger.Info("Created table " + qRepMetadataTableName) - return nil -} - func (c *ClickHouseConnector) ConsolidateQRepPartitions(_ context.Context, config *protos.QRepConfig) error { c.logger.Info("Consolidating partitions noop") return nil diff --git a/flow/connectors/clickhouse/qrep_avro_sync.go b/flow/connectors/clickhouse/qrep_avro_sync.go index c408f3eff6..b507f448e5 100644 --- a/flow/connectors/clickhouse/qrep_avro_sync.go +++ b/flow/connectors/clickhouse/qrep_avro_sync.go @@ -5,7 +5,6 @@ import ( "fmt" "log/slog" "strings" - "time" "github.com/ClickHouse/clickhouse-go/v2/lib/driver" @@ -102,7 +101,6 @@ func (s *ClickHouseAvroSyncMethod) SyncQRepRecords( dstTableSchema []driver.ColumnType, stream *model.QRecordStream, ) (int, error) { - startTime := time.Now() dstTableName := config.DestinationTableIdentifier stagingPath := s.connector.credsProvider.BucketPath @@ -156,11 +154,6 @@ func (s *ClickHouseAvroSyncMethod) SyncQRepRecords( return 0, err } - err = s.insertMetadata(ctx, partition, config.FlowJobName, startTime) - if err != nil { - return -1, err - } - return avroFile.NumRecords, nil } @@ -198,24 +191,3 @@ func (s *ClickHouseAvroSyncMethod) writeToAvroFile( return avroFile, nil } - -func (s *ClickHouseAvroSyncMethod) insertMetadata( - ctx context.Context, - partition *protos.QRepPartition, - flowJobName string, - startTime time.Time, -) error { - partitionLog := slog.String(string(shared.PartitionIDKey), partition.PartitionId) - insertMetadataStmt, err := s.connector.createMetadataInsertStatement(partition, flowJobName, startTime) - if err != nil { - s.connector.logger.Error("failed to create metadata insert statement", - slog.Any("error", err), partitionLog) - return fmt.Errorf("failed to create metadata insert statement: %w", err) - } - - if err := s.connector.database.Exec(ctx, insertMetadataStmt); err != nil { - return fmt.Errorf("failed to execute metadata insert statement: %w", err) - } - - return nil -} From 046f8c96212610febe14912144a916a0ae9dc140 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Tue, 1 Oct 2024 13:50:01 +0000 Subject: [PATCH 02/13] clickhouse: fix normalizing updates to primary key (#2113) when primary key updated row becomes duplicate because old version isn't marked outdated, generate deletion records behind updates disabled by default due to perf impact, can be enabled with `PEERDB_CLICKHOUSE_ENABLE_PRIMARY_UPDATE` if custom ordering is on a non replica identity column then row data may be missing, causing this to fail to prevent duplication --- flow/connectors/clickhouse/normalize.go | 62 ++++++++++++++-- flow/e2e/clickhouse/clickhouse.go | 2 +- flow/e2e/clickhouse/peer_flow_ch_test.go | 92 ++++++++++++++++++++++++ flow/peerdbenv/dynamicconf.go | 80 ++++++++++++++++----- 4 files changed, 212 insertions(+), 24 deletions(-) diff --git a/flow/connectors/clickhouse/normalize.go b/flow/connectors/clickhouse/normalize.go index 588d8f77f3..f69904cb05 100644 --- a/flow/connectors/clickhouse/normalize.go +++ b/flow/connectors/clickhouse/normalize.go @@ -15,6 +15,7 @@ import ( "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" + "github.com/PeerDB-io/peer-flow/peerdbenv" ) const ( @@ -182,7 +183,7 @@ func generateCreateTableSQLForNormalizedTable( stmtBuilder.WriteString(") ") } - orderby := make([]*protos.ColumnSetting, 0) + var orderby []*protos.ColumnSetting if tableMapping != nil { orderby = slices.Clone(tableMapping.Columns) for _, col := range tableMapping.Columns { @@ -190,10 +191,13 @@ func generateCreateTableSQLForNormalizedTable( orderby = append(orderby, col) } } + + if len(orderby) > 0 { + slices.SortStableFunc(orderby, func(a *protos.ColumnSetting, b *protos.ColumnSetting) int { + return cmp.Compare(a.Ordering, b.Ordering) + }) + } } - slices.SortStableFunc(orderby, func(a *protos.ColumnSetting, b *protos.ColumnSetting) int { - return cmp.Compare(a.Ordering, b.Ordering) - }) if pkeyStr != "" || len(orderby) > 0 { stmtBuilder.WriteString("ORDER BY (") @@ -262,7 +266,7 @@ func (c *ClickHouseConnector) NormalizeRecords( selectQuery.WriteString("SELECT ") colSelector := strings.Builder{} - colSelector.WriteString("(") + colSelector.WriteRune('(') schema := req.TableNameSchemaMapping[tbl] @@ -274,7 +278,13 @@ func (c *ClickHouseConnector) NormalizeRecords( } } + enablePrimaryUpdate, err := peerdbenv.PeerDBEnableClickHousePrimaryUpdate(ctx, req.Env) + if err != nil { + return nil, err + } + projection := strings.Builder{} + projectionUpdate := strings.Builder{} for _, column := range schema.Columns { colName := column.Name @@ -321,14 +331,36 @@ func (c *ClickHouseConnector) NormalizeRecords( colName, dstColName, )) + if enablePrimaryUpdate { + projectionUpdate.WriteString(fmt.Sprintf( + "toDate32(parseDateTime64BestEffortOrNull(JSONExtractString(_peerdb_match_data, '%s'))) AS `%s`,", + colName, + dstColName, + )) + } case "DateTime64(6)", "Nullable(DateTime64(6))": projection.WriteString(fmt.Sprintf( "parseDateTime64BestEffortOrNull(JSONExtractString(_peerdb_data, '%s')) AS `%s`,", colName, dstColName, )) + if enablePrimaryUpdate { + projectionUpdate.WriteString(fmt.Sprintf( + "parseDateTime64BestEffortOrNull(JSONExtractString(_peerdb_match_data, '%s')) AS `%s`,", + colName, + dstColName, + )) + } default: projection.WriteString(fmt.Sprintf("JSONExtract(_peerdb_data, '%s', '%s') AS `%s`,", colName, clickHouseType, dstColName)) + if enablePrimaryUpdate { + projectionUpdate.WriteString(fmt.Sprintf( + "JSONExtract(_peerdb_match_data, '%s', '%s') AS `%s`,", + colName, + clickHouseType, + dstColName, + )) + } } } @@ -352,6 +384,26 @@ func (c *ClickHouseConnector) NormalizeRecords( selectQuery.WriteString(tbl) selectQuery.WriteString("'") + if enablePrimaryUpdate { + // projectionUpdate generates delete on previous record, so _peerdb_record_type is filled in as 2 + projectionUpdate.WriteString(fmt.Sprintf("1 AS `%s`,", signColName)) + // decrement timestamp by 1 so delete is ordered before latest data, + // could be same if deletion records were only generated when ordering updated + projectionUpdate.WriteString(fmt.Sprintf("_peerdb_timestamp - 1 AS `%s`", versionColName)) + + selectQuery.WriteString("UNION ALL SELECT ") + selectQuery.WriteString(projectionUpdate.String()) + selectQuery.WriteString(" FROM ") + selectQuery.WriteString(rawTbl) + selectQuery.WriteString(" WHERE _peerdb_batch_id > ") + selectQuery.WriteString(strconv.FormatInt(normBatchID, 10)) + selectQuery.WriteString(" AND _peerdb_batch_id <= ") + selectQuery.WriteString(strconv.FormatInt(req.SyncBatchID, 10)) + selectQuery.WriteString(" AND _peerdb_destination_table_name = '") + selectQuery.WriteString(tbl) + selectQuery.WriteString("' AND _peerdb_record_type = 1") + } + insertIntoSelectQuery := strings.Builder{} insertIntoSelectQuery.WriteString("INSERT INTO ") insertIntoSelectQuery.WriteString(tbl) diff --git a/flow/e2e/clickhouse/clickhouse.go b/flow/e2e/clickhouse/clickhouse.go index 9be7b5102d..891fe55365 100644 --- a/flow/e2e/clickhouse/clickhouse.go +++ b/flow/e2e/clickhouse/clickhouse.go @@ -99,7 +99,7 @@ func (s ClickHouseSuite) GetRows(table string, cols string) (*model.QRecordBatch } rows, err := ch.Query( context.Background(), - fmt.Sprintf(`SELECT %s FROM %s FINAL ORDER BY %s SETTINGS use_query_cache = false`, cols, table, firstCol), + fmt.Sprintf(`SELECT %s FROM %s FINAL WHERE _peerdb_is_deleted = 0 ORDER BY %s SETTINGS use_query_cache = false`, cols, table, firstCol), ) if err != nil { return nil, err diff --git a/flow/e2e/clickhouse/peer_flow_ch_test.go b/flow/e2e/clickhouse/peer_flow_ch_test.go index 590ed4f9e1..d2b6619513 100644 --- a/flow/e2e/clickhouse/peer_flow_ch_test.go +++ b/flow/e2e/clickhouse/peer_flow_ch_test.go @@ -263,3 +263,95 @@ func (s ClickHouseSuite) Test_Date32() { env.Cancel() e2e.RequireEnvCanceled(s.t, env) } + +func (s ClickHouseSuite) Test_Update_PKey_Env_Disabled() { + srcTableName := "test_update_pkey_disabled" + srcFullName := s.attachSchemaSuffix("test_update_pkey_disabled") + dstTableName := "test_update_pkey_disabled_dst" + + _, err := s.Conn().Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id INT PRIMARY KEY, + key TEXT NOT NULL + ); + `, srcFullName)) + require.NoError(s.t, err) + + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s (id,key) VALUES (1,'init'); + `, srcFullName)) + require.NoError(s.t, err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("clickhouse_pkey_update_disabled"), + TableNameMapping: map[string]string{srcFullName: dstTableName}, + Destination: s.Peer().Name, + } + flowConnConfig := connectionGen.GenerateFlowConnectionConfigs(s.t) + flowConnConfig.DoInitialSnapshot = true + flowConnConfig.Env = map[string]string{"PEERDB_CLICKHOUSE_ENABLE_PRIMARY_UPDATE": "false"} + + tc := e2e.NewTemporalClient(s.t) + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, flowConnConfig) + + e2e.EnvWaitForEqualTablesWithNames(env, s, "waiting on initial", srcTableName, dstTableName, "id,key") + + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + UPDATE %s SET id = 2, key = 'update' WHERE id = 1; + `, srcFullName)) + require.NoError(s.t, err) + + e2e.EnvWaitFor(s.t, env, time.Minute, "waiting for duplicate row", func() bool { + rows, err := s.GetRows(dstTableName, "id") + require.NoError(s.t, err) + return len(rows.Records) == 2 + }) + + env.Cancel() + e2e.RequireEnvCanceled(s.t, env) +} + +func (s ClickHouseSuite) Test_Update_PKey_Env_Enabled() { + srcTableName := "test_update_pkey_enabled" + srcFullName := s.attachSchemaSuffix("test_update_pkey_enabled") + dstTableName := "test_update_pkey_enabled_dst" + + _, err := s.Conn().Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id INT PRIMARY KEY, + key TEXT NOT NULL + ); + `, srcFullName)) + require.NoError(s.t, err) + + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s (id,key) VALUES (1,'init'); + `, srcFullName)) + require.NoError(s.t, err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("clickhouse_pkey_update_enabled"), + TableNameMapping: map[string]string{srcFullName: dstTableName}, + Destination: s.Peer().Name, + } + flowConnConfig := connectionGen.GenerateFlowConnectionConfigs(s.t) + flowConnConfig.DoInitialSnapshot = true + flowConnConfig.Env = map[string]string{"PEERDB_CLICKHOUSE_ENABLE_PRIMARY_UPDATE": "true"} + + tc := e2e.NewTemporalClient(s.t) + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, flowConnConfig) + + e2e.EnvWaitForEqualTablesWithNames(env, s, "waiting on initial", srcTableName, dstTableName, "id,key") + + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + UPDATE %s SET id = 2, key = 'update' WHERE id = 1; + `, srcFullName)) + require.NoError(s.t, err) + + e2e.EnvWaitForEqualTablesWithNames(env, s, "waiting on cdc", srcTableName, dstTableName, "id,key") + + env.Cancel() + e2e.RequireEnvCanceled(s.t, env) +} diff --git a/flow/peerdbenv/dynamicconf.go b/flow/peerdbenv/dynamicconf.go index e026a65f30..64909c75e2 100644 --- a/flow/peerdbenv/dynamicconf.go +++ b/flow/peerdbenv/dynamicconf.go @@ -18,44 +18,58 @@ import ( var DynamicSettings = [...]*protos.DynamicSetting{ { - Name: "PEERDB_MAX_SYNCS_PER_CDC_FLOW", DefaultValue: "32", ValueType: protos.DynconfValueType_UINT, + Name: "PEERDB_MAX_SYNCS_PER_CDC_FLOW", Description: "Experimental setting: changes number of syncs per workflow, affects frequency of replication slot disconnects", + DefaultValue: "32", + ValueType: protos.DynconfValueType_UINT, ApplyMode: protos.DynconfApplyMode_APPLY_MODE_IMMEDIATE, TargetForSetting: protos.DynconfTarget_ALL, }, { - Name: "PEERDB_CDC_CHANNEL_BUFFER_SIZE", DefaultValue: "262144", ValueType: protos.DynconfValueType_INT, + Name: "PEERDB_CDC_CHANNEL_BUFFER_SIZE", Description: "Advanced setting: changes buffer size of channel PeerDB uses while streaming rows read to destination in CDC", + DefaultValue: "262144", + ValueType: protos.DynconfValueType_INT, ApplyMode: protos.DynconfApplyMode_APPLY_MODE_IMMEDIATE, TargetForSetting: protos.DynconfTarget_ALL, }, { - Name: "PEERDB_QUEUE_FLUSH_TIMEOUT_SECONDS", DefaultValue: "10", ValueType: protos.DynconfValueType_INT, + Name: "PEERDB_QUEUE_FLUSH_TIMEOUT_SECONDS", Description: "Frequency of flushing to queue, applicable for PeerDB Streams mirrors only", + DefaultValue: "10", + ValueType: protos.DynconfValueType_INT, ApplyMode: protos.DynconfApplyMode_APPLY_MODE_IMMEDIATE, TargetForSetting: protos.DynconfTarget_QUEUES, }, { - Name: "PEERDB_QUEUE_PARALLELISM", DefaultValue: "4", ValueType: protos.DynconfValueType_INT, + Name: "PEERDB_QUEUE_PARALLELISM", Description: "Parallelism for Lua script processing data, applicable for CDC mirrors to Kakfa and PubSub", + DefaultValue: "4", + ValueType: protos.DynconfValueType_INT, ApplyMode: protos.DynconfApplyMode_APPLY_MODE_IMMEDIATE, TargetForSetting: protos.DynconfTarget_QUEUES, }, { - Name: "PEERDB_CDC_DISK_SPILL_RECORDS_THRESHOLD", DefaultValue: "1000000", ValueType: protos.DynconfValueType_INT, + Name: "PEERDB_CDC_DISK_SPILL_RECORDS_THRESHOLD", Description: "CDC: number of records beyond which records are written to disk instead", + DefaultValue: "1000000", + ValueType: protos.DynconfValueType_INT, ApplyMode: protos.DynconfApplyMode_APPLY_MODE_IMMEDIATE, TargetForSetting: protos.DynconfTarget_ALL, }, { - Name: "PEERDB_CDC_DISK_SPILL_MEM_PERCENT_THRESHOLD", DefaultValue: "-1", ValueType: protos.DynconfValueType_INT, + Name: "PEERDB_CDC_DISK_SPILL_MEM_PERCENT_THRESHOLD", Description: "CDC: worker memory usage (in %) beyond which records are written to disk instead, -1 disables", + DefaultValue: "-1", + ValueType: protos.DynconfValueType_INT, ApplyMode: protos.DynconfApplyMode_APPLY_MODE_IMMEDIATE, TargetForSetting: protos.DynconfTarget_ALL, }, { - Name: "PEERDB_ENABLE_WAL_HEARTBEAT", DefaultValue: "false", ValueType: protos.DynconfValueType_BOOL, + Name: "PEERDB_ENABLE_WAL_HEARTBEAT", Description: "Enables WAL heartbeat to prevent replication slot lag from increasing during times of no activity", + DefaultValue: "false", + ValueType: protos.DynconfValueType_BOOL, ApplyMode: protos.DynconfApplyMode_APPLY_MODE_IMMEDIATE, TargetForSetting: protos.DynconfTarget_ALL, }, @@ -64,66 +78,92 @@ var DynamicSettings = [...]*protos.DynamicSetting{ DefaultValue: `BEGIN; DROP AGGREGATE IF EXISTS PEERDB_EPHEMERAL_HEARTBEAT(float4); CREATE AGGREGATE PEERDB_EPHEMERAL_HEARTBEAT(float4) (SFUNC = float4pl, STYPE = float4); -DROP AGGREGATE PEERDB_EPHEMERAL_HEARTBEAT(float4); -END;`, ValueType: protos.DynconfValueType_STRING, +DROP AGGREGATE PEERDB_EPHEMERAL_HEARTBEAT(float4); END;`, + ValueType: protos.DynconfValueType_STRING, Description: "SQL to run during each WAL heartbeat", ApplyMode: protos.DynconfApplyMode_APPLY_MODE_IMMEDIATE, TargetForSetting: protos.DynconfTarget_ALL, }, { - Name: "PEERDB_ENABLE_PARALLEL_SYNC_NORMALIZE", DefaultValue: "false", ValueType: protos.DynconfValueType_BOOL, + Name: "PEERDB_ENABLE_PARALLEL_SYNC_NORMALIZE", Description: "Enables parallel sync (moving rows to target) and normalize (updating rows in target table)", + DefaultValue: "false", + ValueType: protos.DynconfValueType_BOOL, ApplyMode: protos.DynconfApplyMode_APPLY_MODE_AFTER_RESUME, TargetForSetting: protos.DynconfTarget_ALL, }, { - Name: "PEERDB_NULLABLE", DefaultValue: "false", ValueType: protos.DynconfValueType_BOOL, + Name: "PEERDB_NULLABLE", Description: "Propagate nullability in schema", + DefaultValue: "false", + ValueType: protos.DynconfValueType_BOOL, ApplyMode: protos.DynconfApplyMode_APPLY_MODE_NEW_MIRROR, TargetForSetting: protos.DynconfTarget_ALL, }, { - Name: "PEERDB_SNOWFLAKE_MERGE_PARALLELISM", DefaultValue: "8", ValueType: protos.DynconfValueType_INT, + Name: "PEERDB_SNOWFLAKE_MERGE_PARALLELISM", Description: "Parallel MERGE statements to run for CDC mirrors with Snowflake targets. -1 for no limit", + DefaultValue: "8", + ValueType: protos.DynconfValueType_INT, ApplyMode: protos.DynconfApplyMode_APPLY_MODE_IMMEDIATE, TargetForSetting: protos.DynconfTarget_SNOWFLAKE, }, { - Name: "PEERDB_CLICKHOUSE_AWS_S3_BUCKET_NAME", DefaultValue: "", ValueType: protos.DynconfValueType_STRING, + Name: "PEERDB_CLICKHOUSE_AWS_S3_BUCKET_NAME", Description: "S3 buckets to store Avro files for mirrors with ClickHouse target", + DefaultValue: "", + ValueType: protos.DynconfValueType_STRING, ApplyMode: protos.DynconfApplyMode_APPLY_MODE_IMMEDIATE, TargetForSetting: protos.DynconfTarget_CLICKHOUSE, }, { - Name: "PEERDB_QUEUE_FORCE_TOPIC_CREATION", DefaultValue: "false", ValueType: protos.DynconfValueType_BOOL, + Name: "PEERDB_QUEUE_FORCE_TOPIC_CREATION", Description: "Force auto topic creation in mirrors, applies to Kafka and PubSub mirrors", + DefaultValue: "false", + ValueType: protos.DynconfValueType_BOOL, ApplyMode: protos.DynconfApplyMode_APPLY_MODE_NEW_MIRROR, TargetForSetting: protos.DynconfTarget_QUEUES, }, { - Name: "PEERDB_ALERTING_GAP_MINUTES", DefaultValue: "15", ValueType: protos.DynconfValueType_UINT, + Name: "PEERDB_ALERTING_GAP_MINUTES", Description: "Duration in minutes before reraising alerts, 0 disables all alerting entirely", + DefaultValue: "15", + ValueType: protos.DynconfValueType_UINT, ApplyMode: protos.DynconfApplyMode_APPLY_MODE_IMMEDIATE, TargetForSetting: protos.DynconfTarget_ALL, }, { - Name: "PEERDB_SLOT_LAG_MB_ALERT_THRESHOLD", DefaultValue: "5000", ValueType: protos.DynconfValueType_UINT, + Name: "PEERDB_SLOT_LAG_MB_ALERT_THRESHOLD", Description: "Lag (in MB) threshold on PeerDB slot to start sending alerts, 0 disables slot lag alerting entirely", + DefaultValue: "5000", + ValueType: protos.DynconfValueType_UINT, ApplyMode: protos.DynconfApplyMode_APPLY_MODE_IMMEDIATE, TargetForSetting: protos.DynconfTarget_ALL, }, { - Name: "PEERDB_PGPEER_OPEN_CONNECTIONS_ALERT_THRESHOLD", DefaultValue: "5", ValueType: protos.DynconfValueType_UINT, + Name: "PEERDB_PGPEER_OPEN_CONNECTIONS_ALERT_THRESHOLD", Description: "Open connections from PeerDB user threshold to start sending alerts, 0 disables open connections alerting entirely", + DefaultValue: "5", + ValueType: protos.DynconfValueType_UINT, ApplyMode: protos.DynconfApplyMode_APPLY_MODE_IMMEDIATE, TargetForSetting: protos.DynconfTarget_ALL, }, { - Name: "PEERDB_BIGQUERY_ENABLE_SYNCED_AT_PARTITIONING_BY_DAYS", DefaultValue: "false", ValueType: protos.DynconfValueType_BOOL, + Name: "PEERDB_BIGQUERY_ENABLE_SYNCED_AT_PARTITIONING_BY_DAYS", Description: "BigQuery only: create target tables with partitioning by _PEERDB_SYNCED_AT column", + DefaultValue: "false", + ValueType: protos.DynconfValueType_BOOL, ApplyMode: protos.DynconfApplyMode_APPLY_MODE_NEW_MIRROR, TargetForSetting: protos.DynconfTarget_BIGQUERY, }, + { + Name: "PEERDB_CLICKHOUSE_ENABLE_PRIMARY_UPDATE", + Description: "Enable generating deletion records for updates in ClickHouse, avoids stale records when primary key updated", + DefaultValue: "false", + ValueType: protos.DynconfValueType_BOOL, + ApplyMode: protos.DynconfApplyMode_APPLY_MODE_IMMEDIATE, + TargetForSetting: protos.DynconfTarget_CLICKHOUSE, + }, } var DynamicIndex = func() map[string]int { @@ -272,6 +312,10 @@ func PeerDBNullable(ctx context.Context, env map[string]string) (bool, error) { return dynamicConfBool(ctx, env, "PEERDB_NULLABLE") } +func PeerDBEnableClickHousePrimaryUpdate(ctx context.Context, env map[string]string) (bool, error) { + return dynamicConfBool(ctx, env, "PEERDB_CLICKHOUSE_ENABLE_PRIMARY_UPDATE") +} + func PeerDBSnowflakeMergeParallelism(ctx context.Context, env map[string]string) (int64, error) { return dynamicConfSigned[int64](ctx, env, "PEERDB_SNOWFLAKE_MERGE_PARALLELISM") } From 40a4927b0257c3938d00bbc391037e686e24d308 Mon Sep 17 00:00:00 2001 From: Kunal Gupta <39487888+iamKunalGupta@users.noreply.github.com> Date: Tue, 1 Oct 2024 22:21:19 +0530 Subject: [PATCH 03/13] fix: slack invite url (#2118) --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 007778b2aa..ebd4579f19 100644 --- a/README.md +++ b/README.md @@ -7,7 +7,7 @@ [![Workflow Status](https://github.com/PeerDB-io/peerdb/actions/workflows/ci.yml/badge.svg)](https://github.com/Peerdb-io/peerdb/actions/workflows/ci.yml) [![ElV2 License](https://badgen.net/badge/License/Elv2/green?icon=github)](https://github.com/PeerDB-io/peerdb/blob/main/LICENSE.md) -[![Slack Community](https://img.shields.io/badge/slack-peerdb-brightgreen.svg?logo=slack)](https://join.slack.com/t/peerdb-public/shared_invite/zt-1wo9jydev-EXInbMtCtpAKFFWdi7QvLQ) +[![Slack Community](https://img.shields.io/badge/slack-peerdb-brightgreen.svg?logo=slack)](https://slack.peerdb.io) From 679d6ee7e5f8ff29c1ad2d3b952673384516199a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Tue, 1 Oct 2024 18:01:00 +0000 Subject: [PATCH 04/13] ch: retry queries for some kinds of errors (#2119) has impacted multiple customers --- flow/connectors/clickhouse/cdc.go | 2 +- flow/connectors/clickhouse/clickhouse.go | 95 ++++++++++++++++++++++-- flow/connectors/clickhouse/normalize.go | 2 +- flow/connectors/clickhouse/qrep.go | 2 +- 4 files changed, 91 insertions(+), 10 deletions(-) diff --git a/flow/connectors/clickhouse/cdc.go b/flow/connectors/clickhouse/cdc.go index 394dac3a93..1c0b651ad0 100644 --- a/flow/connectors/clickhouse/cdc.go +++ b/flow/connectors/clickhouse/cdc.go @@ -30,7 +30,7 @@ func (c *ClickHouseConnector) getRawTableName(flowJobName string) string { func (c *ClickHouseConnector) checkIfTableExists(ctx context.Context, databaseName string, tableIdentifier string) (bool, error) { var result sql.NullInt32 - err := c.database.QueryRow(ctx, checkIfTableExistsSQL, databaseName, tableIdentifier).Scan(&result) + err := c.queryRow(ctx, checkIfTableExistsSQL, databaseName, tableIdentifier).Scan(&result) if err != nil { return false, fmt.Errorf("error while reading result row: %w", err) } diff --git a/flow/connectors/clickhouse/clickhouse.go b/flow/connectors/clickhouse/clickhouse.go index f9d9979d1f..ce50f53aa6 100644 --- a/flow/connectors/clickhouse/clickhouse.go +++ b/flow/connectors/clickhouse/clickhouse.go @@ -14,6 +14,7 @@ import ( "time" "github.com/ClickHouse/clickhouse-go/v2" + "github.com/ClickHouse/clickhouse-go/v2/lib/driver" "github.com/aws/aws-sdk-go-v2/aws" "go.temporal.io/sdk/log" "golang.org/x/mod/semver" @@ -74,7 +75,7 @@ func (c *ClickHouseConnector) ValidateCheck(ctx context.Context) error { } validateDummyTableName := "peerdb_validation_" + shared.RandomString(4) // create a table - err := c.database.Exec(ctx, fmt.Sprintf(`CREATE TABLE IF NOT EXISTS %s ( + err := c.exec(ctx, fmt.Sprintf(`CREATE TABLE IF NOT EXISTS %s ( id UInt64 ) ENGINE = ReplacingMergeTree ORDER BY id;`, validateDummyTableName+"_temp")) @@ -83,27 +84,27 @@ func (c *ClickHouseConnector) ValidateCheck(ctx context.Context) error { } // add a column - err = c.database.Exec(ctx, fmt.Sprintf("ALTER TABLE %s ADD COLUMN updated_at DateTime64(9) DEFAULT now64()", + err = c.exec(ctx, fmt.Sprintf("ALTER TABLE %s ADD COLUMN updated_at DateTime64(9) DEFAULT now64()", validateDummyTableName+"_temp")) if err != nil { return fmt.Errorf("failed to add column to validation table %s: %w", validateDummyTableName, err) } // rename the table - err = c.database.Exec(ctx, fmt.Sprintf("RENAME TABLE %s TO %s", + err = c.exec(ctx, fmt.Sprintf("RENAME TABLE %s TO %s", validateDummyTableName+"_temp", validateDummyTableName)) if err != nil { return fmt.Errorf("failed to rename validation table %s: %w", validateDummyTableName, err) } // insert a row - err = c.database.Exec(ctx, fmt.Sprintf("INSERT INTO %s VALUES (1, now64())", validateDummyTableName)) + err = c.exec(ctx, fmt.Sprintf("INSERT INTO %s VALUES (1, now64())", validateDummyTableName)) if err != nil { return fmt.Errorf("failed to insert into validation table %s: %w", validateDummyTableName, err) } // drop the table - err = c.database.Exec(ctx, "DROP TABLE IF EXISTS "+validateDummyTableName) + err = c.exec(ctx, "DROP TABLE IF EXISTS "+validateDummyTableName) if err != nil { return fmt.Errorf("failed to drop validation table %s: %w", validateDummyTableName, err) } @@ -261,6 +262,86 @@ func Connect(ctx context.Context, config *protos.ClickhouseConfig) (clickhouse.C return conn, nil } +// https://github.com/ClickHouse/clickhouse-kafka-connect/blob/2e0c17e2f900d29c00482b9d0a1f55cb678244e5/src/main/java/com/clickhouse/kafka/connect/util/Utils.java#L78-L93 +// +//nolint:lll +var retryableExceptions = map[int32]struct{}{ + 3: {}, // UNEXPECTED_END_OF_FILE + 107: {}, // FILE_DOESNT_EXIST + 159: {}, // TIMEOUT_EXCEEDED + 164: {}, // READONLY + 202: {}, // TOO_MANY_SIMULTANEOUS_QUERIES + 203: {}, // NO_FREE_CONNECTION + 209: {}, // SOCKET_TIMEOUT + 210: {}, // NETWORK_ERROR + 241: {}, // MEMORY_LIMIT_EXCEEDED + 242: {}, // TABLE_IS_READ_ONLY + 252: {}, // TOO_MANY_PARTS + 285: {}, // TOO_FEW_LIVE_REPLICAS + 319: {}, // UNKNOWN_STATUS_OF_INSERT + 425: {}, // SYSTEM_ERROR + 999: {}, // KEEPER_EXCEPTION +} + +func isRetryableException(err error) bool { + if ex, ok := err.(*clickhouse.Exception); ok { + if ex == nil { + return false + } + _, yes := retryableExceptions[ex.Code] + return yes + } + return false +} + +//nolint:unparam +func (c *ClickHouseConnector) exec(ctx context.Context, query string, args ...any) error { + var err error + for i := range 5 { + err = c.database.Exec(ctx, query, args...) + if !isRetryableException(err) { + break + } + c.logger.Info("[exec] retryable error", slog.Any("error", err), slog.Any("query", query), slog.Int64("i", int64(i))) + if i < 4 { + time.Sleep(time.Second * time.Duration(i*5+1)) + } + } + return err +} + +func (c *ClickHouseConnector) query(ctx context.Context, query string, args ...any) (driver.Rows, error) { + var rows driver.Rows + var err error + for i := range 5 { + rows, err = c.database.Query(ctx, query, args...) + if !isRetryableException(err) { + break + } + c.logger.Info("[query] retryable error", slog.Any("error", err), slog.Any("query", query), slog.Int64("i", int64(i))) + if i < 4 { + time.Sleep(time.Second * time.Duration(i*5+1)) + } + } + return rows, err +} + +func (c *ClickHouseConnector) queryRow(ctx context.Context, query string, args ...any) driver.Row { + var row driver.Row + for i := range 5 { + row = c.database.QueryRow(ctx, query, args...) + err := row.Err() + if !isRetryableException(err) { + break + } + c.logger.Info("[queryRow] retryable error", slog.Any("error", row.Err()), slog.Any("query", query), slog.Int64("i", int64(i))) + if i < 4 { + time.Sleep(time.Second * time.Duration(i*5+1)) + } + } + return row +} + func (c *ClickHouseConnector) Close() error { if c != nil { err := c.database.Close() @@ -287,7 +368,7 @@ func (c *ClickHouseConnector) checkTablesEmptyAndEngine(ctx context.Context, tab for _, table := range tables { queryInput = append(queryInput, table) } - rows, err := c.database.Query(ctx, + rows, err := c.query(ctx, fmt.Sprintf("SELECT name,engine,total_rows FROM system.tables WHERE database=? AND table IN (%s)", strings.Join(slices.Repeat([]string{"?"}, len(tables)), ",")), queryInput...) if err != nil { @@ -325,7 +406,7 @@ func (c *ClickHouseConnector) getTableColumnsMapping(ctx context.Context, for _, table := range tables { queryInput = append(queryInput, table) } - rows, err := c.database.Query(ctx, + rows, err := c.query(ctx, fmt.Sprintf("SELECT name,type,table FROM system.columns WHERE database=? AND table IN (%s)", strings.Join(slices.Repeat([]string{"?"}, len(tables)), ",")), queryInput...) if err != nil { diff --git a/flow/connectors/clickhouse/normalize.go b/flow/connectors/clickhouse/normalize.go index f69904cb05..8a44f17ffd 100644 --- a/flow/connectors/clickhouse/normalize.go +++ b/flow/connectors/clickhouse/normalize.go @@ -442,7 +442,7 @@ func (c *ClickHouseConnector) getDistinctTableNamesInBatch( `SELECT DISTINCT _peerdb_destination_table_name FROM %s WHERE _peerdb_batch_id > %d AND _peerdb_batch_id <= %d`, rawTbl, normalizeBatchID, syncBatchID) - rows, err := c.database.Query(ctx, q) + rows, err := c.query(ctx, q) if err != nil { return nil, fmt.Errorf("error while querying raw table for distinct table names in batch: %w", err) } diff --git a/flow/connectors/clickhouse/qrep.go b/flow/connectors/clickhouse/qrep.go index ad6c9d6e01..1d5b2c1d71 100644 --- a/flow/connectors/clickhouse/qrep.go +++ b/flow/connectors/clickhouse/qrep.go @@ -46,7 +46,7 @@ func (c *ClickHouseConnector) SyncQRepRecords( func (c *ClickHouseConnector) getTableSchema(ctx context.Context, tableName string) ([]driver.ColumnType, error) { queryString := fmt.Sprintf(`SELECT * FROM %s LIMIT 0`, tableName) - rows, err := c.database.Query(ctx, queryString) + rows, err := c.query(ctx, queryString) if err != nil { return nil, fmt.Errorf("failed to execute query: %w", err) } From 4023f6b556315d92eacc737d49da70e925d4eefd Mon Sep 17 00:00:00 2001 From: Kevin Biju <52661649+heavycrystal@users.noreply.github.com> Date: Wed, 2 Oct 2024 02:16:43 +0530 Subject: [PATCH 05/13] [clickhouse] defer dropping table during ValidateMirror (#2116) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit We're observing this function, called while creating or updating a peer to validate it, can sometimes fail midway and leave this table around, trying to reduce the possibility of that --------- Co-authored-by: Philip Dubé --- flow/connectors/clickhouse/clickhouse.go | 33 ++++++++++++++---------- flow/e2e/postgres/peer_flow_pg_test.go | 6 ++--- 2 files changed, 22 insertions(+), 17 deletions(-) diff --git a/flow/connectors/clickhouse/clickhouse.go b/flow/connectors/clickhouse/clickhouse.go index ce50f53aa6..b7434c2b18 100644 --- a/flow/connectors/clickhouse/clickhouse.go +++ b/flow/connectors/clickhouse/clickhouse.go @@ -78,41 +78,46 @@ func (c *ClickHouseConnector) ValidateCheck(ctx context.Context) error { err := c.exec(ctx, fmt.Sprintf(`CREATE TABLE IF NOT EXISTS %s ( id UInt64 ) ENGINE = ReplacingMergeTree ORDER BY id;`, - validateDummyTableName+"_temp")) + validateDummyTableName)) if err != nil { return fmt.Errorf("failed to create validation table %s: %w", validateDummyTableName, err) } + defer func() { + ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + defer cancel() + if err := c.exec(ctx, "DROP TABLE IF EXISTS "+validateDummyTableName); err != nil { + c.logger.Error("validation failed to drop table", slog.String("table", validateDummyTableName), slog.Any("error", err)) + } + }() // add a column - err = c.exec(ctx, fmt.Sprintf("ALTER TABLE %s ADD COLUMN updated_at DateTime64(9) DEFAULT now64()", - validateDummyTableName+"_temp")) - if err != nil { + if err := c.exec(ctx, + fmt.Sprintf("ALTER TABLE %s ADD COLUMN updated_at DateTime64(9) DEFAULT now64()", validateDummyTableName), + ); err != nil { return fmt.Errorf("failed to add column to validation table %s: %w", validateDummyTableName, err) } // rename the table - err = c.exec(ctx, fmt.Sprintf("RENAME TABLE %s TO %s", - validateDummyTableName+"_temp", validateDummyTableName)) - if err != nil { + if err := c.exec(ctx, + fmt.Sprintf("RENAME TABLE %s TO %s", validateDummyTableName, validateDummyTableName+"_renamed"), + ); err != nil { return fmt.Errorf("failed to rename validation table %s: %w", validateDummyTableName, err) } + validateDummyTableName += "_renamed" // insert a row - err = c.exec(ctx, fmt.Sprintf("INSERT INTO %s VALUES (1, now64())", validateDummyTableName)) - if err != nil { + if err := c.exec(ctx, fmt.Sprintf("INSERT INTO %s VALUES (1, now64())", validateDummyTableName)); err != nil { return fmt.Errorf("failed to insert into validation table %s: %w", validateDummyTableName, err) } // drop the table - err = c.exec(ctx, "DROP TABLE IF EXISTS "+validateDummyTableName) - if err != nil { + if err := c.exec(ctx, "DROP TABLE IF EXISTS "+validateDummyTableName); err != nil { return fmt.Errorf("failed to drop validation table %s: %w", validateDummyTableName, err) } // validate s3 stage - validateErr := ValidateS3(ctx, c.credsProvider) - if validateErr != nil { - return fmt.Errorf("failed to validate S3 bucket: %w", validateErr) + if err := ValidateS3(ctx, c.credsProvider); err != nil { + return fmt.Errorf("failed to validate S3 bucket: %w", err) } return nil diff --git a/flow/e2e/postgres/peer_flow_pg_test.go b/flow/e2e/postgres/peer_flow_pg_test.go index e0f107cb65..f2ca26443a 100644 --- a/flow/e2e/postgres/peer_flow_pg_test.go +++ b/flow/e2e/postgres/peer_flow_pg_test.go @@ -1143,10 +1143,10 @@ func (s PeerFlowE2ETestSuitePG) Test_TransformRowScript() { require.False(s.t, exists) } -func (s PeerFlowE2ETestSuitePG) Test_Simple_Schema_Changes_PG() { +func (s PeerFlowE2ETestSuitePG) Test_Mixed_Case_Schema_Changes_PG() { tc := e2e.NewTemporalClient(s.t) - srcTableName := "test_simple_schema_changes_PG" + srcTableName := "test_mixed_case_schema_changes_PG" dstTableName := srcTableName + "_dst" quotedSourceTableName := s.attachSchemaSuffix(`"` + srcTableName + `"`) quotedDestTableName := s.attachSchemaSuffix(`"` + dstTableName + `"`) @@ -1159,7 +1159,7 @@ func (s PeerFlowE2ETestSuitePG) Test_Simple_Schema_Changes_PG() { require.NoError(s.t, err) flowConnConfig := &protos.FlowConnectionConfigs{ - FlowJobName: s.attachSuffix("test_simple_schema_changes_pg"), + FlowJobName: s.attachSuffix("test_mixed_case_schema_changes_pg"), DestinationName: s.Peer().Name, TableMappings: []*protos.TableMapping{ { From 505f6e42f5146a7fe61b55b50f383654db3def76 Mon Sep 17 00:00:00 2001 From: Kunal Gupta <39487888+iamKunalGupta@users.noreply.github.com> Date: Wed, 2 Oct 2024 04:10:02 +0530 Subject: [PATCH 06/13] feat: add supabase oauth state as env var (#2109) --- ui/app/api/peer-types/route.ts | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ui/app/api/peer-types/route.ts b/ui/app/api/peer-types/route.ts index f5f5243992..8c005b4e3b 100644 --- a/ui/app/api/peer-types/route.ts +++ b/ui/app/api/peer-types/route.ts @@ -40,7 +40,7 @@ export async function GET(request: NextRequest) { process.env.SUPABASE_ID )}&response_type=code&redirect_uri=${encodeURIComponent( process.env.SUPABASE_REDIRECT ?? '' - )}`, + )}&state=${encodeURIComponent(process.env.SUPABASE_OAUTH_STATE ?? '')}`, }); } From 064a2a9b616b31204129c78fa1399a04135213f6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Wed, 2 Oct 2024 14:41:10 +0000 Subject: [PATCH 07/13] Replace 32 sync limit with ContinueAsNewSuggested (#2014) Experiment. Still unsure since long history impacts ui latency, but nice to have something that'll tune restart frequency based on workload --- flow/model/signals.go | 4 ---- flow/peerdbenv/dynamicconf.go | 5 ----- flow/workflows/cdc_flow.go | 9 ++------- flow/workflows/local_activities.go | 18 ------------------ flow/workflows/sync_flow.go | 28 +--------------------------- 5 files changed, 3 insertions(+), 61 deletions(-) diff --git a/flow/model/signals.go b/flow/model/signals.go index 53fcf7e1e5..7e98ab343f 100644 --- a/flow/model/signals.go +++ b/flow/model/signals.go @@ -139,10 +139,6 @@ var SyncStopSignal = TypedSignal[struct{}]{ Name: "sync-stop", } -var SyncResultSignal = TypedSignal[*SyncResponse]{ - Name: "sync-result", -} - var NormalizeSignal = TypedSignal[NormalizePayload]{ Name: "normalize", } diff --git a/flow/peerdbenv/dynamicconf.go b/flow/peerdbenv/dynamicconf.go index 64909c75e2..0c36f0c9cd 100644 --- a/flow/peerdbenv/dynamicconf.go +++ b/flow/peerdbenv/dynamicconf.go @@ -329,8 +329,3 @@ func PeerDBClickHouseAWSS3BucketName(ctx context.Context, env map[string]string) func PeerDBQueueForceTopicCreation(ctx context.Context, env map[string]string) (bool, error) { return dynamicConfBool(ctx, env, "PEERDB_QUEUE_FORCE_TOPIC_CREATION") } - -// experimental, don't increase to greater than 64 -func PeerDBMaxSyncsPerCDCFlow(ctx context.Context, env map[string]string) (uint32, error) { - return dynamicConfUnsigned[uint32](ctx, env, "PEERDB_MAX_SYNCS_PER_CDC_FLOW") -} diff --git a/flow/workflows/cdc_flow.go b/flow/workflows/cdc_flow.go index e7dd65930a..a45ec28793 100644 --- a/flow/workflows/cdc_flow.go +++ b/flow/workflows/cdc_flow.go @@ -572,11 +572,6 @@ func CDCFlowWorkflow( state.ActiveSignal = model.FlowSignalHandler(state.ActiveSignal, val, logger) }) - syncResultChan := model.SyncResultSignal.GetSignalChannel(ctx) - syncResultChan.AddToSelector(mainLoopSelector, func(result *model.SyncResponse, _ bool) { - syncCount += 1 - }) - normChan := model.NormalizeSignal.GetSignalChannel(ctx) normChan.AddToSelector(mainLoopSelector, func(payload model.NormalizePayload, _ bool) { if normFlowFuture != nil { @@ -589,6 +584,7 @@ func CDCFlowWorkflow( normDoneChan := model.NormalizeDoneSignal.GetSignalChannel(ctx) normDoneChan.Drain() normDoneChan.AddToSelector(mainLoopSelector, func(x struct{}, _ bool) { + syncCount += 1 if syncCount == syncCountLimit { logger.Info("sync count limit reached, pausing", slog.Int("limit", syncCountLimit), @@ -604,7 +600,6 @@ func CDCFlowWorkflow( addCdcPropertiesSignalListener(ctx, logger, mainLoopSelector, state) state.CurrentFlowStatus = protos.FlowStatus_STATUS_RUNNING - maxSyncPerCDCFlow := int(getMaxSyncsPerCDCFlow(ctx, logger, cfg.Env)) for { mainLoopSelector.Select(ctx) for ctx.Err() == nil && mainLoopSelector.HasPending() { @@ -615,7 +610,7 @@ func CDCFlowWorkflow( return state, err } - if state.ActiveSignal == model.PauseSignal || syncCount >= maxSyncPerCDCFlow { + if state.ActiveSignal == model.PauseSignal || workflow.GetInfo(ctx).GetContinueAsNewSuggested() { restart = true if syncFlowFuture != nil { err := model.SyncStopSignal.SignalChildWorkflow(ctx, syncFlowFuture, struct{}{}).Get(ctx, nil) diff --git a/flow/workflows/local_activities.go b/flow/workflows/local_activities.go index 7bae7cb5f3..d163352ca2 100644 --- a/flow/workflows/local_activities.go +++ b/flow/workflows/local_activities.go @@ -15,10 +15,6 @@ import ( "github.com/PeerDB-io/peer-flow/shared" ) -const ( - defaultMaxSyncsPerCdcFlow = 32 -) - func getParallelSyncNormalize(wCtx workflow.Context, logger log.Logger, env map[string]string) bool { checkCtx := workflow.WithLocalActivityOptions(wCtx, workflow.LocalActivityOptions{ StartToCloseTimeout: time.Minute, @@ -33,20 +29,6 @@ func getParallelSyncNormalize(wCtx workflow.Context, logger log.Logger, env map[ return parallel } -func getMaxSyncsPerCDCFlow(wCtx workflow.Context, logger log.Logger, env map[string]string) uint32 { - checkCtx := workflow.WithLocalActivityOptions(wCtx, workflow.LocalActivityOptions{ - StartToCloseTimeout: time.Minute, - }) - - getFuture := workflow.ExecuteLocalActivity(checkCtx, peerdbenv.PeerDBMaxSyncsPerCDCFlow, env) - var maxSyncsPerCDCFlow uint32 - if err := getFuture.Get(checkCtx, &maxSyncsPerCDCFlow); err != nil { - logger.Warn("Failed to get max syncs per CDC flow, returning default of 32", slog.Any("error", err)) - return defaultMaxSyncsPerCdcFlow - } - return maxSyncsPerCDCFlow -} - func localPeerType(ctx context.Context, name string) (protos.DBType, error) { pool, err := peerdbenv.GetCatalogConnectionPoolFromEnv(ctx) if err != nil { diff --git a/flow/workflows/sync_flow.go b/flow/workflows/sync_flow.go index 517e31b9c6..8b00364dd2 100644 --- a/flow/workflows/sync_flow.go +++ b/flow/workflows/sync_flow.go @@ -13,13 +13,6 @@ import ( "github.com/PeerDB-io/peer-flow/shared" ) -// For now cdc restarts sync flow whenever it itself restarts, -// set this value high enough to never be met, relying on cdc restarts. -// In the future cdc flow restarts could be decoupled from sync flow restarts. -const ( - maxSyncsPerSyncFlow = 64 -) - func SyncFlowWorkflow( ctx workflow.Context, config *protos.FlowConnectionConfigs, @@ -94,7 +87,6 @@ func SyncFlowWorkflow( var syncDone bool mustWait := waitSelector != nil - // execute the sync flow currentSyncFlowNum += 1 logger.Info("executing sync flow", slog.Int("count", currentSyncFlowNum)) @@ -110,20 +102,8 @@ func SyncFlowWorkflow( var childSyncFlowRes *model.SyncCompositeResponse if err := f.Get(ctx, &childSyncFlowRes); err != nil { logger.Error("failed to execute sync flow", slog.Any("error", err)) - _ = model.SyncResultSignal.SignalExternalWorkflow( - ctx, - parent.ID, - "", - nil, - ).Get(ctx, nil) syncErr = true } else if childSyncFlowRes != nil { - _ = model.SyncResultSignal.SignalExternalWorkflow( - ctx, - parent.ID, - "", - childSyncFlowRes.SyncResponse, - ).Get(ctx, nil) totalRecordsSynced += childSyncFlowRes.SyncResponse.NumRecordsSynced logger.Info("Total records synced", slog.Int64("totalRecordsSynced", totalRecordsSynced)) @@ -149,12 +129,6 @@ func SyncFlowWorkflow( if err := getModifiedSchemaFuture.Get(ctx, nil); err != nil { logger.Error("failed to execute schema update at source", slog.Any("error", err)) - _ = model.SyncResultSignal.SignalExternalWorkflow( - ctx, - parent.ID, - "", - nil, - ).Get(ctx, nil) } } @@ -187,7 +161,7 @@ func SyncFlowWorkflow( break } - restart := currentSyncFlowNum >= maxSyncsPerSyncFlow || syncErr + restart := syncErr || workflow.GetInfo(ctx).GetContinueAsNewSuggested() if !stop && !syncErr && mustWait { waitSelector.Select(ctx) if restart { From 5d847b5c5a31d57866a8da637c216e83289848d9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Wed, 2 Oct 2024 16:57:19 +0000 Subject: [PATCH 08/13] ch retries: also retry on io.EOF (#2122) clickhouse-go error may not be exception when it propagates io errors, io.EOF is indicative of a disconnect, which should be retried --- flow/connectors/clickhouse/clickhouse.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flow/connectors/clickhouse/clickhouse.go b/flow/connectors/clickhouse/clickhouse.go index b7434c2b18..1320bf8a92 100644 --- a/flow/connectors/clickhouse/clickhouse.go +++ b/flow/connectors/clickhouse/clickhouse.go @@ -6,6 +6,7 @@ import ( "crypto/x509" "errors" "fmt" + "io" "log/slog" "maps" "net/url" @@ -296,7 +297,7 @@ func isRetryableException(err error) bool { _, yes := retryableExceptions[ex.Code] return yes } - return false + return errors.Is(err, io.EOF) } //nolint:unparam From bb451152875a370ee597609a196640f29303a617 Mon Sep 17 00:00:00 2001 From: Kevin Biju <52661649+heavycrystal@users.noreply.github.com> Date: Thu, 3 Oct 2024 00:35:12 +0530 Subject: [PATCH 09/13] [clickhouse] test for replident full with unchanged TOAST (#2123) followup on #2092 --- flow/connectors/postgres/cdc.go | 1 - flow/e2e/clickhouse/peer_flow_ch_test.go | 47 +++++++++++++++++++++ flow/e2e/clickhouse/test_data/big_data.json | 1 + 3 files changed, 48 insertions(+), 1 deletion(-) create mode 100644 flow/e2e/clickhouse/test_data/big_data.json diff --git a/flow/connectors/postgres/cdc.go b/flow/connectors/postgres/cdc.go index 9d666d97da..9f43cd6599 100644 --- a/flow/connectors/postgres/cdc.go +++ b/flow/connectors/postgres/cdc.go @@ -758,7 +758,6 @@ func processUpdateMessage[Items model.Items]( and only the old tuple does. So we can backfill the new tuple with the unchanged columns from the old tuple. Otherwise, _peerdb_unchanged_toast_columns is set correctly and we fallback to normal unchanged TOAST handling in normalize, but this doesn't work in connectors where we don't do unchanged TOAST handling in normalize. - TODO: investigate the cases where this happens in more detail. */ backfilledCols := newItems.UpdateIfNotExists(oldItems) for _, col := range backfilledCols { diff --git a/flow/e2e/clickhouse/peer_flow_ch_test.go b/flow/e2e/clickhouse/peer_flow_ch_test.go index d2b6619513..f65aaa69bc 100644 --- a/flow/e2e/clickhouse/peer_flow_ch_test.go +++ b/flow/e2e/clickhouse/peer_flow_ch_test.go @@ -2,6 +2,7 @@ package e2e_clickhouse import ( "context" + "embed" "fmt" "testing" "time" @@ -16,6 +17,9 @@ import ( peerflow "github.com/PeerDB-io/peer-flow/workflows" ) +//go:embed test_data/* +var testData embed.FS + func TestPeerFlowE2ETestSuiteCH(t *testing.T) { e2eshared.RunSuite(t, SetupSuite) } @@ -355,3 +359,46 @@ func (s ClickHouseSuite) Test_Update_PKey_Env_Enabled() { env.Cancel() e2e.RequireEnvCanceled(s.t, env) } + +func (s ClickHouseSuite) Test_Replident_Full_Unchanged_TOAST_Updates() { + srcTableName := "test_replident_full_toast" + srcFullName := s.attachSchemaSuffix("test_replident_full_toast") + dstTableName := "test_replident_full_toast_dst" + + _, err := s.Conn().Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s( + id INT PRIMARY KEY GENERATED BY DEFAULT AS IDENTITY, + c1 INT, + c2 INT, + t TEXT); + ALTER TABLE %s REPLICA IDENTITY FULL`, srcFullName, srcFullName)) + require.NoError(s.t, err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("clickhouse_test_replident_full_toast"), + TableNameMapping: map[string]string{srcFullName: dstTableName}, + Destination: s.Peer().Name, + } + flowConnConfig := connectionGen.GenerateFlowConnectionConfigs(s.t) + + tc := e2e.NewTemporalClient(s.t) + env := e2e.ExecutePeerflow(tc, peerflow.CDCFlowWorkflow, flowConnConfig, nil) + e2e.SetupCDCFlowStatusQuery(s.t, env, flowConnConfig) + + content, err := testData.ReadFile("test_data/big_data.json") + require.NoError(s.t, err) + contentStr := string(content) + + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s (c1,c2,t) VALUES ($1,$2,$3)`, srcFullName), 1, 2, contentStr) + require.NoError(s.t, err) + e2e.EnvWaitForEqualTablesWithNames(env, s, "waiting on initial insert", srcTableName, dstTableName, "id,c1,c2,t") + + _, err = s.Conn().Exec(context.Background(), fmt.Sprintf(` + UPDATE %s SET c1=$1 WHERE id=$2`, srcFullName), 3, 1) + require.NoError(s.t, err) + e2e.EnvWaitForEqualTablesWithNames(env, s, "waiting on update", srcTableName, dstTableName, "id,c1,c2,t") + + env.Cancel() + e2e.RequireEnvCanceled(s.t, env) +} diff --git a/flow/e2e/clickhouse/test_data/big_data.json b/flow/e2e/clickhouse/test_data/big_data.json new file mode 100644 index 0000000000..328b9e5883 --- /dev/null +++ b/flow/e2e/clickhouse/test_data/big_data.json @@ -0,0 +1 @@ +[{"store":{"name":"SuperMart","location":{"address":"123 Market Street","city":"Metropolis","state":"NY","zip":"10001"},"departments":[{"id":1,"name":"Groceries","categories":[{"id":101,"name":"Fruits & Vegetables","products":[{"id":1001,"name":"Apple","price":0.99,"stock":120,"details":{"weight":"1lb","origin":"USA","organic":true}},{"id":1002,"name":"Banana","price":0.39,"stock":200,"details":{"weight":"1lb","origin":"Ecuador","organic":false}}]},{"id":102,"name":"Dairy","products":[{"id":2001,"name":"Milk","price":3.49,"stock":50,"details":{"volume":"1 gallon","fatContent":"2%","brand":"DairyPure"}},{"id":2002,"name":"Cheddar Cheese","price":5.99,"stock":75,"details":{"weight":"8oz","origin":"Wisconsin","aged":"2 years"}}]}]},{"id":2,"name":"Electronics","categories":[{"id":201,"name":"Smartphones","products":[{"id":3001,"name":"iPhone 14","price":999.99,"stock":30,"details":{"color":"Black","storage":"128GB","carrier":"Unlocked"}},{"id":3002,"name":"Samsung Galaxy S21","price":849.99,"stock":25,"details":{"color":"White","storage":"256GB","carrier":"Unlocked"}}]},{"id":202,"name":"Laptops","products":[{"id":4001,"name":"MacBook Pro","price":2399.99,"stock":10,"details":{"screenSize":"16 inch","processor":"M1","ram":"16GB","storage":"512GB SSD"}},{"id":4002,"name":"Dell XPS 13","price":1299.99,"stock":15,"details":{"screenSize":"13 inch","processor":"Intel i7","ram":"16GB","storage":"256GB SSD"}}]}]}],"employees":[{"id":5001,"name":"John Doe","position":"Manager","schedule":[{"day":"Monday","shift":"9am-5pm"},{"day":"Wednesday","shift":"9am-5pm"},{"day":"Friday","shift":"9am-5pm"}]},{"id":5002,"name":"Jane Smith","position":"Cashier","schedule":[{"day":"Tuesday","shift":"10am-6pm"},{"day":"Thursday","shift":"10am-6pm"},{"day":"Saturday","shift":"12pm-8pm"}]}]},"customers":[{"id":6001,"name":"Alice Johnson","email":"alice.johnson@example.com","orders":[{"orderId":7001,"date":"2024-09-01","status":"Shipped","items":[{"productId":1001,"quantity":10},{"productId":2001,"quantity":2}],"total":12.48},{"orderId":7002,"date":"2024-09-15","status":"Delivered","items":[{"productId":3001,"quantity":1}],"total":999.99}]},{"id":6002,"name":"Bob Williams","email":"bob.williams@example.com","orders":[{"orderId":7003,"date":"2024-09-07","status":"Pending","items":[{"productId":1002,"quantity":5},{"productId":4001,"quantity":1}],"total":2403.94}]}]},{"store":{"name":"SuperMart","location":{"address":"123 Market Street","city":"Metropolis","state":"NY","zip":"10001"},"departments":[{"id":1,"name":"Groceries","categories":[{"id":101,"name":"Fruits & Vegetables","products":[{"id":1001,"name":"Apple","price":0.99,"stock":120,"details":{"weight":"1lb","origin":"USA","organic":true}},{"id":1002,"name":"Banana","price":0.39,"stock":200,"details":{"weight":"1lb","origin":"Ecuador","organic":false}}]},{"id":102,"name":"Dairy","products":[{"id":2001,"name":"Milk","price":3.49,"stock":50,"details":{"volume":"1 gallon","fatContent":"2%","brand":"DairyPure"}},{"id":2002,"name":"Cheddar Cheese","price":5.99,"stock":75,"details":{"weight":"8oz","origin":"Wisconsin","aged":"2 years"}}]}]},{"id":2,"name":"Electronics","categories":[{"id":201,"name":"Smartphones","products":[{"id":3001,"name":"iPhone 14","price":999.99,"stock":30,"details":{"color":"Black","storage":"128GB","carrier":"Unlocked"}},{"id":3002,"name":"Samsung Galaxy S21","price":849.99,"stock":25,"details":{"color":"White","storage":"256GB","carrier":"Unlocked"}}]},{"id":202,"name":"Laptops","products":[{"id":4001,"name":"MacBook Pro","price":2399.99,"stock":10,"details":{"screenSize":"16 inch","processor":"M1","ram":"16GB","storage":"512GB SSD"}},{"id":4002,"name":"Dell XPS 13","price":1299.99,"stock":15,"details":{"screenSize":"13 inch","processor":"Intel i7","ram":"16GB","storage":"256GB SSD"}}]}]}],"employees":[{"id":5001,"name":"John Doe","position":"Manager","schedule":[{"day":"Monday","shift":"9am-5pm"},{"day":"Wednesday","shift":"9am-5pm"},{"day":"Friday","shift":"9am-5pm"}]},{"id":5002,"name":"Jane Smith","position":"Cashier","schedule":[{"day":"Tuesday","shift":"10am-6pm"},{"day":"Thursday","shift":"10am-6pm"},{"day":"Saturday","shift":"12pm-8pm"}]}]},"customers":[{"id":6001,"name":"Alice Johnson","email":"alice.johnson@example.com","orders":[{"orderId":7001,"date":"2024-09-01","status":"Shipped","items":[{"productId":1001,"quantity":10},{"productId":2001,"quantity":2}],"total":12.48},{"orderId":7002,"date":"2024-09-15","status":"Delivered","items":[{"productId":3001,"quantity":1}],"total":999.99}]},{"id":6002,"name":"Bob Williams","email":"bob.williams@example.com","orders":[{"orderId":7003,"date":"2024-09-07","status":"Pending","items":[{"productId":1002,"quantity":5},{"productId":4001,"quantity":1}],"total":2403.94}]}]},{"store":{"name":"SuperMart","location":{"address":"123 Market Street","city":"Metropolis","state":"NY","zip":"10001"},"departments":[{"id":1,"name":"Groceries","categories":[{"id":101,"name":"Fruits & Vegetables","products":[{"id":1001,"name":"Apple","price":0.99,"stock":120,"details":{"weight":"1lb","origin":"USA","organic":true}},{"id":1002,"name":"Banana","price":0.39,"stock":200,"details":{"weight":"1lb","origin":"Ecuador","organic":false}}]},{"id":102,"name":"Dairy","products":[{"id":2001,"name":"Milk","price":3.49,"stock":50,"details":{"volume":"1 gallon","fatContent":"2%","brand":"DairyPure"}},{"id":2002,"name":"Cheddar Cheese","price":5.99,"stock":75,"details":{"weight":"8oz","origin":"Wisconsin","aged":"2 years"}}]}]},{"id":2,"name":"Electronics","categories":[{"id":201,"name":"Smartphones","products":[{"id":3001,"name":"iPhone 14","price":999.99,"stock":30,"details":{"color":"Black","storage":"128GB","carrier":"Unlocked"}},{"id":3002,"name":"Samsung Galaxy S21","price":849.99,"stock":25,"details":{"color":"White","storage":"256GB","carrier":"Unlocked"}}]},{"id":202,"name":"Laptops","products":[{"id":4001,"name":"MacBook Pro","price":2399.99,"stock":10,"details":{"screenSize":"16 inch","processor":"M1","ram":"16GB","storage":"512GB SSD"}},{"id":4002,"name":"Dell XPS 13","price":1299.99,"stock":15,"details":{"screenSize":"13 inch","processor":"Intel i7","ram":"16GB","storage":"256GB SSD"}}]}]}],"employees":[{"id":5001,"name":"John Doe","position":"Manager","schedule":[{"day":"Monday","shift":"9am-5pm"},{"day":"Wednesday","shift":"9am-5pm"},{"day":"Friday","shift":"9am-5pm"}]},{"id":5002,"name":"Jane Smith","position":"Cashier","schedule":[{"day":"Tuesday","shift":"10am-6pm"},{"day":"Thursday","shift":"10am-6pm"},{"day":"Saturday","shift":"12pm-8pm"}]}]},"customers":[{"id":6001,"name":"Alice Johnson","email":"alice.johnson@example.com","orders":[{"orderId":7001,"date":"2024-09-01","status":"Shipped","items":[{"productId":1001,"quantity":10},{"productId":2001,"quantity":2}],"total":12.48},{"orderId":7002,"date":"2024-09-15","status":"Delivered","items":[{"productId":3001,"quantity":1}],"total":999.99}]},{"id":6002,"name":"Bob Williams","email":"bob.williams@example.com","orders":[{"orderId":7003,"date":"2024-09-07","status":"Pending","items":[{"productId":1002,"quantity":5},{"productId":4001,"quantity":1}],"total":2403.94}]}]},{"store":{"name":"SuperMart","location":{"address":"123 Market Street","city":"Metropolis","state":"NY","zip":"10001"},"departments":[{"id":1,"name":"Groceries","categories":[{"id":101,"name":"Fruits & Vegetables","products":[{"id":1001,"name":"Apple","price":0.99,"stock":120,"details":{"weight":"1lb","origin":"USA","organic":true}},{"id":1002,"name":"Banana","price":0.39,"stock":200,"details":{"weight":"1lb","origin":"Ecuador","organic":false}}]},{"id":102,"name":"Dairy","products":[{"id":2001,"name":"Milk","price":3.49,"stock":50,"details":{"volume":"1 gallon","fatContent":"2%","brand":"DairyPure"}},{"id":2002,"name":"Cheddar Cheese","price":5.99,"stock":75,"details":{"weight":"8oz","origin":"Wisconsin","aged":"2 years"}}]}]},{"id":2,"name":"Electronics","categories":[{"id":201,"name":"Smartphones","products":[{"id":3001,"name":"iPhone 14","price":999.99,"stock":30,"details":{"color":"Black","storage":"128GB","carrier":"Unlocked"}},{"id":3002,"name":"Samsung Galaxy S21","price":849.99,"stock":25,"details":{"color":"White","storage":"256GB","carrier":"Unlocked"}}]},{"id":202,"name":"Laptops","products":[{"id":4001,"name":"MacBook Pro","price":2399.99,"stock":10,"details":{"screenSize":"16 inch","processor":"M1","ram":"16GB","storage":"512GB SSD"}},{"id":4002,"name":"Dell XPS 13","price":1299.99,"stock":15,"details":{"screenSize":"13 inch","processor":"Intel i7","ram":"16GB","storage":"256GB SSD"}}]}]}],"employees":[{"id":5001,"name":"John Doe","position":"Manager","schedule":[{"day":"Monday","shift":"9am-5pm"},{"day":"Wednesday","shift":"9am-5pm"},{"day":"Friday","shift":"9am-5pm"}]},{"id":5002,"name":"Jane Smith","position":"Cashier","schedule":[{"day":"Tuesday","shift":"10am-6pm"},{"day":"Thursday","shift":"10am-6pm"},{"day":"Saturday","shift":"12pm-8pm"}]}]},"customers":[{"id":6001,"name":"Alice Johnson","email":"alice.johnson@example.com","orders":[{"orderId":7001,"date":"2024-09-01","status":"Shipped","items":[{"productId":1001,"quantity":10},{"productId":2001,"quantity":2}],"total":12.48},{"orderId":7002,"date":"2024-09-15","status":"Delivered","items":[{"productId":3001,"quantity":1}],"total":999.99}]},{"id":6002,"name":"Bob Williams","email":"bob.williams@example.com","orders":[{"orderId":7003,"date":"2024-09-07","status":"Pending","items":[{"productId":1002,"quantity":5},{"productId":4001,"quantity":1}],"total":2403.94}]}]},{"store":{"name":"SuperMart","location":{"address":"123 Market Street","city":"Metropolis","state":"NY","zip":"10001"},"departments":[{"id":1,"name":"Groceries","categories":[{"id":101,"name":"Fruits & Vegetables","products":[{"id":1001,"name":"Apple","price":0.99,"stock":120,"details":{"weight":"1lb","origin":"USA","organic":true}},{"id":1002,"name":"Banana","price":0.39,"stock":200,"details":{"weight":"1lb","origin":"Ecuador","organic":false}}]},{"id":102,"name":"Dairy","products":[{"id":2001,"name":"Milk","price":3.49,"stock":50,"details":{"volume":"1 gallon","fatContent":"2%","brand":"DairyPure"}},{"id":2002,"name":"Cheddar Cheese","price":5.99,"stock":75,"details":{"weight":"8oz","origin":"Wisconsin","aged":"2 years"}}]}]},{"id":2,"name":"Electronics","categories":[{"id":201,"name":"Smartphones","products":[{"id":3001,"name":"iPhone 14","price":999.99,"stock":30,"details":{"color":"Black","storage":"128GB","carrier":"Unlocked"}},{"id":3002,"name":"Samsung Galaxy S21","price":849.99,"stock":25,"details":{"color":"White","storage":"256GB","carrier":"Unlocked"}}]},{"id":202,"name":"Laptops","products":[{"id":4001,"name":"MacBook Pro","price":2399.99,"stock":10,"details":{"screenSize":"16 inch","processor":"M1","ram":"16GB","storage":"512GB SSD"}},{"id":4002,"name":"Dell XPS 13","price":1299.99,"stock":15,"details":{"screenSize":"13 inch","processor":"Intel i7","ram":"16GB","storage":"256GB SSD"}}]}]}],"employees":[{"id":5001,"name":"John Doe","position":"Manager","schedule":[{"day":"Monday","shift":"9am-5pm"},{"day":"Wednesday","shift":"9am-5pm"},{"day":"Friday","shift":"9am-5pm"}]},{"id":5002,"name":"Jane Smith","position":"Cashier","schedule":[{"day":"Tuesday","shift":"10am-6pm"},{"day":"Thursday","shift":"10am-6pm"},{"day":"Saturday","shift":"12pm-8pm"}]}]},"customers":[{"id":6001,"name":"Alice Johnson","email":"alice.johnson@example.com","orders":[{"orderId":7001,"date":"2024-09-01","status":"Shipped","items":[{"productId":1001,"quantity":10},{"productId":2001,"quantity":2}],"total":12.48},{"orderId":7002,"date":"2024-09-15","status":"Delivered","items":[{"productId":3001,"quantity":1}],"total":999.99}]},{"id":6002,"name":"Bob Williams","email":"bob.williams@example.com","orders":[{"orderId":7003,"date":"2024-09-07","status":"Pending","items":[{"productId":1002,"quantity":5},{"productId":4001,"quantity":1}],"total":2403.94}]}]},{"store":{"name":"SuperMart","location":{"address":"123 Market Street","city":"Metropolis","state":"NY","zip":"10001"},"departments":[{"id":1,"name":"Groceries","categories":[{"id":101,"name":"Fruits & Vegetables","products":[{"id":1001,"name":"Apple","price":0.99,"stock":120,"details":{"weight":"1lb","origin":"USA","organic":true}},{"id":1002,"name":"Banana","price":0.39,"stock":200,"details":{"weight":"1lb","origin":"Ecuador","organic":false}}]},{"id":102,"name":"Dairy","products":[{"id":2001,"name":"Milk","price":3.49,"stock":50,"details":{"volume":"1 gallon","fatContent":"2%","brand":"DairyPure"}},{"id":2002,"name":"Cheddar Cheese","price":5.99,"stock":75,"details":{"weight":"8oz","origin":"Wisconsin","aged":"2 years"}}]}]},{"id":2,"name":"Electronics","categories":[{"id":201,"name":"Smartphones","products":[{"id":3001,"name":"iPhone 14","price":999.99,"stock":30,"details":{"color":"Black","storage":"128GB","carrier":"Unlocked"}},{"id":3002,"name":"Samsung Galaxy S21","price":849.99,"stock":25,"details":{"color":"White","storage":"256GB","carrier":"Unlocked"}}]},{"id":202,"name":"Laptops","products":[{"id":4001,"name":"MacBook Pro","price":2399.99,"stock":10,"details":{"screenSize":"16 inch","processor":"M1","ram":"16GB","storage":"512GB SSD"}},{"id":4002,"name":"Dell XPS 13","price":1299.99,"stock":15,"details":{"screenSize":"13 inch","processor":"Intel i7","ram":"16GB","storage":"256GB SSD"}}]}]}],"employees":[{"id":5001,"name":"John Doe","position":"Manager","schedule":[{"day":"Monday","shift":"9am-5pm"},{"day":"Wednesday","shift":"9am-5pm"},{"day":"Friday","shift":"9am-5pm"}]},{"id":5002,"name":"Jane Smith","position":"Cashier","schedule":[{"day":"Tuesday","shift":"10am-6pm"},{"day":"Thursday","shift":"10am-6pm"},{"day":"Saturday","shift":"12pm-8pm"}]}]},"customers":[{"id":6001,"name":"Alice Johnson","email":"alice.johnson@example.com","orders":[{"orderId":7001,"date":"2024-09-01","status":"Shipped","items":[{"productId":1001,"quantity":10},{"productId":2001,"quantity":2}],"total":12.48},{"orderId":7002,"date":"2024-09-15","status":"Delivered","items":[{"productId":3001,"quantity":1}],"total":999.99}]},{"id":6002,"name":"Bob Williams","email":"bob.williams@example.com","orders":[{"orderId":7003,"date":"2024-09-07","status":"Pending","items":[{"productId":1002,"quantity":5},{"productId":4001,"quantity":1}],"total":2403.94}]}]},{"store":{"name":"SuperMart","location":{"address":"123 Market Street","city":"Metropolis","state":"NY","zip":"10001"},"departments":[{"id":1,"name":"Groceries","categories":[{"id":101,"name":"Fruits & Vegetables","products":[{"id":1001,"name":"Apple","price":0.99,"stock":120,"details":{"weight":"1lb","origin":"USA","organic":true}},{"id":1002,"name":"Banana","price":0.39,"stock":200,"details":{"weight":"1lb","origin":"Ecuador","organic":false}}]},{"id":102,"name":"Dairy","products":[{"id":2001,"name":"Milk","price":3.49,"stock":50,"details":{"volume":"1 gallon","fatContent":"2%","brand":"DairyPure"}},{"id":2002,"name":"Cheddar Cheese","price":5.99,"stock":75,"details":{"weight":"8oz","origin":"Wisconsin","aged":"2 years"}}]}]},{"id":2,"name":"Electronics","categories":[{"id":201,"name":"Smartphones","products":[{"id":3001,"name":"iPhone 14","price":999.99,"stock":30,"details":{"color":"Black","storage":"128GB","carrier":"Unlocked"}},{"id":3002,"name":"Samsung Galaxy S21","price":849.99,"stock":25,"details":{"color":"White","storage":"256GB","carrier":"Unlocked"}}]},{"id":202,"name":"Laptops","products":[{"id":4001,"name":"MacBook Pro","price":2399.99,"stock":10,"details":{"screenSize":"16 inch","processor":"M1","ram":"16GB","storage":"512GB SSD"}},{"id":4002,"name":"Dell XPS 13","price":1299.99,"stock":15,"details":{"screenSize":"13 inch","processor":"Intel i7","ram":"16GB","storage":"256GB SSD"}}]}]}],"employees":[{"id":5001,"name":"John Doe","position":"Manager","schedule":[{"day":"Monday","shift":"9am-5pm"},{"day":"Wednesday","shift":"9am-5pm"},{"day":"Friday","shift":"9am-5pm"}]},{"id":5002,"name":"Jane Smith","position":"Cashier","schedule":[{"day":"Tuesday","shift":"10am-6pm"},{"day":"Thursday","shift":"10am-6pm"},{"day":"Saturday","shift":"12pm-8pm"}]}]},"customers":[{"id":6001,"name":"Alice Johnson","email":"alice.johnson@example.com","orders":[{"orderId":7001,"date":"2024-09-01","status":"Shipped","items":[{"productId":1001,"quantity":10},{"productId":2001,"quantity":2}],"total":12.48},{"orderId":7002,"date":"2024-09-15","status":"Delivered","items":[{"productId":3001,"quantity":1}],"total":999.99}]},{"id":6002,"name":"Bob Williams","email":"bob.williams@example.com","orders":[{"orderId":7003,"date":"2024-09-07","status":"Pending","items":[{"productId":1002,"quantity":5},{"productId":4001,"quantity":1}],"total":2403.94}]}]},{"store":{"name":"SuperMart","location":{"address":"123 Market Street","city":"Metropolis","state":"NY","zip":"10001"},"departments":[{"id":1,"name":"Groceries","categories":[{"id":101,"name":"Fruits & Vegetables","products":[{"id":1001,"name":"Apple","price":0.99,"stock":120,"details":{"weight":"1lb","origin":"USA","organic":true}},{"id":1002,"name":"Banana","price":0.39,"stock":200,"details":{"weight":"1lb","origin":"Ecuador","organic":false}}]},{"id":102,"name":"Dairy","products":[{"id":2001,"name":"Milk","price":3.49,"stock":50,"details":{"volume":"1 gallon","fatContent":"2%","brand":"DairyPure"}},{"id":2002,"name":"Cheddar Cheese","price":5.99,"stock":75,"details":{"weight":"8oz","origin":"Wisconsin","aged":"2 years"}}]}]},{"id":2,"name":"Electronics","categories":[{"id":201,"name":"Smartphones","products":[{"id":3001,"name":"iPhone 14","price":999.99,"stock":30,"details":{"color":"Black","storage":"128GB","carrier":"Unlocked"}},{"id":3002,"name":"Samsung Galaxy S21","price":849.99,"stock":25,"details":{"color":"White","storage":"256GB","carrier":"Unlocked"}}]},{"id":202,"name":"Laptops","products":[{"id":4001,"name":"MacBook Pro","price":2399.99,"stock":10,"details":{"screenSize":"16 inch","processor":"M1","ram":"16GB","storage":"512GB SSD"}},{"id":4002,"name":"Dell XPS 13","price":1299.99,"stock":15,"details":{"screenSize":"13 inch","processor":"Intel i7","ram":"16GB","storage":"256GB SSD"}}]}]}],"employees":[{"id":5001,"name":"John Doe","position":"Manager","schedule":[{"day":"Monday","shift":"9am-5pm"},{"day":"Wednesday","shift":"9am-5pm"},{"day":"Friday","shift":"9am-5pm"}]},{"id":5002,"name":"Jane Smith","position":"Cashier","schedule":[{"day":"Tuesday","shift":"10am-6pm"},{"day":"Thursday","shift":"10am-6pm"},{"day":"Saturday","shift":"12pm-8pm"}]}]},"customers":[{"id":6001,"name":"Alice Johnson","email":"alice.johnson@example.com","orders":[{"orderId":7001,"date":"2024-09-01","status":"Shipped","items":[{"productId":1001,"quantity":10},{"productId":2001,"quantity":2}],"total":12.48},{"orderId":7002,"date":"2024-09-15","status":"Delivered","items":[{"productId":3001,"quantity":1}],"total":999.99}]},{"id":6002,"name":"Bob Williams","email":"bob.williams@example.com","orders":[{"orderId":7003,"date":"2024-09-07","status":"Pending","items":[{"productId":1002,"quantity":5},{"productId":4001,"quantity":1}],"total":2403.94}]}]},{"store":{"name":"SuperMart","location":{"address":"123 Market Street","city":"Metropolis","state":"NY","zip":"10001"},"departments":[{"id":1,"name":"Groceries","categories":[{"id":101,"name":"Fruits & Vegetables","products":[{"id":1001,"name":"Apple","price":0.99,"stock":120,"details":{"weight":"1lb","origin":"USA","organic":true}},{"id":1002,"name":"Banana","price":0.39,"stock":200,"details":{"weight":"1lb","origin":"Ecuador","organic":false}}]},{"id":102,"name":"Dairy","products":[{"id":2001,"name":"Milk","price":3.49,"stock":50,"details":{"volume":"1 gallon","fatContent":"2%","brand":"DairyPure"}},{"id":2002,"name":"Cheddar Cheese","price":5.99,"stock":75,"details":{"weight":"8oz","origin":"Wisconsin","aged":"2 years"}}]}]},{"id":2,"name":"Electronics","categories":[{"id":201,"name":"Smartphones","products":[{"id":3001,"name":"iPhone 14","price":999.99,"stock":30,"details":{"color":"Black","storage":"128GB","carrier":"Unlocked"}},{"id":3002,"name":"Samsung Galaxy S21","price":849.99,"stock":25,"details":{"color":"White","storage":"256GB","carrier":"Unlocked"}}]},{"id":202,"name":"Laptops","products":[{"id":4001,"name":"MacBook Pro","price":2399.99,"stock":10,"details":{"screenSize":"16 inch","processor":"M1","ram":"16GB","storage":"512GB SSD"}},{"id":4002,"name":"Dell XPS 13","price":1299.99,"stock":15,"details":{"screenSize":"13 inch","processor":"Intel i7","ram":"16GB","storage":"256GB SSD"}}]}]}],"employees":[{"id":5001,"name":"John Doe","position":"Manager","schedule":[{"day":"Monday","shift":"9am-5pm"},{"day":"Wednesday","shift":"9am-5pm"},{"day":"Friday","shift":"9am-5pm"}]},{"id":5002,"name":"Jane Smith","position":"Cashier","schedule":[{"day":"Tuesday","shift":"10am-6pm"},{"day":"Thursday","shift":"10am-6pm"},{"day":"Saturday","shift":"12pm-8pm"}]}]},"customers":[{"id":6001,"name":"Alice Johnson","email":"alice.johnson@example.com","orders":[{"orderId":7001,"date":"2024-09-01","status":"Shipped","items":[{"productId":1001,"quantity":10},{"productId":2001,"quantity":2}],"total":12.48},{"orderId":7002,"date":"2024-09-15","status":"Delivered","items":[{"productId":3001,"quantity":1}],"total":999.99}]},{"id":6002,"name":"Bob Williams","email":"bob.williams@example.com","orders":[{"orderId":7003,"date":"2024-09-07","status":"Pending","items":[{"productId":1002,"quantity":5},{"productId":4001,"quantity":1}],"total":2403.94}]}]},{"store":{"name":"SuperMart","location":{"address":"123 Market Street","city":"Metropolis","state":"NY","zip":"10001"},"departments":[{"id":1,"name":"Groceries","categories":[{"id":101,"name":"Fruits & Vegetables","products":[{"id":1001,"name":"Apple","price":0.99,"stock":120,"details":{"weight":"1lb","origin":"USA","organic":true}},{"id":1002,"name":"Banana","price":0.39,"stock":200,"details":{"weight":"1lb","origin":"Ecuador","organic":false}}]},{"id":102,"name":"Dairy","products":[{"id":2001,"name":"Milk","price":3.49,"stock":50,"details":{"volume":"1 gallon","fatContent":"2%","brand":"DairyPure"}},{"id":2002,"name":"Cheddar Cheese","price":5.99,"stock":75,"details":{"weight":"8oz","origin":"Wisconsin","aged":"2 years"}}]}]},{"id":2,"name":"Electronics","categories":[{"id":201,"name":"Smartphones","products":[{"id":3001,"name":"iPhone 14","price":999.99,"stock":30,"details":{"color":"Black","storage":"128GB","carrier":"Unlocked"}},{"id":3002,"name":"Samsung Galaxy S21","price":849.99,"stock":25,"details":{"color":"White","storage":"256GB","carrier":"Unlocked"}}]},{"id":202,"name":"Laptops","products":[{"id":4001,"name":"MacBook Pro","price":2399.99,"stock":10,"details":{"screenSize":"16 inch","processor":"M1","ram":"16GB","storage":"512GB SSD"}},{"id":4002,"name":"Dell XPS 13","price":1299.99,"stock":15,"details":{"screenSize":"13 inch","processor":"Intel i7","ram":"16GB","storage":"256GB SSD"}}]}]}],"employees":[{"id":5001,"name":"John Doe","position":"Manager","schedule":[{"day":"Monday","shift":"9am-5pm"},{"day":"Wednesday","shift":"9am-5pm"},{"day":"Friday","shift":"9am-5pm"}]},{"id":5002,"name":"Jane Smith","position":"Cashier","schedule":[{"day":"Tuesday","shift":"10am-6pm"},{"day":"Thursday","shift":"10am-6pm"},{"day":"Saturday","shift":"12pm-8pm"}]}]},"customers":[{"id":6001,"name":"Alice Johnson","email":"alice.johnson@example.com","orders":[{"orderId":7001,"date":"2024-09-01","status":"Shipped","items":[{"productId":1001,"quantity":10},{"productId":2001,"quantity":2}],"total":12.48},{"orderId":7002,"date":"2024-09-15","status":"Delivered","items":[{"productId":3001,"quantity":1}],"total":999.99}]},{"id":6002,"name":"Bob Williams","email":"bob.williams@example.com","orders":[{"orderId":7003,"date":"2024-09-07","status":"Pending","items":[{"productId":1002,"quantity":5},{"productId":4001,"quantity":1}],"total":2403.94}]}]},{"store":{"name":"SuperMart","location":{"address":"123 Market Street","city":"Metropolis","state":"NY","zip":"10001"},"departments":[{"id":1,"name":"Groceries","categories":[{"id":101,"name":"Fruits & Vegetables","products":[{"id":1001,"name":"Apple","price":0.99,"stock":120,"details":{"weight":"1lb","origin":"USA","organic":true}},{"id":1002,"name":"Banana","price":0.39,"stock":200,"details":{"weight":"1lb","origin":"Ecuador","organic":false}}]},{"id":102,"name":"Dairy","products":[{"id":2001,"name":"Milk","price":3.49,"stock":50,"details":{"volume":"1 gallon","fatContent":"2%","brand":"DairyPure"}},{"id":2002,"name":"Cheddar Cheese","price":5.99,"stock":75,"details":{"weight":"8oz","origin":"Wisconsin","aged":"2 years"}}]}]},{"id":2,"name":"Electronics","categories":[{"id":201,"name":"Smartphones","products":[{"id":3001,"name":"iPhone 14","price":999.99,"stock":30,"details":{"color":"Black","storage":"128GB","carrier":"Unlocked"}},{"id":3002,"name":"Samsung Galaxy S21","price":849.99,"stock":25,"details":{"color":"White","storage":"256GB","carrier":"Unlocked"}}]},{"id":202,"name":"Laptops","products":[{"id":4001,"name":"MacBook Pro","price":2399.99,"stock":10,"details":{"screenSize":"16 inch","processor":"M1","ram":"16GB","storage":"512GB SSD"}},{"id":4002,"name":"Dell XPS 13","price":1299.99,"stock":15,"details":{"screenSize":"13 inch","processor":"Intel i7","ram":"16GB","storage":"256GB SSD"}}]}]}],"employees":[{"id":5001,"name":"John Doe","position":"Manager","schedule":[{"day":"Monday","shift":"9am-5pm"},{"day":"Wednesday","shift":"9am-5pm"},{"day":"Friday","shift":"9am-5pm"}]},{"id":5002,"name":"Jane Smith","position":"Cashier","schedule":[{"day":"Tuesday","shift":"10am-6pm"},{"day":"Thursday","shift":"10am-6pm"},{"day":"Saturday","shift":"12pm-8pm"}]}]},"customers":[{"id":6001,"name":"Alice Johnson","email":"alice.johnson@example.com","orders":[{"orderId":7001,"date":"2024-09-01","status":"Shipped","items":[{"productId":1001,"quantity":10},{"productId":2001,"quantity":2}],"total":12.48},{"orderId":7002,"date":"2024-09-15","status":"Delivered","items":[{"productId":3001,"quantity":1}],"total":999.99}]},{"id":6002,"name":"Bob Williams","email":"bob.williams@example.com","orders":[{"orderId":7003,"date":"2024-09-07","status":"Pending","items":[{"productId":1002,"quantity":5},{"productId":4001,"quantity":1}],"total":2403.94}]}]},{"store":{"name":"SuperMart","location":{"address":"123 Market Street","city":"Metropolis","state":"NY","zip":"10001"},"departments":[{"id":1,"name":"Groceries","categories":[{"id":101,"name":"Fruits & Vegetables","products":[{"id":1001,"name":"Apple","price":0.99,"stock":120,"details":{"weight":"1lb","origin":"USA","organic":true}},{"id":1002,"name":"Banana","price":0.39,"stock":200,"details":{"weight":"1lb","origin":"Ecuador","organic":false}}]},{"id":102,"name":"Dairy","products":[{"id":2001,"name":"Milk","price":3.49,"stock":50,"details":{"volume":"1 gallon","fatContent":"2%","brand":"DairyPure"}},{"id":2002,"name":"Cheddar Cheese","price":5.99,"stock":75,"details":{"weight":"8oz","origin":"Wisconsin","aged":"2 years"}}]}]},{"id":2,"name":"Electronics","categories":[{"id":201,"name":"Smartphones","products":[{"id":3001,"name":"iPhone 14","price":999.99,"stock":30,"details":{"color":"Black","storage":"128GB","carrier":"Unlocked"}},{"id":3002,"name":"Samsung Galaxy S21","price":849.99,"stock":25,"details":{"color":"White","storage":"256GB","carrier":"Unlocked"}}]},{"id":202,"name":"Laptops","products":[{"id":4001,"name":"MacBook Pro","price":2399.99,"stock":10,"details":{"screenSize":"16 inch","processor":"M1","ram":"16GB","storage":"512GB SSD"}},{"id":4002,"name":"Dell XPS 13","price":1299.99,"stock":15,"details":{"screenSize":"13 inch","processor":"Intel i7","ram":"16GB","storage":"256GB SSD"}}]}]}],"employees":[{"id":5001,"name":"John Doe","position":"Manager","schedule":[{"day":"Monday","shift":"9am-5pm"},{"day":"Wednesday","shift":"9am-5pm"},{"day":"Friday","shift":"9am-5pm"}]},{"id":5002,"name":"Jane Smith","position":"Cashier","schedule":[{"day":"Tuesday","shift":"10am-6pm"},{"day":"Thursday","shift":"10am-6pm"},{"day":"Saturday","shift":"12pm-8pm"}]}]},"customers":[{"id":6001,"name":"Alice Johnson","email":"alice.johnson@example.com","orders":[{"orderId":7001,"date":"2024-09-01","status":"Shipped","items":[{"productId":1001,"quantity":10},{"productId":2001,"quantity":2}],"total":12.48},{"orderId":7002,"date":"2024-09-15","status":"Delivered","items":[{"productId":3001,"quantity":1}],"total":999.99}]},{"id":6002,"name":"Bob Williams","email":"bob.williams@example.com","orders":[{"orderId":7003,"date":"2024-09-07","status":"Pending","items":[{"productId":1002,"quantity":5},{"productId":4001,"quantity":1}],"total":2403.94}]}]},{"store":{"name":"SuperMart","location":{"address":"123 Market Street","city":"Metropolis","state":"NY","zip":"10001"},"departments":[{"id":1,"name":"Groceries","categories":[{"id":101,"name":"Fruits & Vegetables","products":[{"id":1001,"name":"Apple","price":0.99,"stock":120,"details":{"weight":"1lb","origin":"USA","organic":true}},{"id":1002,"name":"Banana","price":0.39,"stock":200,"details":{"weight":"1lb","origin":"Ecuador","organic":false}}]},{"id":102,"name":"Dairy","products":[{"id":2001,"name":"Milk","price":3.49,"stock":50,"details":{"volume":"1 gallon","fatContent":"2%","brand":"DairyPure"}},{"id":2002,"name":"Cheddar Cheese","price":5.99,"stock":75,"details":{"weight":"8oz","origin":"Wisconsin","aged":"2 years"}}]}]},{"id":2,"name":"Electronics","categories":[{"id":201,"name":"Smartphones","products":[{"id":3001,"name":"iPhone 14","price":999.99,"stock":30,"details":{"color":"Black","storage":"128GB","carrier":"Unlocked"}},{"id":3002,"name":"Samsung Galaxy S21","price":849.99,"stock":25,"details":{"color":"White","storage":"256GB","carrier":"Unlocked"}}]},{"id":202,"name":"Laptops","products":[{"id":4001,"name":"MacBook Pro","price":2399.99,"stock":10,"details":{"screenSize":"16 inch","processor":"M1","ram":"16GB","storage":"512GB SSD"}},{"id":4002,"name":"Dell XPS 13","price":1299.99,"stock":15,"details":{"screenSize":"13 inch","processor":"Intel i7","ram":"16GB","storage":"256GB SSD"}}]}]}],"employees":[{"id":5001,"name":"John Doe","position":"Manager","schedule":[{"day":"Monday","shift":"9am-5pm"},{"day":"Wednesday","shift":"9am-5pm"},{"day":"Friday","shift":"9am-5pm"}]},{"id":5002,"name":"Jane Smith","position":"Cashier","schedule":[{"day":"Tuesday","shift":"10am-6pm"},{"day":"Thursday","shift":"10am-6pm"},{"day":"Saturday","shift":"12pm-8pm"}]}]},"customers":[{"id":6001,"name":"Alice Johnson","email":"alice.johnson@example.com","orders":[{"orderId":7001,"date":"2024-09-01","status":"Shipped","items":[{"productId":1001,"quantity":10},{"productId":2001,"quantity":2}],"total":12.48},{"orderId":7002,"date":"2024-09-15","status":"Delivered","items":[{"productId":3001,"quantity":1}],"total":999.99}]},{"id":6002,"name":"Bob Williams","email":"bob.williams@example.com","orders":[{"orderId":7003,"date":"2024-09-07","status":"Pending","items":[{"productId":1002,"quantity":5},{"productId":4001,"quantity":1}],"total":2403.94}]}]},{"store":{"name":"SuperMart","location":{"address":"123 Market Street","city":"Metropolis","state":"NY","zip":"10001"},"departments":[{"id":1,"name":"Groceries","categories":[{"id":101,"name":"Fruits & Vegetables","products":[{"id":1001,"name":"Apple","price":0.99,"stock":120,"details":{"weight":"1lb","origin":"USA","organic":true}},{"id":1002,"name":"Banana","price":0.39,"stock":200,"details":{"weight":"1lb","origin":"Ecuador","organic":false}}]},{"id":102,"name":"Dairy","products":[{"id":2001,"name":"Milk","price":3.49,"stock":50,"details":{"volume":"1 gallon","fatContent":"2%","brand":"DairyPure"}},{"id":2002,"name":"Cheddar Cheese","price":5.99,"stock":75,"details":{"weight":"8oz","origin":"Wisconsin","aged":"2 years"}}]}]},{"id":2,"name":"Electronics","categories":[{"id":201,"name":"Smartphones","products":[{"id":3001,"name":"iPhone 14","price":999.99,"stock":30,"details":{"color":"Black","storage":"128GB","carrier":"Unlocked"}},{"id":3002,"name":"Samsung Galaxy S21","price":849.99,"stock":25,"details":{"color":"White","storage":"256GB","carrier":"Unlocked"}}]},{"id":202,"name":"Laptops","products":[{"id":4001,"name":"MacBook Pro","price":2399.99,"stock":10,"details":{"screenSize":"16 inch","processor":"M1","ram":"16GB","storage":"512GB SSD"}},{"id":4002,"name":"Dell XPS 13","price":1299.99,"stock":15,"details":{"screenSize":"13 inch","processor":"Intel i7","ram":"16GB","storage":"256GB SSD"}}]}]}],"employees":[{"id":5001,"name":"John Doe","position":"Manager","schedule":[{"day":"Monday","shift":"9am-5pm"},{"day":"Wednesday","shift":"9am-5pm"},{"day":"Friday","shift":"9am-5pm"}]},{"id":5002,"name":"Jane Smith","position":"Cashier","schedule":[{"day":"Tuesday","shift":"10am-6pm"},{"day":"Thursday","shift":"10am-6pm"},{"day":"Saturday","shift":"12pm-8pm"}]}]},"customers":[{"id":6001,"name":"Alice Johnson","email":"alice.johnson@example.com","orders":[{"orderId":7001,"date":"2024-09-01","status":"Shipped","items":[{"productId":1001,"quantity":10},{"productId":2001,"quantity":2}],"total":12.48},{"orderId":7002,"date":"2024-09-15","status":"Delivered","items":[{"productId":3001,"quantity":1}],"total":999.99}]},{"id":6002,"name":"Bob Williams","email":"bob.williams@example.com","orders":[{"orderId":7003,"date":"2024-09-07","status":"Pending","items":[{"productId":1002,"quantity":5},{"productId":4001,"quantity":1}],"total":2403.94}]}]},{"store":{"name":"SuperMart","location":{"address":"123 Market Street","city":"Metropolis","state":"NY","zip":"10001"},"departments":[{"id":1,"name":"Groceries","categories":[{"id":101,"name":"Fruits & Vegetables","products":[{"id":1001,"name":"Apple","price":0.99,"stock":120,"details":{"weight":"1lb","origin":"USA","organic":true}},{"id":1002,"name":"Banana","price":0.39,"stock":200,"details":{"weight":"1lb","origin":"Ecuador","organic":false}}]},{"id":102,"name":"Dairy","products":[{"id":2001,"name":"Milk","price":3.49,"stock":50,"details":{"volume":"1 gallon","fatContent":"2%","brand":"DairyPure"}},{"id":2002,"name":"Cheddar Cheese","price":5.99,"stock":75,"details":{"weight":"8oz","origin":"Wisconsin","aged":"2 years"}}]}]},{"id":2,"name":"Electronics","categories":[{"id":201,"name":"Smartphones","products":[{"id":3001,"name":"iPhone 14","price":999.99,"stock":30,"details":{"color":"Black","storage":"128GB","carrier":"Unlocked"}},{"id":3002,"name":"Samsung Galaxy S21","price":849.99,"stock":25,"details":{"color":"White","storage":"256GB","carrier":"Unlocked"}}]},{"id":202,"name":"Laptops","products":[{"id":4001,"name":"MacBook Pro","price":2399.99,"stock":10,"details":{"screenSize":"16 inch","processor":"M1","ram":"16GB","storage":"512GB SSD"}},{"id":4002,"name":"Dell XPS 13","price":1299.99,"stock":15,"details":{"screenSize":"13 inch","processor":"Intel i7","ram":"16GB","storage":"256GB SSD"}}]}]}],"employees":[{"id":5001,"name":"John Doe","position":"Manager","schedule":[{"day":"Monday","shift":"9am-5pm"},{"day":"Wednesday","shift":"9am-5pm"},{"day":"Friday","shift":"9am-5pm"}]},{"id":5002,"name":"Jane Smith","position":"Cashier","schedule":[{"day":"Tuesday","shift":"10am-6pm"},{"day":"Thursday","shift":"10am-6pm"},{"day":"Saturday","shift":"12pm-8pm"}]}]},"customers":[{"id":6001,"name":"Alice Johnson","email":"alice.johnson@example.com","orders":[{"orderId":7001,"date":"2024-09-01","status":"Shipped","items":[{"productId":1001,"quantity":10},{"productId":2001,"quantity":2}],"total":12.48},{"orderId":7002,"date":"2024-09-15","status":"Delivered","items":[{"productId":3001,"quantity":1}],"total":999.99}]},{"id":6002,"name":"Bob Williams","email":"bob.williams@example.com","orders":[{"orderId":7003,"date":"2024-09-07","status":"Pending","items":[{"productId":1002,"quantity":5},{"productId":4001,"quantity":1}],"total":2403.94}]}]},{"store":{"name":"SuperMart","location":{"address":"123 Market Street","city":"Metropolis","state":"NY","zip":"10001"},"departments":[{"id":1,"name":"Groceries","categories":[{"id":101,"name":"Fruits & Vegetables","products":[{"id":1001,"name":"Apple","price":0.99,"stock":120,"details":{"weight":"1lb","origin":"USA","organic":true}},{"id":1002,"name":"Banana","price":0.39,"stock":200,"details":{"weight":"1lb","origin":"Ecuador","organic":false}}]},{"id":102,"name":"Dairy","products":[{"id":2001,"name":"Milk","price":3.49,"stock":50,"details":{"volume":"1 gallon","fatContent":"2%","brand":"DairyPure"}},{"id":2002,"name":"Cheddar Cheese","price":5.99,"stock":75,"details":{"weight":"8oz","origin":"Wisconsin","aged":"2 years"}}]}]},{"id":2,"name":"Electronics","categories":[{"id":201,"name":"Smartphones","products":[{"id":3001,"name":"iPhone 14","price":999.99,"stock":30,"details":{"color":"Black","storage":"128GB","carrier":"Unlocked"}},{"id":3002,"name":"Samsung Galaxy S21","price":849.99,"stock":25,"details":{"color":"White","storage":"256GB","carrier":"Unlocked"}}]},{"id":202,"name":"Laptops","products":[{"id":4001,"name":"MacBook Pro","price":2399.99,"stock":10,"details":{"screenSize":"16 inch","processor":"M1","ram":"16GB","storage":"512GB SSD"}},{"id":4002,"name":"Dell XPS 13","price":1299.99,"stock":15,"details":{"screenSize":"13 inch","processor":"Intel i7","ram":"16GB","storage":"256GB SSD"}}]}]}],"employees":[{"id":5001,"name":"John Doe","position":"Manager","schedule":[{"day":"Monday","shift":"9am-5pm"},{"day":"Wednesday","shift":"9am-5pm"},{"day":"Friday","shift":"9am-5pm"}]},{"id":5002,"name":"Jane Smith","position":"Cashier","schedule":[{"day":"Tuesday","shift":"10am-6pm"},{"day":"Thursday","shift":"10am-6pm"},{"day":"Saturday","shift":"12pm-8pm"}]}]},"customers":[{"id":6001,"name":"Alice Johnson","email":"alice.johnson@example.com","orders":[{"orderId":7001,"date":"2024-09-01","status":"Shipped","items":[{"productId":1001,"quantity":10},{"productId":2001,"quantity":2}],"total":12.48},{"orderId":7002,"date":"2024-09-15","status":"Delivered","items":[{"productId":3001,"quantity":1}],"total":999.99}]},{"id":6002,"name":"Bob Williams","email":"bob.williams@example.com","orders":[{"orderId":7003,"date":"2024-09-07","status":"Pending","items":[{"productId":1002,"quantity":5},{"productId":4001,"quantity":1}],"total":2403.94}]}]},{"store":{"name":"SuperMart","location":{"address":"123 Market Street","city":"Metropolis","state":"NY","zip":"10001"},"departments":[{"id":1,"name":"Groceries","categories":[{"id":101,"name":"Fruits & Vegetables","products":[{"id":1001,"name":"Apple","price":0.99,"stock":120,"details":{"weight":"1lb","origin":"USA","organic":true}},{"id":1002,"name":"Banana","price":0.39,"stock":200,"details":{"weight":"1lb","origin":"Ecuador","organic":false}}]},{"id":102,"name":"Dairy","products":[{"id":2001,"name":"Milk","price":3.49,"stock":50,"details":{"volume":"1 gallon","fatContent":"2%","brand":"DairyPure"}},{"id":2002,"name":"Cheddar Cheese","price":5.99,"stock":75,"details":{"weight":"8oz","origin":"Wisconsin","aged":"2 years"}}]}]},{"id":2,"name":"Electronics","categories":[{"id":201,"name":"Smartphones","products":[{"id":3001,"name":"iPhone 14","price":999.99,"stock":30,"details":{"color":"Black","storage":"128GB","carrier":"Unlocked"}},{"id":3002,"name":"Samsung Galaxy S21","price":849.99,"stock":25,"details":{"color":"White","storage":"256GB","carrier":"Unlocked"}}]},{"id":202,"name":"Laptops","products":[{"id":4001,"name":"MacBook Pro","price":2399.99,"stock":10,"details":{"screenSize":"16 inch","processor":"M1","ram":"16GB","storage":"512GB SSD"}},{"id":4002,"name":"Dell XPS 13","price":1299.99,"stock":15,"details":{"screenSize":"13 inch","processor":"Intel i7","ram":"16GB","storage":"256GB SSD"}}]}]}],"employees":[{"id":5001,"name":"John Doe","position":"Manager","schedule":[{"day":"Monday","shift":"9am-5pm"},{"day":"Wednesday","shift":"9am-5pm"},{"day":"Friday","shift":"9am-5pm"}]},{"id":5002,"name":"Jane Smith","position":"Cashier","schedule":[{"day":"Tuesday","shift":"10am-6pm"},{"day":"Thursday","shift":"10am-6pm"},{"day":"Saturday","shift":"12pm-8pm"}]}]},"customers":[{"id":6001,"name":"Alice Johnson","email":"alice.johnson@example.com","orders":[{"orderId":7001,"date":"2024-09-01","status":"Shipped","items":[{"productId":1001,"quantity":10},{"productId":2001,"quantity":2}],"total":12.48},{"orderId":7002,"date":"2024-09-15","status":"Delivered","items":[{"productId":3001,"quantity":1}],"total":999.99}]},{"id":6002,"name":"Bob Williams","email":"bob.williams@example.com","orders":[{"orderId":7003,"date":"2024-09-07","status":"Pending","items":[{"productId":1002,"quantity":5},{"productId":4001,"quantity":1}],"total":2403.94}]}]},{"store":{"name":"SuperMart","location":{"address":"123 Market Street","city":"Metropolis","state":"NY","zip":"10001"},"departments":[{"id":1,"name":"Groceries","categories":[{"id":101,"name":"Fruits & Vegetables","products":[{"id":1001,"name":"Apple","price":0.99,"stock":120,"details":{"weight":"1lb","origin":"USA","organic":true}},{"id":1002,"name":"Banana","price":0.39,"stock":200,"details":{"weight":"1lb","origin":"Ecuador","organic":false}}]},{"id":102,"name":"Dairy","products":[{"id":2001,"name":"Milk","price":3.49,"stock":50,"details":{"volume":"1 gallon","fatContent":"2%","brand":"DairyPure"}},{"id":2002,"name":"Cheddar Cheese","price":5.99,"stock":75,"details":{"weight":"8oz","origin":"Wisconsin","aged":"2 years"}}]}]},{"id":2,"name":"Electronics","categories":[{"id":201,"name":"Smartphones","products":[{"id":3001,"name":"iPhone 14","price":999.99,"stock":30,"details":{"color":"Black","storage":"128GB","carrier":"Unlocked"}},{"id":3002,"name":"Samsung Galaxy S21","price":849.99,"stock":25,"details":{"color":"White","storage":"256GB","carrier":"Unlocked"}}]},{"id":202,"name":"Laptops","products":[{"id":4001,"name":"MacBook Pro","price":2399.99,"stock":10,"details":{"screenSize":"16 inch","processor":"M1","ram":"16GB","storage":"512GB SSD"}},{"id":4002,"name":"Dell XPS 13","price":1299.99,"stock":15,"details":{"screenSize":"13 inch","processor":"Intel i7","ram":"16GB","storage":"256GB SSD"}}]}]}],"employees":[{"id":5001,"name":"John Doe","position":"Manager","schedule":[{"day":"Monday","shift":"9am-5pm"},{"day":"Wednesday","shift":"9am-5pm"},{"day":"Friday","shift":"9am-5pm"}]},{"id":5002,"name":"Jane Smith","position":"Cashier","schedule":[{"day":"Tuesday","shift":"10am-6pm"},{"day":"Thursday","shift":"10am-6pm"},{"day":"Saturday","shift":"12pm-8pm"}]}]},"customers":[{"id":6001,"name":"Alice Johnson","email":"alice.johnson@example.com","orders":[{"orderId":7001,"date":"2024-09-01","status":"Shipped","items":[{"productId":1001,"quantity":10},{"productId":2001,"quantity":2}],"total":12.48},{"orderId":7002,"date":"2024-09-15","status":"Delivered","items":[{"productId":3001,"quantity":1}],"total":999.99}]},{"id":6002,"name":"Bob Williams","email":"bob.williams@example.com","orders":[{"orderId":7003,"date":"2024-09-07","status":"Pending","items":[{"productId":1002,"quantity":5},{"productId":4001,"quantity":1}],"total":2403.94}]}]},{"store":{"name":"SuperMart","location":{"address":"123 Market Street","city":"Metropolis","state":"NY","zip":"10001"},"departments":[{"id":1,"name":"Groceries","categories":[{"id":101,"name":"Fruits & Vegetables","products":[{"id":1001,"name":"Apple","price":0.99,"stock":120,"details":{"weight":"1lb","origin":"USA","organic":true}},{"id":1002,"name":"Banana","price":0.39,"stock":200,"details":{"weight":"1lb","origin":"Ecuador","organic":false}}]},{"id":102,"name":"Dairy","products":[{"id":2001,"name":"Milk","price":3.49,"stock":50,"details":{"volume":"1 gallon","fatContent":"2%","brand":"DairyPure"}},{"id":2002,"name":"Cheddar Cheese","price":5.99,"stock":75,"details":{"weight":"8oz","origin":"Wisconsin","aged":"2 years"}}]}]},{"id":2,"name":"Electronics","categories":[{"id":201,"name":"Smartphones","products":[{"id":3001,"name":"iPhone 14","price":999.99,"stock":30,"details":{"color":"Black","storage":"128GB","carrier":"Unlocked"}},{"id":3002,"name":"Samsung Galaxy S21","price":849.99,"stock":25,"details":{"color":"White","storage":"256GB","carrier":"Unlocked"}}]},{"id":202,"name":"Laptops","products":[{"id":4001,"name":"MacBook Pro","price":2399.99,"stock":10,"details":{"screenSize":"16 inch","processor":"M1","ram":"16GB","storage":"512GB SSD"}},{"id":4002,"name":"Dell XPS 13","price":1299.99,"stock":15,"details":{"screenSize":"13 inch","processor":"Intel i7","ram":"16GB","storage":"256GB SSD"}}]}]}],"employees":[{"id":5001,"name":"John Doe","position":"Manager","schedule":[{"day":"Monday","shift":"9am-5pm"},{"day":"Wednesday","shift":"9am-5pm"},{"day":"Friday","shift":"9am-5pm"}]},{"id":5002,"name":"Jane Smith","position":"Cashier","schedule":[{"day":"Tuesday","shift":"10am-6pm"},{"day":"Thursday","shift":"10am-6pm"},{"day":"Saturday","shift":"12pm-8pm"}]}]},"customers":[{"id":6001,"name":"Alice Johnson","email":"alice.johnson@example.com","orders":[{"orderId":7001,"date":"2024-09-01","status":"Shipped","items":[{"productId":1001,"quantity":10},{"productId":2001,"quantity":2}],"total":12.48},{"orderId":7002,"date":"2024-09-15","status":"Delivered","items":[{"productId":3001,"quantity":1}],"total":999.99}]},{"id":6002,"name":"Bob Williams","email":"bob.williams@example.com","orders":[{"orderId":7003,"date":"2024-09-07","status":"Pending","items":[{"productId":1002,"quantity":5},{"productId":4001,"quantity":1}],"total":2403.94}]}]},{"store":{"name":"SuperMart","location":{"address":"123 Market Street","city":"Metropolis","state":"NY","zip":"10001"},"departments":[{"id":1,"name":"Groceries","categories":[{"id":101,"name":"Fruits & Vegetables","products":[{"id":1001,"name":"Apple","price":0.99,"stock":120,"details":{"weight":"1lb","origin":"USA","organic":true}},{"id":1002,"name":"Banana","price":0.39,"stock":200,"details":{"weight":"1lb","origin":"Ecuador","organic":false}}]},{"id":102,"name":"Dairy","products":[{"id":2001,"name":"Milk","price":3.49,"stock":50,"details":{"volume":"1 gallon","fatContent":"2%","brand":"DairyPure"}},{"id":2002,"name":"Cheddar Cheese","price":5.99,"stock":75,"details":{"weight":"8oz","origin":"Wisconsin","aged":"2 years"}}]}]},{"id":2,"name":"Electronics","categories":[{"id":201,"name":"Smartphones","products":[{"id":3001,"name":"iPhone 14","price":999.99,"stock":30,"details":{"color":"Black","storage":"128GB","carrier":"Unlocked"}},{"id":3002,"name":"Samsung Galaxy S21","price":849.99,"stock":25,"details":{"color":"White","storage":"256GB","carrier":"Unlocked"}}]},{"id":202,"name":"Laptops","products":[{"id":4001,"name":"MacBook Pro","price":2399.99,"stock":10,"details":{"screenSize":"16 inch","processor":"M1","ram":"16GB","storage":"512GB SSD"}},{"id":4002,"name":"Dell XPS 13","price":1299.99,"stock":15,"details":{"screenSize":"13 inch","processor":"Intel i7","ram":"16GB","storage":"256GB SSD"}}]}]}],"employees":[{"id":5001,"name":"John Doe","position":"Manager","schedule":[{"day":"Monday","shift":"9am-5pm"},{"day":"Wednesday","shift":"9am-5pm"},{"day":"Friday","shift":"9am-5pm"}]},{"id":5002,"name":"Jane Smith","position":"Cashier","schedule":[{"day":"Tuesday","shift":"10am-6pm"},{"day":"Thursday","shift":"10am-6pm"},{"day":"Saturday","shift":"12pm-8pm"}]}]},"customers":[{"id":6001,"name":"Alice Johnson","email":"alice.johnson@example.com","orders":[{"orderId":7001,"date":"2024-09-01","status":"Shipped","items":[{"productId":1001,"quantity":10},{"productId":2001,"quantity":2}],"total":12.48},{"orderId":7002,"date":"2024-09-15","status":"Delivered","items":[{"productId":3001,"quantity":1}],"total":999.99}]},{"id":6002,"name":"Bob Williams","email":"bob.williams@example.com","orders":[{"orderId":7003,"date":"2024-09-07","status":"Pending","items":[{"productId":1002,"quantity":5},{"productId":4001,"quantity":1}],"total":2403.94}]}]},{"store":{"name":"SuperMart","location":{"address":"123 Market Street","city":"Metropolis","state":"NY","zip":"10001"},"departments":[{"id":1,"name":"Groceries","categories":[{"id":101,"name":"Fruits & Vegetables","products":[{"id":1001,"name":"Apple","price":0.99,"stock":120,"details":{"weight":"1lb","origin":"USA","organic":true}},{"id":1002,"name":"Banana","price":0.39,"stock":200,"details":{"weight":"1lb","origin":"Ecuador","organic":false}}]},{"id":102,"name":"Dairy","products":[{"id":2001,"name":"Milk","price":3.49,"stock":50,"details":{"volume":"1 gallon","fatContent":"2%","brand":"DairyPure"}},{"id":2002,"name":"Cheddar Cheese","price":5.99,"stock":75,"details":{"weight":"8oz","origin":"Wisconsin","aged":"2 years"}}]}]},{"id":2,"name":"Electronics","categories":[{"id":201,"name":"Smartphones","products":[{"id":3001,"name":"iPhone 14","price":999.99,"stock":30,"details":{"color":"Black","storage":"128GB","carrier":"Unlocked"}},{"id":3002,"name":"Samsung Galaxy S21","price":849.99,"stock":25,"details":{"color":"White","storage":"256GB","carrier":"Unlocked"}}]},{"id":202,"name":"Laptops","products":[{"id":4001,"name":"MacBook Pro","price":2399.99,"stock":10,"details":{"screenSize":"16 inch","processor":"M1","ram":"16GB","storage":"512GB SSD"}},{"id":4002,"name":"Dell XPS 13","price":1299.99,"stock":15,"details":{"screenSize":"13 inch","processor":"Intel i7","ram":"16GB","storage":"256GB SSD"}}]}]}],"employees":[{"id":5001,"name":"John Doe","position":"Manager","schedule":[{"day":"Monday","shift":"9am-5pm"},{"day":"Wednesday","shift":"9am-5pm"},{"day":"Friday","shift":"9am-5pm"}]},{"id":5002,"name":"Jane Smith","position":"Cashier","schedule":[{"day":"Tuesday","shift":"10am-6pm"},{"day":"Thursday","shift":"10am-6pm"},{"day":"Saturday","shift":"12pm-8pm"}]}]},"customers":[{"id":6001,"name":"Alice Johnson","email":"alice.johnson@example.com","orders":[{"orderId":7001,"date":"2024-09-01","status":"Shipped","items":[{"productId":1001,"quantity":10},{"productId":2001,"quantity":2}],"total":12.48},{"orderId":7002,"date":"2024-09-15","status":"Delivered","items":[{"productId":3001,"quantity":1}],"total":999.99}]},{"id":6002,"name":"Bob Williams","email":"bob.williams@example.com","orders":[{"orderId":7003,"date":"2024-09-07","status":"Pending","items":[{"productId":1002,"quantity":5},{"productId":4001,"quantity":1}],"total":2403.94}]}]},{"store":{"name":"SuperMart","location":{"address":"123 Market Street","city":"Metropolis","state":"NY","zip":"10001"},"departments":[{"id":1,"name":"Groceries","categories":[{"id":101,"name":"Fruits & Vegetables","products":[{"id":1001,"name":"Apple","price":0.99,"stock":120,"details":{"weight":"1lb","origin":"USA","organic":true}},{"id":1002,"name":"Banana","price":0.39,"stock":200,"details":{"weight":"1lb","origin":"Ecuador","organic":false}}]},{"id":102,"name":"Dairy","products":[{"id":2001,"name":"Milk","price":3.49,"stock":50,"details":{"volume":"1 gallon","fatContent":"2%","brand":"DairyPure"}},{"id":2002,"name":"Cheddar Cheese","price":5.99,"stock":75,"details":{"weight":"8oz","origin":"Wisconsin","aged":"2 years"}}]}]},{"id":2,"name":"Electronics","categories":[{"id":201,"name":"Smartphones","products":[{"id":3001,"name":"iPhone 14","price":999.99,"stock":30,"details":{"color":"Black","storage":"128GB","carrier":"Unlocked"}},{"id":3002,"name":"Samsung Galaxy S21","price":849.99,"stock":25,"details":{"color":"White","storage":"256GB","carrier":"Unlocked"}}]},{"id":202,"name":"Laptops","products":[{"id":4001,"name":"MacBook Pro","price":2399.99,"stock":10,"details":{"screenSize":"16 inch","processor":"M1","ram":"16GB","storage":"512GB SSD"}},{"id":4002,"name":"Dell XPS 13","price":1299.99,"stock":15,"details":{"screenSize":"13 inch","processor":"Intel i7","ram":"16GB","storage":"256GB SSD"}}]}]}],"employees":[{"id":5001,"name":"John Doe","position":"Manager","schedule":[{"day":"Monday","shift":"9am-5pm"},{"day":"Wednesday","shift":"9am-5pm"},{"day":"Friday","shift":"9am-5pm"}]},{"id":5002,"name":"Jane Smith","position":"Cashier","schedule":[{"day":"Tuesday","shift":"10am-6pm"},{"day":"Thursday","shift":"10am-6pm"},{"day":"Saturday","shift":"12pm-8pm"}]}]},"customers":[{"id":6001,"name":"Alice Johnson","email":"alice.johnson@example.com","orders":[{"orderId":7001,"date":"2024-09-01","status":"Shipped","items":[{"productId":1001,"quantity":10},{"productId":2001,"quantity":2}],"total":12.48},{"orderId":7002,"date":"2024-09-15","status":"Delivered","items":[{"productId":3001,"quantity":1}],"total":999.99}]},{"id":6002,"name":"Bob Williams","email":"bob.williams@example.com","orders":[{"orderId":7003,"date":"2024-09-07","status":"Pending","items":[{"productId":1002,"quantity":5},{"productId":4001,"quantity":1}],"total":2403.94}]}]},{"store":{"name":"SuperMart","location":{"address":"123 Market Street","city":"Metropolis","state":"NY","zip":"10001"},"departments":[{"id":1,"name":"Groceries","categories":[{"id":101,"name":"Fruits & Vegetables","products":[{"id":1001,"name":"Apple","price":0.99,"stock":120,"details":{"weight":"1lb","origin":"USA","organic":true}},{"id":1002,"name":"Banana","price":0.39,"stock":200,"details":{"weight":"1lb","origin":"Ecuador","organic":false}}]},{"id":102,"name":"Dairy","products":[{"id":2001,"name":"Milk","price":3.49,"stock":50,"details":{"volume":"1 gallon","fatContent":"2%","brand":"DairyPure"}},{"id":2002,"name":"Cheddar Cheese","price":5.99,"stock":75,"details":{"weight":"8oz","origin":"Wisconsin","aged":"2 years"}}]}]},{"id":2,"name":"Electronics","categories":[{"id":201,"name":"Smartphones","products":[{"id":3001,"name":"iPhone 14","price":999.99,"stock":30,"details":{"color":"Black","storage":"128GB","carrier":"Unlocked"}},{"id":3002,"name":"Samsung Galaxy S21","price":849.99,"stock":25,"details":{"color":"White","storage":"256GB","carrier":"Unlocked"}}]},{"id":202,"name":"Laptops","products":[{"id":4001,"name":"MacBook Pro","price":2399.99,"stock":10,"details":{"screenSize":"16 inch","processor":"M1","ram":"16GB","storage":"512GB SSD"}},{"id":4002,"name":"Dell XPS 13","price":1299.99,"stock":15,"details":{"screenSize":"13 inch","processor":"Intel i7","ram":"16GB","storage":"256GB SSD"}}]}]}],"employees":[{"id":5001,"name":"John Doe","position":"Manager","schedule":[{"day":"Monday","shift":"9am-5pm"},{"day":"Wednesday","shift":"9am-5pm"},{"day":"Friday","shift":"9am-5pm"}]},{"id":5002,"name":"Jane Smith","position":"Cashier","schedule":[{"day":"Tuesday","shift":"10am-6pm"},{"day":"Thursday","shift":"10am-6pm"},{"day":"Saturday","shift":"12pm-8pm"}]}]},"customers":[{"id":6001,"name":"Alice Johnson","email":"alice.johnson@example.com","orders":[{"orderId":7001,"date":"2024-09-01","status":"Shipped","items":[{"productId":1001,"quantity":10},{"productId":2001,"quantity":2}],"total":12.48},{"orderId":7002,"date":"2024-09-15","status":"Delivered","items":[{"productId":3001,"quantity":1}],"total":999.99}]},{"id":6002,"name":"Bob Williams","email":"bob.williams@example.com","orders":[{"orderId":7003,"date":"2024-09-07","status":"Pending","items":[{"productId":1002,"quantity":5},{"productId":4001,"quantity":1}],"total":2403.94}]}]},{"store":{"name":"SuperMart","location":{"address":"123 Market Street","city":"Metropolis","state":"NY","zip":"10001"},"departments":[{"id":1,"name":"Groceries","categories":[{"id":101,"name":"Fruits & Vegetables","products":[{"id":1001,"name":"Apple","price":0.99,"stock":120,"details":{"weight":"1lb","origin":"USA","organic":true}},{"id":1002,"name":"Banana","price":0.39,"stock":200,"details":{"weight":"1lb","origin":"Ecuador","organic":false}}]},{"id":102,"name":"Dairy","products":[{"id":2001,"name":"Milk","price":3.49,"stock":50,"details":{"volume":"1 gallon","fatContent":"2%","brand":"DairyPure"}},{"id":2002,"name":"Cheddar Cheese","price":5.99,"stock":75,"details":{"weight":"8oz","origin":"Wisconsin","aged":"2 years"}}]}]},{"id":2,"name":"Electronics","categories":[{"id":201,"name":"Smartphones","products":[{"id":3001,"name":"iPhone 14","price":999.99,"stock":30,"details":{"color":"Black","storage":"128GB","carrier":"Unlocked"}},{"id":3002,"name":"Samsung Galaxy S21","price":849.99,"stock":25,"details":{"color":"White","storage":"256GB","carrier":"Unlocked"}}]},{"id":202,"name":"Laptops","products":[{"id":4001,"name":"MacBook Pro","price":2399.99,"stock":10,"details":{"screenSize":"16 inch","processor":"M1","ram":"16GB","storage":"512GB SSD"}},{"id":4002,"name":"Dell XPS 13","price":1299.99,"stock":15,"details":{"screenSize":"13 inch","processor":"Intel i7","ram":"16GB","storage":"256GB SSD"}}]}]}],"employees":[{"id":5001,"name":"John Doe","position":"Manager","schedule":[{"day":"Monday","shift":"9am-5pm"},{"day":"Wednesday","shift":"9am-5pm"},{"day":"Friday","shift":"9am-5pm"}]},{"id":5002,"name":"Jane Smith","position":"Cashier","schedule":[{"day":"Tuesday","shift":"10am-6pm"},{"day":"Thursday","shift":"10am-6pm"},{"day":"Saturday","shift":"12pm-8pm"}]}]},"customers":[{"id":6001,"name":"Alice Johnson","email":"alice.johnson@example.com","orders":[{"orderId":7001,"date":"2024-09-01","status":"Shipped","items":[{"productId":1001,"quantity":10},{"productId":2001,"quantity":2}],"total":12.48},{"orderId":7002,"date":"2024-09-15","status":"Delivered","items":[{"productId":3001,"quantity":1}],"total":999.99}]},{"id":6002,"name":"Bob Williams","email":"bob.williams@example.com","orders":[{"orderId":7003,"date":"2024-09-07","status":"Pending","items":[{"productId":1002,"quantity":5},{"productId":4001,"quantity":1}],"total":2403.94}]}]},{"store":{"name":"SuperMart","location":{"address":"123 Market Street","city":"Metropolis","state":"NY","zip":"10001"},"departments":[{"id":1,"name":"Groceries","categories":[{"id":101,"name":"Fruits & Vegetables","products":[{"id":1001,"name":"Apple","price":0.99,"stock":120,"details":{"weight":"1lb","origin":"USA","organic":true}},{"id":1002,"name":"Banana","price":0.39,"stock":200,"details":{"weight":"1lb","origin":"Ecuador","organic":false}}]},{"id":102,"name":"Dairy","products":[{"id":2001,"name":"Milk","price":3.49,"stock":50,"details":{"volume":"1 gallon","fatContent":"2%","brand":"DairyPure"}},{"id":2002,"name":"Cheddar Cheese","price":5.99,"stock":75,"details":{"weight":"8oz","origin":"Wisconsin","aged":"2 years"}}]}]},{"id":2,"name":"Electronics","categories":[{"id":201,"name":"Smartphones","products":[{"id":3001,"name":"iPhone 14","price":999.99,"stock":30,"details":{"color":"Black","storage":"128GB","carrier":"Unlocked"}},{"id":3002,"name":"Samsung Galaxy S21","price":849.99,"stock":25,"details":{"color":"White","storage":"256GB","carrier":"Unlocked"}}]},{"id":202,"name":"Laptops","products":[{"id":4001,"name":"MacBook Pro","price":2399.99,"stock":10,"details":{"screenSize":"16 inch","processor":"M1","ram":"16GB","storage":"512GB SSD"}},{"id":4002,"name":"Dell XPS 13","price":1299.99,"stock":15,"details":{"screenSize":"13 inch","processor":"Intel i7","ram":"16GB","storage":"256GB SSD"}}]}]}],"employees":[{"id":5001,"name":"John Doe","position":"Manager","schedule":[{"day":"Monday","shift":"9am-5pm"},{"day":"Wednesday","shift":"9am-5pm"},{"day":"Friday","shift":"9am-5pm"}]},{"id":5002,"name":"Jane Smith","position":"Cashier","schedule":[{"day":"Tuesday","shift":"10am-6pm"},{"day":"Thursday","shift":"10am-6pm"},{"day":"Saturday","shift":"12pm-8pm"}]}]},"customers":[{"id":6001,"name":"Alice Johnson","email":"alice.johnson@example.com","orders":[{"orderId":7001,"date":"2024-09-01","status":"Shipped","items":[{"productId":1001,"quantity":10},{"productId":2001,"quantity":2}],"total":12.48},{"orderId":7002,"date":"2024-09-15","status":"Delivered","items":[{"productId":3001,"quantity":1}],"total":999.99}]},{"id":6002,"name":"Bob Williams","email":"bob.williams@example.com","orders":[{"orderId":7003,"date":"2024-09-07","status":"Pending","items":[{"productId":1002,"quantity":5},{"productId":4001,"quantity":1}],"total":2403.94}]}]},{"store":{"name":"SuperMart","location":{"address":"123 Market Street","city":"Metropolis","state":"NY","zip":"10001"},"departments":[{"id":1,"name":"Groceries","categories":[{"id":101,"name":"Fruits & Vegetables","products":[{"id":1001,"name":"Apple","price":0.99,"stock":120,"details":{"weight":"1lb","origin":"USA","organic":true}},{"id":1002,"name":"Banana","price":0.39,"stock":200,"details":{"weight":"1lb","origin":"Ecuador","organic":false}}]},{"id":102,"name":"Dairy","products":[{"id":2001,"name":"Milk","price":3.49,"stock":50,"details":{"volume":"1 gallon","fatContent":"2%","brand":"DairyPure"}},{"id":2002,"name":"Cheddar Cheese","price":5.99,"stock":75,"details":{"weight":"8oz","origin":"Wisconsin","aged":"2 years"}}]}]},{"id":2,"name":"Electronics","categories":[{"id":201,"name":"Smartphones","products":[{"id":3001,"name":"iPhone 14","price":999.99,"stock":30,"details":{"color":"Black","storage":"128GB","carrier":"Unlocked"}},{"id":3002,"name":"Samsung Galaxy S21","price":849.99,"stock":25,"details":{"color":"White","storage":"256GB","carrier":"Unlocked"}}]},{"id":202,"name":"Laptops","products":[{"id":4001,"name":"MacBook Pro","price":2399.99,"stock":10,"details":{"screenSize":"16 inch","processor":"M1","ram":"16GB","storage":"512GB SSD"}},{"id":4002,"name":"Dell XPS 13","price":1299.99,"stock":15,"details":{"screenSize":"13 inch","processor":"Intel i7","ram":"16GB","storage":"256GB SSD"}}]}]}],"employees":[{"id":5001,"name":"John Doe","position":"Manager","schedule":[{"day":"Monday","shift":"9am-5pm"},{"day":"Wednesday","shift":"9am-5pm"},{"day":"Friday","shift":"9am-5pm"}]},{"id":5002,"name":"Jane Smith","position":"Cashier","schedule":[{"day":"Tuesday","shift":"10am-6pm"},{"day":"Thursday","shift":"10am-6pm"},{"day":"Saturday","shift":"12pm-8pm"}]}]},"customers":[{"id":6001,"name":"Alice Johnson","email":"alice.johnson@example.com","orders":[{"orderId":7001,"date":"2024-09-01","status":"Shipped","items":[{"productId":1001,"quantity":10},{"productId":2001,"quantity":2}],"total":12.48},{"orderId":7002,"date":"2024-09-15","status":"Delivered","items":[{"productId":3001,"quantity":1}],"total":999.99}]},{"id":6002,"name":"Bob Williams","email":"bob.williams@example.com","orders":[{"orderId":7003,"date":"2024-09-07","status":"Pending","items":[{"productId":1002,"quantity":5},{"productId":4001,"quantity":1}],"total":2403.94}]}]},{"store":{"name":"SuperMart","location":{"address":"123 Market Street","city":"Metropolis","state":"NY","zip":"10001"},"departments":[{"id":1,"name":"Groceries","categories":[{"id":101,"name":"Fruits & Vegetables","products":[{"id":1001,"name":"Apple","price":0.99,"stock":120,"details":{"weight":"1lb","origin":"USA","organic":true}},{"id":1002,"name":"Banana","price":0.39,"stock":200,"details":{"weight":"1lb","origin":"Ecuador","organic":false}}]},{"id":102,"name":"Dairy","products":[{"id":2001,"name":"Milk","price":3.49,"stock":50,"details":{"volume":"1 gallon","fatContent":"2%","brand":"DairyPure"}},{"id":2002,"name":"Cheddar Cheese","price":5.99,"stock":75,"details":{"weight":"8oz","origin":"Wisconsin","aged":"2 years"}}]}]},{"id":2,"name":"Electronics","categories":[{"id":201,"name":"Smartphones","products":[{"id":3001,"name":"iPhone 14","price":999.99,"stock":30,"details":{"color":"Black","storage":"128GB","carrier":"Unlocked"}},{"id":3002,"name":"Samsung Galaxy S21","price":849.99,"stock":25,"details":{"color":"White","storage":"256GB","carrier":"Unlocked"}}]},{"id":202,"name":"Laptops","products":[{"id":4001,"name":"MacBook Pro","price":2399.99,"stock":10,"details":{"screenSize":"16 inch","processor":"M1","ram":"16GB","storage":"512GB SSD"}},{"id":4002,"name":"Dell XPS 13","price":1299.99,"stock":15,"details":{"screenSize":"13 inch","processor":"Intel i7","ram":"16GB","storage":"256GB SSD"}}]}]}],"employees":[{"id":5001,"name":"John Doe","position":"Manager","schedule":[{"day":"Monday","shift":"9am-5pm"},{"day":"Wednesday","shift":"9am-5pm"},{"day":"Friday","shift":"9am-5pm"}]},{"id":5002,"name":"Jane Smith","position":"Cashier","schedule":[{"day":"Tuesday","shift":"10am-6pm"},{"day":"Thursday","shift":"10am-6pm"},{"day":"Saturday","shift":"12pm-8pm"}]}]},"customers":[{"id":6001,"name":"Alice Johnson","email":"alice.johnson@example.com","orders":[{"orderId":7001,"date":"2024-09-01","status":"Shipped","items":[{"productId":1001,"quantity":10},{"productId":2001,"quantity":2}],"total":12.48},{"orderId":7002,"date":"2024-09-15","status":"Delivered","items":[{"productId":3001,"quantity":1}],"total":999.99}]},{"id":6002,"name":"Bob Williams","email":"bob.williams@example.com","orders":[{"orderId":7003,"date":"2024-09-07","status":"Pending","items":[{"productId":1002,"quantity":5},{"productId":4001,"quantity":1}],"total":2403.94}]}]},{"store":{"name":"SuperMart","location":{"address":"123 Market Street","city":"Metropolis","state":"NY","zip":"10001"},"departments":[{"id":1,"name":"Groceries","categories":[{"id":101,"name":"Fruits & Vegetables","products":[{"id":1001,"name":"Apple","price":0.99,"stock":120,"details":{"weight":"1lb","origin":"USA","organic":true}},{"id":1002,"name":"Banana","price":0.39,"stock":200,"details":{"weight":"1lb","origin":"Ecuador","organic":false}}]},{"id":102,"name":"Dairy","products":[{"id":2001,"name":"Milk","price":3.49,"stock":50,"details":{"volume":"1 gallon","fatContent":"2%","brand":"DairyPure"}},{"id":2002,"name":"Cheddar Cheese","price":5.99,"stock":75,"details":{"weight":"8oz","origin":"Wisconsin","aged":"2 years"}}]}]},{"id":2,"name":"Electronics","categories":[{"id":201,"name":"Smartphones","products":[{"id":3001,"name":"iPhone 14","price":999.99,"stock":30,"details":{"color":"Black","storage":"128GB","carrier":"Unlocked"}},{"id":3002,"name":"Samsung Galaxy S21","price":849.99,"stock":25,"details":{"color":"White","storage":"256GB","carrier":"Unlocked"}}]},{"id":202,"name":"Laptops","products":[{"id":4001,"name":"MacBook Pro","price":2399.99,"stock":10,"details":{"screenSize":"16 inch","processor":"M1","ram":"16GB","storage":"512GB SSD"}},{"id":4002,"name":"Dell XPS 13","price":1299.99,"stock":15,"details":{"screenSize":"13 inch","processor":"Intel i7","ram":"16GB","storage":"256GB SSD"}}]}]}],"employees":[{"id":5001,"name":"John Doe","position":"Manager","schedule":[{"day":"Monday","shift":"9am-5pm"},{"day":"Wednesday","shift":"9am-5pm"},{"day":"Friday","shift":"9am-5pm"}]},{"id":5002,"name":"Jane Smith","position":"Cashier","schedule":[{"day":"Tuesday","shift":"10am-6pm"},{"day":"Thursday","shift":"10am-6pm"},{"day":"Saturday","shift":"12pm-8pm"}]}]},"customers":[{"id":6001,"name":"Alice Johnson","email":"alice.johnson@example.com","orders":[{"orderId":7001,"date":"2024-09-01","status":"Shipped","items":[{"productId":1001,"quantity":10},{"productId":2001,"quantity":2}],"total":12.48},{"orderId":7002,"date":"2024-09-15","status":"Delivered","items":[{"productId":3001,"quantity":1}],"total":999.99}]},{"id":6002,"name":"Bob Williams","email":"bob.williams@example.com","orders":[{"orderId":7003,"date":"2024-09-07","status":"Pending","items":[{"productId":1002,"quantity":5},{"productId":4001,"quantity":1}],"total":2403.94}]}]},{"store":{"name":"SuperMart","location":{"address":"123 Market Street","city":"Metropolis","state":"NY","zip":"10001"},"departments":[{"id":1,"name":"Groceries","categories":[{"id":101,"name":"Fruits & Vegetables","products":[{"id":1001,"name":"Apple","price":0.99,"stock":120,"details":{"weight":"1lb","origin":"USA","organic":true}},{"id":1002,"name":"Banana","price":0.39,"stock":200,"details":{"weight":"1lb","origin":"Ecuador","organic":false}}]},{"id":102,"name":"Dairy","products":[{"id":2001,"name":"Milk","price":3.49,"stock":50,"details":{"volume":"1 gallon","fatContent":"2%","brand":"DairyPure"}},{"id":2002,"name":"Cheddar Cheese","price":5.99,"stock":75,"details":{"weight":"8oz","origin":"Wisconsin","aged":"2 years"}}]}]},{"id":2,"name":"Electronics","categories":[{"id":201,"name":"Smartphones","products":[{"id":3001,"name":"iPhone 14","price":999.99,"stock":30,"details":{"color":"Black","storage":"128GB","carrier":"Unlocked"}},{"id":3002,"name":"Samsung Galaxy S21","price":849.99,"stock":25,"details":{"color":"White","storage":"256GB","carrier":"Unlocked"}}]},{"id":202,"name":"Laptops","products":[{"id":4001,"name":"MacBook Pro","price":2399.99,"stock":10,"details":{"screenSize":"16 inch","processor":"M1","ram":"16GB","storage":"512GB SSD"}},{"id":4002,"name":"Dell XPS 13","price":1299.99,"stock":15,"details":{"screenSize":"13 inch","processor":"Intel i7","ram":"16GB","storage":"256GB SSD"}}]}]}],"employees":[{"id":5001,"name":"John Doe","position":"Manager","schedule":[{"day":"Monday","shift":"9am-5pm"},{"day":"Wednesday","shift":"9am-5pm"},{"day":"Friday","shift":"9am-5pm"}]},{"id":5002,"name":"Jane Smith","position":"Cashier","schedule":[{"day":"Tuesday","shift":"10am-6pm"},{"day":"Thursday","shift":"10am-6pm"},{"day":"Saturday","shift":"12pm-8pm"}]}]},"customers":[{"id":6001,"name":"Alice Johnson","email":"alice.johnson@example.com","orders":[{"orderId":7001,"date":"2024-09-01","status":"Shipped","items":[{"productId":1001,"quantity":10},{"productId":2001,"quantity":2}],"total":12.48},{"orderId":7002,"date":"2024-09-15","status":"Delivered","items":[{"productId":3001,"quantity":1}],"total":999.99}]},{"id":6002,"name":"Bob Williams","email":"bob.williams@example.com","orders":[{"orderId":7003,"date":"2024-09-07","status":"Pending","items":[{"productId":1002,"quantity":5},{"productId":4001,"quantity":1}],"total":2403.94}]}]},{"store":{"name":"SuperMart","location":{"address":"123 Market Street","city":"Metropolis","state":"NY","zip":"10001"},"departments":[{"id":1,"name":"Groceries","categories":[{"id":101,"name":"Fruits & Vegetables","products":[{"id":1001,"name":"Apple","price":0.99,"stock":120,"details":{"weight":"1lb","origin":"USA","organic":true}},{"id":1002,"name":"Banana","price":0.39,"stock":200,"details":{"weight":"1lb","origin":"Ecuador","organic":false}}]},{"id":102,"name":"Dairy","products":[{"id":2001,"name":"Milk","price":3.49,"stock":50,"details":{"volume":"1 gallon","fatContent":"2%","brand":"DairyPure"}},{"id":2002,"name":"Cheddar Cheese","price":5.99,"stock":75,"details":{"weight":"8oz","origin":"Wisconsin","aged":"2 years"}}]}]},{"id":2,"name":"Electronics","categories":[{"id":201,"name":"Smartphones","products":[{"id":3001,"name":"iPhone 14","price":999.99,"stock":30,"details":{"color":"Black","storage":"128GB","carrier":"Unlocked"}},{"id":3002,"name":"Samsung Galaxy S21","price":849.99,"stock":25,"details":{"color":"White","storage":"256GB","carrier":"Unlocked"}}]},{"id":202,"name":"Laptops","products":[{"id":4001,"name":"MacBook Pro","price":2399.99,"stock":10,"details":{"screenSize":"16 inch","processor":"M1","ram":"16GB","storage":"512GB SSD"}},{"id":4002,"name":"Dell XPS 13","price":1299.99,"stock":15,"details":{"screenSize":"13 inch","processor":"Intel i7","ram":"16GB","storage":"256GB SSD"}}]}]}],"employees":[{"id":5001,"name":"John Doe","position":"Manager","schedule":[{"day":"Monday","shift":"9am-5pm"},{"day":"Wednesday","shift":"9am-5pm"},{"day":"Friday","shift":"9am-5pm"}]},{"id":5002,"name":"Jane Smith","position":"Cashier","schedule":[{"day":"Tuesday","shift":"10am-6pm"},{"day":"Thursday","shift":"10am-6pm"},{"day":"Saturday","shift":"12pm-8pm"}]}]},"customers":[{"id":6001,"name":"Alice Johnson","email":"alice.johnson@example.com","orders":[{"orderId":7001,"date":"2024-09-01","status":"Shipped","items":[{"productId":1001,"quantity":10},{"productId":2001,"quantity":2}],"total":12.48},{"orderId":7002,"date":"2024-09-15","status":"Delivered","items":[{"productId":3001,"quantity":1}],"total":999.99}]},{"id":6002,"name":"Bob Williams","email":"bob.williams@example.com","orders":[{"orderId":7003,"date":"2024-09-07","status":"Pending","items":[{"productId":1002,"quantity":5},{"productId":4001,"quantity":1}],"total":2403.94}]}]},{"store":{"name":"SuperMart","location":{"address":"123 Market Street","city":"Metropolis","state":"NY","zip":"10001"},"departments":[{"id":1,"name":"Groceries","categories":[{"id":101,"name":"Fruits & Vegetables","products":[{"id":1001,"name":"Apple","price":0.99,"stock":120,"details":{"weight":"1lb","origin":"USA","organic":true}},{"id":1002,"name":"Banana","price":0.39,"stock":200,"details":{"weight":"1lb","origin":"Ecuador","organic":false}}]},{"id":102,"name":"Dairy","products":[{"id":2001,"name":"Milk","price":3.49,"stock":50,"details":{"volume":"1 gallon","fatContent":"2%","brand":"DairyPure"}},{"id":2002,"name":"Cheddar Cheese","price":5.99,"stock":75,"details":{"weight":"8oz","origin":"Wisconsin","aged":"2 years"}}]}]},{"id":2,"name":"Electronics","categories":[{"id":201,"name":"Smartphones","products":[{"id":3001,"name":"iPhone 14","price":999.99,"stock":30,"details":{"color":"Black","storage":"128GB","carrier":"Unlocked"}},{"id":3002,"name":"Samsung Galaxy S21","price":849.99,"stock":25,"details":{"color":"White","storage":"256GB","carrier":"Unlocked"}}]},{"id":202,"name":"Laptops","products":[{"id":4001,"name":"MacBook Pro","price":2399.99,"stock":10,"details":{"screenSize":"16 inch","processor":"M1","ram":"16GB","storage":"512GB SSD"}},{"id":4002,"name":"Dell XPS 13","price":1299.99,"stock":15,"details":{"screenSize":"13 inch","processor":"Intel i7","ram":"16GB","storage":"256GB SSD"}}]}]}],"employees":[{"id":5001,"name":"John Doe","position":"Manager","schedule":[{"day":"Monday","shift":"9am-5pm"},{"day":"Wednesday","shift":"9am-5pm"},{"day":"Friday","shift":"9am-5pm"}]},{"id":5002,"name":"Jane Smith","position":"Cashier","schedule":[{"day":"Tuesday","shift":"10am-6pm"},{"day":"Thursday","shift":"10am-6pm"},{"day":"Saturday","shift":"12pm-8pm"}]}]},"customers":[{"id":6001,"name":"Alice Johnson","email":"alice.johnson@example.com","orders":[{"orderId":7001,"date":"2024-09-01","status":"Shipped","items":[{"productId":1001,"quantity":10},{"productId":2001,"quantity":2}],"total":12.48},{"orderId":7002,"date":"2024-09-15","status":"Delivered","items":[{"productId":3001,"quantity":1}],"total":999.99}]},{"id":6002,"name":"Bob Williams","email":"bob.williams@example.com","orders":[{"orderId":7003,"date":"2024-09-07","status":"Pending","items":[{"productId":1002,"quantity":5},{"productId":4001,"quantity":1}],"total":2403.94}]}]},{"store":{"name":"SuperMart","location":{"address":"123 Market Street","city":"Metropolis","state":"NY","zip":"10001"},"departments":[{"id":1,"name":"Groceries","categories":[{"id":101,"name":"Fruits & Vegetables","products":[{"id":1001,"name":"Apple","price":0.99,"stock":120,"details":{"weight":"1lb","origin":"USA","organic":true}},{"id":1002,"name":"Banana","price":0.39,"stock":200,"details":{"weight":"1lb","origin":"Ecuador","organic":false}}]},{"id":102,"name":"Dairy","products":[{"id":2001,"name":"Milk","price":3.49,"stock":50,"details":{"volume":"1 gallon","fatContent":"2%","brand":"DairyPure"}},{"id":2002,"name":"Cheddar Cheese","price":5.99,"stock":75,"details":{"weight":"8oz","origin":"Wisconsin","aged":"2 years"}}]}]},{"id":2,"name":"Electronics","categories":[{"id":201,"name":"Smartphones","products":[{"id":3001,"name":"iPhone 14","price":999.99,"stock":30,"details":{"color":"Black","storage":"128GB","carrier":"Unlocked"}},{"id":3002,"name":"Samsung Galaxy S21","price":849.99,"stock":25,"details":{"color":"White","storage":"256GB","carrier":"Unlocked"}}]},{"id":202,"name":"Laptops","products":[{"id":4001,"name":"MacBook Pro","price":2399.99,"stock":10,"details":{"screenSize":"16 inch","processor":"M1","ram":"16GB","storage":"512GB SSD"}},{"id":4002,"name":"Dell XPS 13","price":1299.99,"stock":15,"details":{"screenSize":"13 inch","processor":"Intel i7","ram":"16GB","storage":"256GB SSD"}}]}]}],"employees":[{"id":5001,"name":"John Doe","position":"Manager","schedule":[{"day":"Monday","shift":"9am-5pm"},{"day":"Wednesday","shift":"9am-5pm"},{"day":"Friday","shift":"9am-5pm"}]},{"id":5002,"name":"Jane Smith","position":"Cashier","schedule":[{"day":"Tuesday","shift":"10am-6pm"},{"day":"Thursday","shift":"10am-6pm"},{"day":"Saturday","shift":"12pm-8pm"}]}]},"customers":[{"id":6001,"name":"Alice Johnson","email":"alice.johnson@example.com","orders":[{"orderId":7001,"date":"2024-09-01","status":"Shipped","items":[{"productId":1001,"quantity":10},{"productId":2001,"quantity":2}],"total":12.48},{"orderId":7002,"date":"2024-09-15","status":"Delivered","items":[{"productId":3001,"quantity":1}],"total":999.99}]},{"id":6002,"name":"Bob Williams","email":"bob.williams@example.com","orders":[{"orderId":7003,"date":"2024-09-07","status":"Pending","items":[{"productId":1002,"quantity":5},{"productId":4001,"quantity":1}],"total":2403.94}]}]}] \ No newline at end of file From 79cd46e6755a65429ef08f17d9711130a4fc5868 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj <65964360+Amogh-Bharadwaj@users.noreply.github.com> Date: Thu, 3 Oct 2024 18:51:11 +0530 Subject: [PATCH 10/13] Error message fix for CH validate (#2124) Need to know in which destination table the source column is missing --- flow/connectors/clickhouse/clickhouse.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flow/connectors/clickhouse/clickhouse.go b/flow/connectors/clickhouse/clickhouse.go index 1320bf8a92..f76ebb480e 100644 --- a/flow/connectors/clickhouse/clickhouse.go +++ b/flow/connectors/clickhouse/clickhouse.go @@ -458,7 +458,7 @@ func (c *ClickHouseConnector) processTableComparison(dstTableName string, srcSch } } if !found { - return fmt.Errorf("field %s not found in destination table", srcField.Name) + return fmt.Errorf("field %s not found in destination table %s", srcField.Name, dstTableName) } } foundPeerDBColumns := 0 From 38f1f2b2b6e1a387449d92d851885d25153e2ae1 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj <65964360+Amogh-Bharadwaj@users.noreply.github.com> Date: Thu, 3 Oct 2024 19:45:33 +0530 Subject: [PATCH 11/13] SSH host key parsing: use ParseAuthorizedKey (#2125) `ParsePublicKey` expects the public key in a different SSH wire protocol format while `ParseAuthorizedKey` expects what you see in an `id_rsa.pub` file (which gives an error - `short read` when trying to parse it with `ParsePublicKey`) References: - [ParsePublicKey](https://pkg.go.dev/golang.org/x/crypto/ssh#ParsePublicKey) - [ParseAuthorizedKey](https://pkg.go.dev/golang.org/x/crypto/ssh#ParseAuthorizedKey) - [x] Functionally tested --- flow/connectors/utils/ssh.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flow/connectors/utils/ssh.go b/flow/connectors/utils/ssh.go index f9f83102ff..b3576ba486 100644 --- a/flow/connectors/utils/ssh.go +++ b/flow/connectors/utils/ssh.go @@ -45,7 +45,7 @@ func GetSSHClientConfig(config *protos.SSHConfig) (*ssh.ClientConfig, error) { var hostKeyCallback ssh.HostKeyCallback if config.HostKey != "" { - pubKey, err := ssh.ParsePublicKey([]byte(config.HostKey)) + pubKey, _, _, _, err := ssh.ParseAuthorizedKey([]byte(config.HostKey)) if err != nil { return nil, fmt.Errorf("failed to parse host key: %w", err) } From 888446a99fe4bbf76262ec575540bce79640bbb8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Mon, 7 Oct 2024 04:26:55 +0000 Subject: [PATCH 12/13] chore: update dependencies (#2126) --- e2e_cleanup/go.mod | 98 +++---- e2e_cleanup/go.sum | 244 ++++++++-------- flow/go.mod | 90 +++--- flow/go.sum | 186 +++++++----- nexus/Cargo.lock | 196 ++++++------- ui/package-lock.json | 676 ++++++++++++++++++++++++------------------- ui/package.json | 40 +-- 7 files changed, 830 insertions(+), 700 deletions(-) diff --git a/e2e_cleanup/go.mod b/e2e_cleanup/go.mod index ffc2158271..29854ac42f 100644 --- a/e2e_cleanup/go.mod +++ b/e2e_cleanup/go.mod @@ -3,43 +3,43 @@ module github.com/PeerDB-io/peer-flow-cleanup go 1.23.0 require ( - cloud.google.com/go/bigquery v1.62.0 - cloud.google.com/go/pubsub v1.40.0 - github.com/snowflakedb/gosnowflake v1.10.1 - github.com/youmark/pkcs8 v0.0.0-20240424034433-3c2c7870ae76 - google.golang.org/api v0.189.0 + cloud.google.com/go/bigquery v1.63.1 + cloud.google.com/go/pubsub v1.43.0 + github.com/snowflakedb/gosnowflake v1.11.2 + github.com/youmark/pkcs8 v0.0.0-20240726163527-a2c0da244d78 + google.golang.org/api v0.199.0 ) require ( - cloud.google.com/go v0.115.0 // indirect - cloud.google.com/go/auth v0.7.2 // indirect - cloud.google.com/go/auth/oauth2adapt v0.2.3 // indirect - cloud.google.com/go/compute/metadata v0.5.0 // indirect - cloud.google.com/go/iam v1.1.11 // indirect + cloud.google.com/go v0.115.1 // indirect + cloud.google.com/go/auth v0.9.7 // indirect + cloud.google.com/go/auth/oauth2adapt v0.2.4 // indirect + cloud.google.com/go/compute/metadata v0.5.2 // indirect + cloud.google.com/go/iam v1.2.1 // indirect github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4 // indirect github.com/99designs/keyring v1.2.2 // indirect - github.com/Azure/azure-sdk-for-go/sdk/azcore v1.13.0 // indirect + github.com/Azure/azure-sdk-for-go/sdk/azcore v1.14.0 // indirect github.com/Azure/azure-sdk-for-go/sdk/internal v1.10.0 // indirect - github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.4.0 // indirect + github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.4.1 // indirect github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c // indirect github.com/apache/arrow/go/v15 v15.0.2 // indirect - github.com/aws/aws-sdk-go-v2 v1.30.3 // indirect - github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.3 // indirect - github.com/aws/aws-sdk-go-v2/credentials v1.17.27 // indirect - github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.17.8 // indirect - github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.15 // indirect - github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.15 // indirect - github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.15 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.11.3 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.3.17 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.11.17 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.17.15 // indirect - github.com/aws/aws-sdk-go-v2/service/s3 v1.58.2 // indirect - github.com/aws/smithy-go v1.20.3 // indirect + github.com/aws/aws-sdk-go-v2 v1.32.0 // indirect + github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.6 // indirect + github.com/aws/aws-sdk-go-v2/credentials v1.17.39 // indirect + github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.17.28 // indirect + github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.19 // indirect + github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.19 // indirect + github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.19 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.12.0 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.4.0 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.12.0 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.18.0 // indirect + github.com/aws/aws-sdk-go-v2/service/s3 v1.65.0 // indirect + github.com/aws/smithy-go v1.22.0 // indirect github.com/danieljoos/wincred v1.2.2 // indirect github.com/dvsekhvalnov/jose2go v1.7.0 // indirect github.com/felixge/httpsnoop v1.0.4 // indirect - github.com/gabriel-vasile/mimetype v1.4.4 // indirect + github.com/gabriel-vasile/mimetype v1.4.5 // indirect github.com/go-logr/logr v1.4.2 // indirect github.com/go-logr/stdr v1.2.2 // indirect github.com/goccy/go-json v0.10.3 // indirect @@ -49,10 +49,10 @@ require ( github.com/google/flatbuffers v24.3.25+incompatible // indirect github.com/google/s2a-go v0.1.8 // indirect github.com/google/uuid v1.6.0 // indirect - github.com/googleapis/enterprise-certificate-proxy v0.3.2 // indirect + github.com/googleapis/enterprise-certificate-proxy v0.3.4 // indirect github.com/googleapis/gax-go/v2 v2.13.0 // indirect github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c // indirect - github.com/klauspost/compress v1.17.9 // indirect + github.com/klauspost/compress v1.17.10 // indirect github.com/klauspost/cpuid/v2 v2.2.8 // indirect github.com/mtibben/percent v0.2.1 // indirect github.com/pierrec/lz4/v4 v4.1.21 // indirect @@ -60,26 +60,26 @@ require ( github.com/sirupsen/logrus v1.9.3 // indirect github.com/zeebo/xxh3 v1.0.2 // indirect go.opencensus.io v0.24.0 // indirect - go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.53.0 // indirect - go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.53.0 // indirect - go.opentelemetry.io/otel v1.28.0 // indirect - go.opentelemetry.io/otel/metric v1.28.0 // indirect - go.opentelemetry.io/otel/trace v1.28.0 // indirect - golang.org/x/crypto v0.25.0 // indirect - golang.org/x/exp v0.0.0-20240719175910-8a7402abbf56 // indirect - golang.org/x/mod v0.19.0 // indirect - golang.org/x/net v0.27.0 // indirect - golang.org/x/oauth2 v0.21.0 // indirect - golang.org/x/sync v0.7.0 // indirect - golang.org/x/sys v0.22.0 // indirect - golang.org/x/term v0.22.0 // indirect - golang.org/x/text v0.16.0 // indirect - golang.org/x/time v0.5.0 // indirect - golang.org/x/tools v0.23.0 // indirect - golang.org/x/xerrors v0.0.0-20240716161551-93cc26a95ae9 // indirect - google.golang.org/genproto v0.0.0-20240722135656-d784300faade // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20240722135656-d784300faade // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240722135656-d784300faade // indirect - google.golang.org/grpc v1.65.0 // indirect + go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.55.0 // indirect + go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.55.0 // indirect + go.opentelemetry.io/otel v1.30.0 // indirect + go.opentelemetry.io/otel/metric v1.30.0 // indirect + go.opentelemetry.io/otel/trace v1.30.0 // indirect + golang.org/x/crypto v0.28.0 // indirect + golang.org/x/exp v0.0.0-20241004190924-225e2abe05e6 // indirect + golang.org/x/mod v0.21.0 // indirect + golang.org/x/net v0.30.0 // indirect + golang.org/x/oauth2 v0.23.0 // indirect + golang.org/x/sync v0.8.0 // indirect + golang.org/x/sys v0.26.0 // indirect + golang.org/x/term v0.25.0 // indirect + golang.org/x/text v0.19.0 // indirect + golang.org/x/time v0.7.0 // indirect + golang.org/x/tools v0.26.0 // indirect + golang.org/x/xerrors v0.0.0-20240903120638-7835f813f4da // indirect + google.golang.org/genproto v0.0.0-20240930140551-af27646dc61f // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240930140551-af27646dc61f // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240930140551-af27646dc61f // indirect + google.golang.org/grpc v1.67.1 // indirect google.golang.org/protobuf v1.34.2 // indirect ) diff --git a/e2e_cleanup/go.sum b/e2e_cleanup/go.sum index aa20fbd0ff..0293c672de 100644 --- a/e2e_cleanup/go.sum +++ b/e2e_cleanup/go.sum @@ -1,40 +1,40 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= -cloud.google.com/go v0.115.0 h1:CnFSK6Xo3lDYRoBKEcAtia6VSC837/ZkJuRduSFnr14= -cloud.google.com/go v0.115.0/go.mod h1:8jIM5vVgoAEoiVxQ/O4BFTfHqulPZgs/ufEzMcFMdWU= -cloud.google.com/go/auth v0.7.2 h1:uiha352VrCDMXg+yoBtaD0tUF4Kv9vrtrWPYXwutnDE= -cloud.google.com/go/auth v0.7.2/go.mod h1:VEc4p5NNxycWQTMQEDQF0bd6aTMb6VgYDXEwiJJQAbs= -cloud.google.com/go/auth/oauth2adapt v0.2.3 h1:MlxF+Pd3OmSudg/b1yZ5lJwoXCEaeedAguodky1PcKI= -cloud.google.com/go/auth/oauth2adapt v0.2.3/go.mod h1:tMQXOfZzFuNuUxOypHlQEXgdfX5cuhwU+ffUuXRJE8I= -cloud.google.com/go/bigquery v1.62.0 h1:SYEA2f7fKqbSRRBHb7g0iHTtZvtPSPYdXfmqsjpsBwo= -cloud.google.com/go/bigquery v1.62.0/go.mod h1:5ee+ZkF1x/ntgCsFQJAQTM3QkAZOecfCmvxhkJsWRSA= -cloud.google.com/go/compute/metadata v0.5.0 h1:Zr0eK8JbFv6+Wi4ilXAR8FJ3wyNdpxHKJNPos6LTZOY= -cloud.google.com/go/compute/metadata v0.5.0/go.mod h1:aHnloV2TPI38yx4s9+wAZhHykWvVCfu7hQbF+9CWoiY= -cloud.google.com/go/datacatalog v1.20.3 h1:lzMtWaUlaz9Bd9anvq2KBZwcFujzhVuxhIz1MsqRJv8= -cloud.google.com/go/datacatalog v1.20.3/go.mod h1:AKC6vAy5urnMg5eJK3oUjy8oa5zMbiY33h125l8lmlo= -cloud.google.com/go/iam v1.1.11 h1:0mQ8UKSfdHLut6pH9FM3bI55KWR46ketn0PuXleDyxw= -cloud.google.com/go/iam v1.1.11/go.mod h1:biXoiLWYIKntto2joP+62sd9uW5EpkZmKIvfNcTWlnQ= -cloud.google.com/go/kms v1.18.2 h1:EGgD0B9k9tOOkbPhYW1PHo2W0teamAUYMOUIcDRMfPk= -cloud.google.com/go/kms v1.18.2/go.mod h1:YFz1LYrnGsXARuRePL729oINmN5J/5e7nYijgvfiIeY= -cloud.google.com/go/longrunning v0.5.9 h1:haH9pAuXdPAMqHvzX0zlWQigXT7B0+CL4/2nXXdBo5k= -cloud.google.com/go/longrunning v0.5.9/go.mod h1:HD+0l9/OOW0za6UWdKJtXoFAX/BGg/3Wj8p10NeWF7c= -cloud.google.com/go/pubsub v1.40.0 h1:0LdP+zj5XaPAGtWr2V6r88VXJlmtaB/+fde1q3TU8M0= -cloud.google.com/go/pubsub v1.40.0/go.mod h1:BVJI4sI2FyXp36KFKvFwcfDRDfR8MiLT8mMhmIhdAeA= -cloud.google.com/go/storage v1.42.0 h1:4QtGpplCVt1wz6g5o1ifXd656P5z+yNgzdw1tVfp0cU= -cloud.google.com/go/storage v1.42.0/go.mod h1:HjMXRFq65pGKFn6hxj6x3HCyR41uSB72Z0SO/Vn6JFQ= +cloud.google.com/go v0.115.1 h1:Jo0SM9cQnSkYfp44+v+NQXHpcHqlnRJk2qxh6yvxxxQ= +cloud.google.com/go v0.115.1/go.mod h1:DuujITeaufu3gL68/lOFIirVNJwQeyf5UXyi+Wbgknc= +cloud.google.com/go/auth v0.9.7 h1:ha65jNwOfI48YmUzNfMaUDfqt5ykuYIUnSartpU1+BA= +cloud.google.com/go/auth v0.9.7/go.mod h1:Xo0n7n66eHyOWWCnitop6870Ilwo3PiZyodVkkH1xWM= +cloud.google.com/go/auth/oauth2adapt v0.2.4 h1:0GWE/FUsXhf6C+jAkWgYm7X9tK8cuEIfy19DBn6B6bY= +cloud.google.com/go/auth/oauth2adapt v0.2.4/go.mod h1:jC/jOpwFP6JBxhB3P5Rr0a9HLMC/Pe3eaL4NmdvqPtc= +cloud.google.com/go/bigquery v1.63.1 h1:/6syiWrSpardKNxdvldS5CUTRJX1iIkSPXCjLjiGL+g= +cloud.google.com/go/bigquery v1.63.1/go.mod h1:ufaITfroCk17WTqBhMpi8CRjsfHjMX07pDrQaRKKX2o= +cloud.google.com/go/compute/metadata v0.5.2 h1:UxK4uu/Tn+I3p2dYWTfiX4wva7aYlKixAHn3fyqngqo= +cloud.google.com/go/compute/metadata v0.5.2/go.mod h1:C66sj2AluDcIqakBq/M8lw8/ybHgOZqin2obFxa/E5k= +cloud.google.com/go/datacatalog v1.22.1 h1:i0DyKb/o7j+0vgaFtimcRFjYsD6wFw1jpnODYUyiYRs= +cloud.google.com/go/datacatalog v1.22.1/go.mod h1:MscnJl9B2lpYlFoxRjicw19kFTwEke8ReKL5Y/6TWg8= +cloud.google.com/go/iam v1.2.1 h1:QFct02HRb7H12J/3utj0qf5tobFh9V4vR6h9eX5EBRU= +cloud.google.com/go/iam v1.2.1/go.mod h1:3VUIJDPpwT6p/amXRC5GY8fCCh70lxPygguVtI0Z4/g= +cloud.google.com/go/kms v1.20.0 h1:uKUvjGqbBlI96xGE669hcVnEMw1Px/Mvfa62dhM5UrY= +cloud.google.com/go/kms v1.20.0/go.mod h1:/dMbFF1tLLFnQV44AoI2GlotbjowyUfgVwezxW291fM= +cloud.google.com/go/longrunning v0.6.1 h1:lOLTFxYpr8hcRtcwWir5ITh1PAKUD/sG2lKrTSYjyMc= +cloud.google.com/go/longrunning v0.6.1/go.mod h1:nHISoOZpBcmlwbJmiVk5oDRz0qG/ZxPynEGs1iZ79s0= +cloud.google.com/go/pubsub v1.43.0 h1:s3Qx+F96J7Kwey/uVHdK3QxFLIlOvvw4SfMYw2jFjb4= +cloud.google.com/go/pubsub v1.43.0/go.mod h1:LNLfqItblovg7mHWgU5g84Vhza4J8kTxx0YqIeTzcXY= +cloud.google.com/go/storage v1.43.0 h1:CcxnSohZwizt4LCzQHWvBf1/kvtHUn7gk9QERXPyXFs= +cloud.google.com/go/storage v1.43.0/go.mod h1:ajvxEa7WmZS1PxvKRq4bq0tFT3vMd502JwstCcYv0Q0= 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= github.com/99designs/keyring v1.2.2/go.mod h1:wes/FrByc8j7lFOAGLGSNEg8f/PaI3cgTBqhFkHUrPk= -github.com/Azure/azure-sdk-for-go/sdk/azcore v1.13.0 h1:GJHeeA2N7xrG3q30L2UXDyuWRzDM900/65j70wcM4Ww= -github.com/Azure/azure-sdk-for-go/sdk/azcore v1.13.0/go.mod h1:l38EPgmsp71HHLq9j7De57JcKOWPyhrsW1Awm1JS6K0= +github.com/Azure/azure-sdk-for-go/sdk/azcore v1.14.0 h1:nyQWyZvwGTvunIMxi1Y9uXkcyr+I7TeNrr/foo4Kpk8= +github.com/Azure/azure-sdk-for-go/sdk/azcore v1.14.0/go.mod h1:l38EPgmsp71HHLq9j7De57JcKOWPyhrsW1Awm1JS6K0= github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.7.0 h1:tfLQ34V6F7tVSwoTf/4lH5sE0o6eCJuNDTmH09nDpbc= github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.7.0/go.mod h1:9kIvujWAA58nmPmWB1m23fyWic1kYZMxD9CxaWn4Qpg= github.com/Azure/azure-sdk-for-go/sdk/internal v1.10.0 h1:ywEEhmNahHBihViHepv3xPBn1663uRv2t2q/ESv9seY= github.com/Azure/azure-sdk-for-go/sdk/internal v1.10.0/go.mod h1:iZDifYGJTIgIIkYRNWPENUnqx6bJ2xnSDFI2tjwZNuY= github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/storage/armstorage v1.6.0 h1:PiSrjRPpkQNjrM8H0WwKMnZUdu1RGMtd/LdGKUrOo+c= github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/storage/armstorage v1.6.0/go.mod h1:oDrbWx4ewMylP7xHivfgixbfGBT6APAwsSoHRKotnIc= -github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.4.0 h1:Be6KInmFEKV81c0pOAEbRYehLMwmmGI1exuFj248AMk= -github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.4.0/go.mod h1:WCPBHsOXfBVnivScjs2ypRfimjEW0qPVLGgJkZlrIOA= +github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.4.1 h1:cf+OIKbkmMHBaC3u78AXomweqM0oxQSgBXRZf3WH4yM= +github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.4.1/go.mod h1:ap1dmS6vQKJxSMNiGJcq4QuUQkOynyD93gLw6MDF7ek= github.com/AzureAD/microsoft-authentication-library-for-go v1.2.2 h1:XHOnouVk1mxXfQidrMEnLlPk9UMeRtyBTnEFtxkV0kU= github.com/AzureAD/microsoft-authentication-library-for-go v1.2.2/go.mod h1:wP83P5OoQ5p6ip3ScPr0BAq0BvuPAvacpEuSzyouqAI= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= @@ -42,44 +42,44 @@ github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c h1:RGWPOewvK github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c/go.mod h1:X0CRv0ky0k6m906ixxpzmDRLvX58TFUKS2eePweuyxk= 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.30.3 h1:jUeBtG0Ih+ZIFH0F4UkmL9w3cSpaMv9tYYDbzILP8dY= -github.com/aws/aws-sdk-go-v2 v1.30.3/go.mod h1:nIQjQVp5sfpQcTc9mPSr1B0PaWK5ByX9MOoDadSN4lc= -github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.3 h1:tW1/Rkad38LA15X4UQtjXZXNKsCgkshC3EbmcUmghTg= -github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.3/go.mod h1:UbnqO+zjqk3uIt9yCACHJ9IVNhyhOCnYk8yA19SAWrM= -github.com/aws/aws-sdk-go-v2/config v1.27.27 h1:HdqgGt1OAP0HkEDDShEl0oSYa9ZZBSOmKpdpsDMdO90= -github.com/aws/aws-sdk-go-v2/config v1.27.27/go.mod h1:MVYamCg76dFNINkZFu4n4RjDixhVr51HLj4ErWzrVwg= -github.com/aws/aws-sdk-go-v2/credentials v1.17.27 h1:2raNba6gr2IfA0eqqiP2XiQ0UVOpGPgDSi0I9iAP+UI= -github.com/aws/aws-sdk-go-v2/credentials v1.17.27/go.mod h1:gniiwbGahQByxan6YjQUMcW4Aov6bLC3m+evgcoN4r4= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.11 h1:KreluoV8FZDEtI6Co2xuNk/UqI9iwMrOx/87PBNIKqw= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.11/go.mod h1:SeSUYBLsMYFoRvHE0Tjvn7kbxaUhl75CJi1sbfhMxkU= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.17.8 h1:u1KOU1S15ufyZqmH/rA3POkiRH6EcDANHj2xHRzq+zc= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.17.8/go.mod h1:WPv2FRnkIOoDv/8j2gSUsI4qDc7392w5anFB/I89GZ8= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.15 h1:SoNJ4RlFEQEbtDcCEt+QG56MY4fm4W8rYirAmq+/DdU= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.15/go.mod h1:U9ke74k1n2bf+RIgoX1SXFed1HLs51OgUSs+Ph0KJP8= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.15 h1:C6WHdGnTDIYETAm5iErQUiVNsclNx9qbJVPIt03B6bI= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.15/go.mod h1:ZQLZqhcu+JhSrA9/NXRm8SkDvsycE+JkV3WGY41e+IM= -github.com/aws/aws-sdk-go-v2/internal/ini v1.8.0 h1:hT8rVHwugYE2lEfdFE0QWVo81lF7jMrYJVDWI+f+VxU= -github.com/aws/aws-sdk-go-v2/internal/ini v1.8.0/go.mod h1:8tu/lYfQfFe6IGnaOdrpVgEL2IrrDOf6/m9RQum4NkY= -github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.15 h1:Z5r7SycxmSllHYmaAZPpmN8GviDrSGhMS6bldqtXZPw= -github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.15/go.mod h1:CetW7bDE00QoGEmPUoZuRog07SGVAUVW6LFpNP0YfIg= -github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.11.3 h1:dT3MqvGhSoaIhRseqw2I0yH81l7wiR2vjs57O51EAm8= -github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.11.3/go.mod h1:GlAeCkHwugxdHaueRr4nhPuY+WW+gR8UjlcqzPr1SPI= -github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.3.17 h1:YPYe6ZmvUfDDDELqEKtAd6bo8zxhkm+XEFEzQisqUIE= -github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.3.17/go.mod h1:oBtcnYua/CgzCWYN7NZ5j7PotFDaFSUjCYVTtfyn7vw= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.11.17 h1:HGErhhrxZlQ044RiM+WdoZxp0p+EGM62y3L6pwA4olE= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.11.17/go.mod h1:RkZEx4l0EHYDJpWppMJ3nD9wZJAa8/0lq9aVC+r2UII= -github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.17.15 h1:246A4lSTXWJw/rmlQI+TT2OcqeDMKBdyjEQrafMaQdA= -github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.17.15/go.mod h1:haVfg3761/WF7YPuJOER2MP0k4UAXyHaLclKXB6usDg= -github.com/aws/aws-sdk-go-v2/service/s3 v1.58.2 h1:sZXIzO38GZOU+O0C+INqbH7C2yALwfMWpd64tONS/NE= -github.com/aws/aws-sdk-go-v2/service/s3 v1.58.2/go.mod h1:Lcxzg5rojyVPU/0eFwLtcyTaek/6Mtic5B1gJo7e/zE= -github.com/aws/aws-sdk-go-v2/service/sso v1.22.4 h1:BXx0ZIxvrJdSgSvKTZ+yRBeSqqgPM89VPlulEcl37tM= -github.com/aws/aws-sdk-go-v2/service/sso v1.22.4/go.mod h1:ooyCOXjvJEsUw7x+ZDHeISPMhtwI3ZCB7ggFMcFfWLU= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.26.4 h1:yiwVzJW2ZxZTurVbYWA7QOrAaCYQR72t0wrSBfoesUE= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.26.4/go.mod h1:0oxfLkpz3rQ/CHlx5hB7H69YUpFiI1tql6Q6Ne+1bCw= -github.com/aws/aws-sdk-go-v2/service/sts v1.30.3 h1:ZsDKRLXGWHk8WdtyYMoGNO7bTudrvuKpDKgMVRlepGE= -github.com/aws/aws-sdk-go-v2/service/sts v1.30.3/go.mod h1:zwySh8fpFyXp9yOr/KVzxOl8SRqgf/IDw5aUt9UKFcQ= -github.com/aws/smithy-go v1.20.3 h1:ryHwveWzPV5BIof6fyDvor6V3iUL7nTfiTKXHiW05nE= -github.com/aws/smithy-go v1.20.3/go.mod h1:krry+ya/rV9RDcV/Q16kpu6ypI4K2czasz0NC3qS14E= +github.com/aws/aws-sdk-go-v2 v1.32.0 h1:GuHp7GvMN74PXD5C97KT5D87UhIy4bQPkflQKbfkndg= +github.com/aws/aws-sdk-go-v2 v1.32.0/go.mod h1:2SK5n0a2karNTv5tbP1SjsX0uhttou00v/HpXKM1ZUo= +github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.6 h1:pT3hpW0cOHRJx8Y0DfJUEQuqPild8jRGmSFmBgvydr0= +github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.6/go.mod h1:j/I2++U0xX+cr44QjHay4Cvxj6FUbnxrgmqN3H1jTZA= +github.com/aws/aws-sdk-go-v2/config v1.27.41 h1:esG3WpmEuNJ6F4kVFLumN8nCfA5VBav1KKb3JPx83O4= +github.com/aws/aws-sdk-go-v2/config v1.27.41/go.mod h1:haUg09ebP+ClvPjU3EB/xe0HF9PguO19PD2fdjM2X14= +github.com/aws/aws-sdk-go-v2/credentials v1.17.39 h1:tmVexAhoGqJxNE2oc4/SJqL+Jz1x1iCPt5ts9XcqZCU= +github.com/aws/aws-sdk-go-v2/credentials v1.17.39/go.mod h1:zgOdbDI9epE608PdboJ87CYvPIejAgFevazeJW6iauQ= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.15 h1:kGjlNc2IXXcxPDcfMyCshNCjVgxUhC/vTJv7NvC9wKk= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.15/go.mod h1:rk/HmqPo+dX0Uv0Q1+4w3QKFdICEGSsTYz1hRWvH8UI= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.17.28 h1:yUPy1fwOKNZ9L52E9TCMomU+mKXNCgqi17dtYIdSolk= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.17.28/go.mod h1:bJJP1cGMO0fPBgCjqHAWbc0WRbKrxrWU4hQfc/0ciAA= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.19 h1:Q/k5wCeJkSWs+62kDfOillkNIJ5NqmE3iOfm48g/W8c= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.19/go.mod h1:Wns1C66VvtA2Bv/cUBuKZKQKdjo7EVMhp90aAa+8oTI= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.19 h1:AYLE0lUfKvN6icFTR/p+NmD1amYKTbqHQ1Nm+jwE6BM= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.19/go.mod h1:1giLakj64GjuH1NBzF/DXqly5DWHtMTaOzRZ53nFX0I= +github.com/aws/aws-sdk-go-v2/internal/ini v1.8.1 h1:VaRN3TlFdd6KxX1x3ILT5ynH6HvKgqdiXoTxAF4HQcQ= +github.com/aws/aws-sdk-go-v2/internal/ini v1.8.1/go.mod h1:FbtygfRFze9usAadmnGJNc8KsP346kEe+y2/oyhGAGc= +github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.19 h1:FKdiFzTxlTRO71p0C7VrLbkkdW8qfMKF5+ej6bTmkT0= +github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.19/go.mod h1:abO3pCj7WLQPTllnSeYImqFfkGrmJV0JovWo/gqT5N0= +github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.12.0 h1:TToQNkvGguu209puTojY/ozlqy2d/SFNcoLIqTFi42g= +github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.12.0/go.mod h1:0jp+ltwkf+SwG2fm/PKo8t4y8pJSgOCO4D8Lz3k0aHQ= +github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.4.0 h1:FQNWhRuSq8QwW74GtU0MrveNhZbqvHsA4dkA9w8fTDQ= +github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.4.0/go.mod h1:j/zZ3zmWfGCK91K73YsfHP53BSTLSjL/y6YN39XbBLM= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.12.0 h1:AdbiDUgQZmM28rDIZbiSwFxz8+3B94aOXxzs6oH+EA0= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.12.0/go.mod h1:uV476Bd80tiDTX4X2redMtagQUg65aU/gzPojSJ4kSI= +github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.18.0 h1:1NKXS8XfhMM0bg5wVYa/eOH8AM2f6JijugbKEyQFTIg= +github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.18.0/go.mod h1:ph931DUfVfgrhZR7py9olSvHCiRpvaGxNvlWBcXxFds= +github.com/aws/aws-sdk-go-v2/service/s3 v1.65.0 h1:2dSm7frMrw2tdJ0QvyccQNJyPGaP24dyDgZ6h1QJMGU= +github.com/aws/aws-sdk-go-v2/service/s3 v1.65.0/go.mod h1:4XSVpw66upN8wND3JZA29eXl2NOZvfFVq7DIP6xvfuQ= +github.com/aws/aws-sdk-go-v2/service/sso v1.24.0 h1:71FvP6XFj53NK+YiAEGVzeiccLVeFnHOCvMig0zOHsE= +github.com/aws/aws-sdk-go-v2/service/sso v1.24.0/go.mod h1:UVJqtKXSd9YppRKgdBIkyv7qgbSGv5DchM3yX0BN2mU= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.28.0 h1:Uco4o19bi3AmBapImNzuMk+rfzlui52BDyVK1UfJeRA= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.28.0/go.mod h1:+HLFhCpnG08hBee8bUdfd1mBK+rFKPt4O5igR9lXDfk= +github.com/aws/aws-sdk-go-v2/service/sts v1.32.0 h1:GiQUjZM2KUZX68o/LpZ1xqxYMuvoxpRrOwYARYog3vc= +github.com/aws/aws-sdk-go-v2/service/sts v1.32.0/go.mod h1:dKnu7M4MAS2SDlng1ytxd03H+y0LoUfEQ5E2VaaSw/4= +github.com/aws/smithy-go v1.22.0 h1:uunKnWlcoL3zO7q+gG2Pk53joueEOsnNB28QdMsmiMM= +github.com/aws/smithy-go v1.22.0/go.mod h1:irrKGvNn1InZwb2d7fkIRNucdfwR8R+Ts3wxYa/cJHg= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= @@ -96,8 +96,8 @@ github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1m github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/felixge/httpsnoop v1.0.4 h1:NFTV2Zj1bL4mc9sqWACXbQFVBBg2W3GPvqp8/ESS2Wg= github.com/felixge/httpsnoop v1.0.4/go.mod h1:m8KPJKqk1gH5J9DgRY2ASl2lWCfGKXixSwevea8zH2U= -github.com/gabriel-vasile/mimetype v1.4.4 h1:QjV6pZ7/XZ7ryI2KuyeEDE8wnh7fHP9YnQy+R0LnH8I= -github.com/gabriel-vasile/mimetype v1.4.4/go.mod h1:JwLei5XPtWdGiMFB5Pjle1oEeoSeEuJfJE+TtfvdB/s= +github.com/gabriel-vasile/mimetype v1.4.5 h1:J7wGKdGu33ocBOhGy0z653k/lFKLFDPJMG8Gql0kxn4= +github.com/gabriel-vasile/mimetype v1.4.5/go.mod h1:ibHel+/kbxn9x2407k1izTA1S81ku1z/DlgOW2QE0M4= github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= github.com/go-logr/logr v1.4.2 h1:6pFjapn8bFcIbiKo3XT4j/BhANplGihG6tvd+8rYgrY= github.com/go-logr/logr v1.4.2/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY= @@ -142,14 +142,14 @@ github.com/google/s2a-go v0.1.8/go.mod h1:6iNWHTpQ+nfNRN5E00MSdfDwVesa8hhS32PhPO github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= 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/enterprise-certificate-proxy v0.3.4 h1:XYIDZApgAnrN1c855gTgghdIA6Stxb52D5RnLI1SLyw= +github.com/googleapis/enterprise-certificate-proxy v0.3.4/go.mod h1:YKe7cfqYXjKGpGvmSg28/fFvhNzinZQm8DGnaburhGA= github.com/googleapis/gax-go/v2 v2.13.0 h1:yitjD5f7jQHhyDsnhKEBU52NdvvdSeGzlAnDPT0hH1s= github.com/googleapis/gax-go/v2 v2.13.0/go.mod h1:Z/fvTZXF8/uw7Xu5GuslPw+bplx6SS338j1Is2S+B7A= github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c h1:6rhixN/i8ZofjG1Y75iExal34USq5p+wiN1tpie8IrU= github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c/go.mod h1:NMPJylDgVpX0MLRlPy15sqSwOFv/U1GZ2m21JhFfek0= -github.com/klauspost/compress v1.17.9 h1:6KIumPrER1LHsvBVuDa0r5xaG0Es51mhhB9BQB2qeMA= -github.com/klauspost/compress v1.17.9/go.mod h1:Di0epgTjJY877eYKx5yC51cX2A2Vl2ibi7bDH9ttBbw= +github.com/klauspost/compress v1.17.10 h1:oXAz+Vh0PMUvJczoi+flxpnBEPxoER1IaAnU/NMPtT0= +github.com/klauspost/compress v1.17.10/go.mod h1:pMDklpSncoRMuLFrf1W9Ss9KT+0rH90U12bZKk7uwG0= github.com/klauspost/cpuid/v2 v2.2.8 h1:+StwCXwm9PdpiEkPyzBXIy+M9KUb4ODm0Zarf1kS5BM= github.com/klauspost/cpuid/v2 v2.2.8/go.mod h1:Lcz8mBdAVJIBVzewtcLocK12l3Y+JytZYpaMropDUws= github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= @@ -174,8 +174,8 @@ github.com/rogpeppe/go-internal v1.12.0 h1:exVL4IDcn6na9z1rAb56Vxr+CgyK3nn3O+epU github.com/rogpeppe/go-internal v1.12.0/go.mod h1:E+RYuTGaKKdloAfM02xzb0FW3Paa99yedzYV+kq4uf4= github.com/sirupsen/logrus v1.9.3 h1:dueUQJ1C2q9oE3F7wvmSGAaVtTmUizReu6fjN8uqzbQ= github.com/sirupsen/logrus v1.9.3/go.mod h1:naHLuLoDiP4jHNo9R0sCBMtWGeIprob74mVsIT4qYEQ= -github.com/snowflakedb/gosnowflake v1.10.1 h1:VGeQxsQj5s3hP0cRmtNYozhUvs2Y7Reu5Pk5pKuRGpI= -github.com/snowflakedb/gosnowflake v1.10.1/go.mod h1:hvc58mU03qg78mSz5z17/qnzI56hOdYYK2txWbM0hN0= +github.com/snowflakedb/gosnowflake v1.11.2 h1:eAMsxrCiC6ij5wX3dHx1TQCBOdDmCK062Ir8rndUkRg= +github.com/snowflakedb/gosnowflake v1.11.2/go.mod h1:WFe+8mpsapDaQjHX6BqJBKtfQCGlGD3lHKeDsKfpx2A= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo= @@ -187,56 +187,56 @@ github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= -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/youmark/pkcs8 v0.0.0-20240726163527-a2c0da244d78 h1:ilQV1hzziu+LLM3zUTJ0trRztfwgjqKnBWNtSRkbmwM= +github.com/youmark/pkcs8 v0.0.0-20240726163527-a2c0da244d78/go.mod h1:aL8wCCfTfSfmXjznFBSZNN13rSJjlIOI1fUNAtF7rmI= github.com/zeebo/assert v1.3.0 h1:g7C04CbJuIDKNPFHmsk4hwZDO5O+kntRxzaUoNXj+IQ= github.com/zeebo/assert v1.3.0/go.mod h1:Pq9JiuJQpG8JLJdtkwrJESF0Foym2/D9XMU5ciN/wJ0= github.com/zeebo/xxh3 v1.0.2 h1:xZmwmqxHZA8AI603jOQ0tMqmBr9lPeFwGg6d+xy9DC0= github.com/zeebo/xxh3 v1.0.2/go.mod h1:5NWz9Sef7zIDm2JHfFlcQvNekmcEl9ekUZQQKCYaDcA= -go.einride.tech/aip v0.67.1 h1:d/4TW92OxXBngkSOwWS2CH5rez869KpKMaN44mdxkFI= -go.einride.tech/aip v0.67.1/go.mod h1:ZGX4/zKw8dcgzdLsrvpOOGxfxI2QSk12SlP7d6c0/XI= +go.einride.tech/aip v0.68.0 h1:4seM66oLzTpz50u4K1zlJyOXQ3tCzcJN7I22tKkjipw= +go.einride.tech/aip v0.68.0/go.mod h1:7y9FF8VtPWqpxuAxl0KQWqaULxW4zFIesD6zF5RIHHg= 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.53.0 h1:9G6E0TXzGFVfTnawRzrPl83iHOAV7L8NJiR8RSGYV1g= -go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.53.0/go.mod h1:azvtTADFQJA8mX80jIH/akaE7h+dbm/sVuaHqN13w74= -go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.53.0 h1:4K4tsIXefpVJtvA/8srF4V4y0akAoPHkIslgAkjixJA= -go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.53.0/go.mod h1:jjdQuTGVsXV4vSs+CJ2qYDeDPf9yIJV23qlIzBm73Vg= -go.opentelemetry.io/otel v1.28.0 h1:/SqNcYk+idO0CxKEUOtKQClMK/MimZihKYMruSMViUo= -go.opentelemetry.io/otel v1.28.0/go.mod h1:q68ijF8Fc8CnMHKyzqL6akLO46ePnjkgfIMIjUIX9z4= -go.opentelemetry.io/otel/metric v1.28.0 h1:f0HGvSl1KRAU1DLgLGFjrwVyismPlnuU6JD6bOeuA5Q= -go.opentelemetry.io/otel/metric v1.28.0/go.mod h1:Fb1eVBFZmLVTMb6PPohq3TO9IIhUisDsbJoL/+uQW4s= -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.28.0 h1:GhQ9cUuQGmNDd5BTCP2dAvv75RdMxEfTmYejp+lkx9g= -go.opentelemetry.io/otel/trace v1.28.0/go.mod h1:jPyXzNPg6da9+38HEwElrQiHlVMTnVfM3/yv2OlIHaI= +go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.55.0 h1:hCq2hNMwsegUvPzI7sPOvtO9cqyy5GbWt/Ybp2xrx8Q= +go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.55.0/go.mod h1:LqaApwGx/oUmzsbqxkzuBvyoPpkxk3JQWnqfVrJ3wCA= +go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.55.0 h1:ZIg3ZT/aQ7AfKqdwp7ECpOK6vHqquXXuyTjIO8ZdmPs= +go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.55.0/go.mod h1:DQAwmETtZV00skUwgD6+0U89g80NKsJE3DCKeLLPQMI= +go.opentelemetry.io/otel v1.30.0 h1:F2t8sK4qf1fAmY9ua4ohFS/K+FUuOPemHUIXHtktrts= +go.opentelemetry.io/otel v1.30.0/go.mod h1:tFw4Br9b7fOS+uEao81PJjVMjW/5fvNCbpsDIXqP0pc= +go.opentelemetry.io/otel/metric v1.30.0 h1:4xNulvn9gjzo4hjg+wzIKG7iNFEaBMX00Qd4QIZs7+w= +go.opentelemetry.io/otel/metric v1.30.0/go.mod h1:aXTfST94tswhWEb+5QjlSqG+cZlmyXy/u8jFpor3WqQ= +go.opentelemetry.io/otel/sdk v1.29.0 h1:vkqKjk7gwhS8VaWb0POZKmIEDimRCMsopNYnriHyryo= +go.opentelemetry.io/otel/sdk v1.29.0/go.mod h1:pM8Dx5WKnvxLCb+8lG1PRNIDxu9g9b9g59Qr7hfAAok= +go.opentelemetry.io/otel/trace v1.30.0 h1:7UBkkYzeg3C7kQX8VAidWh2biiQbtAKjyIML8dQ9wmc= +go.opentelemetry.io/otel/trace v1.30.0/go.mod h1:5EyKqTzzmyqB9bwtCCq6pDLktPK6fmGf/Dph+8VI02o= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/crypto v0.25.0 h1:ypSNr+bnYL2YhwoMt2zPxHFmbAN1KZs/njMG3hxUp30= -golang.org/x/crypto v0.25.0/go.mod h1:T+wALwcMOSE0kXgUAnPAHqTLW+XHgcELELW8VaDgm/M= +golang.org/x/crypto v0.28.0 h1:GBDwsMXVQi34v5CCYUm2jkJvu4cbtru2U4TN2PSyQnw= +golang.org/x/crypto v0.28.0/go.mod h1:rmgy+3RHxRZMyY0jjAJShp2zgEdOqj2AO7U0pYmeQ7U= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20240719175910-8a7402abbf56 h1:2dVuKD2vS7b0QIHQbpyTISPd0LeHDbnYEryqj5Q1ug8= -golang.org/x/exp v0.0.0-20240719175910-8a7402abbf56/go.mod h1:M4RDyNAINzryxdtnbRXRL/OHtkFuWGRjvuhBJpk2IlY= +golang.org/x/exp v0.0.0-20241004190924-225e2abe05e6 h1:1wqE9dj9NpSm04INVsJhhEUzhuDVjbcyKH91sVyPATw= +golang.org/x/exp v0.0.0-20241004190924-225e2abe05e6/go.mod h1:NQtJDoLvd6faHhE7m4T/1IY708gDefGGjR/iUW8yQQ8= 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= -golang.org/x/mod v0.19.0 h1:fEdghXQSo20giMthA7cd28ZC+jts4amQ3YMXiP5oMQ8= -golang.org/x/mod v0.19.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= +golang.org/x/mod v0.21.0 h1:vvrHzRwRfVKSiLrG+d4FMl/Qi4ukBCE6kZlTUkDYRT0= +golang.org/x/mod v0.21.0/go.mod h1:6SkKJ3Xj0I0BrPOZoBy3bdMptDDU9oJrpohJ3eWZ1fY= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= -golang.org/x/net v0.27.0 h1:5K3Njcw06/l2y9vpGCSdcxWOYHOUk3dVNGDXN+FvAys= -golang.org/x/net v0.27.0/go.mod h1:dDi0PyhWNoiUOrAS8uXv/vnScO4wnHQO4mj9fn/RytE= +golang.org/x/net v0.30.0 h1:AcW1SDZMkb8IpzCdQUaIq2sP4sZ4zw+55h6ynffypl4= +golang.org/x/net v0.30.0/go.mod h1:2wGyMJ5iFasEhkwi13ChkO/t1ECNC4X4eBKkVFyYFlU= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= -golang.org/x/oauth2 v0.21.0 h1:tsimM75w1tF/uws5rbeHzIWxEqElMehnc+iW793zsZs= -golang.org/x/oauth2 v0.21.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= +golang.org/x/oauth2 v0.23.0 h1:PbgcYx2W7i4LvjJWEbf0ngHV6qJYr86PkAV3bXdLEbs= +golang.org/x/oauth2 v0.23.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= -golang.org/x/sync v0.7.0 h1:YsImfSBoP9QPYL0xyKJPq0gcaJdG3rInoqxTWbfQu9M= -golang.org/x/sync v0.7.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= +golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -244,48 +244,48 @@ golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7w 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.22.0 h1:RI27ohtqKCnwULzJLqkv897zojh5/DwS/ENaMzUOaWI= -golang.org/x/sys v0.22.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/term v0.22.0 h1:BbsgPEJULsl2fV/AT3v15Mjva5yXKQDyKf+TbDz7QJk= -golang.org/x/term v0.22.0/go.mod h1:F3qCibpT5AMpCRfhfT53vVJwhLtIVHhB9XDjfFvnMI4= +golang.org/x/sys v0.26.0 h1:KHjCJyddX0LoSTb3J+vWpupP9p0oznkqVk/IfjymZbo= +golang.org/x/sys v0.26.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/term v0.25.0 h1:WtHI/ltw4NvSUig5KARz9h521QvRC8RmF/cuYqifU24= +golang.org/x/term v0.25.0/go.mod h1:RPyXicDX+6vLxogjjRxjgD2TKtmAO6NZBsBRfrOLu7M= 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.16.0 h1:a94ExnEXNtEwYLGJSIUxnWoxoRz/ZcCsV63ROupILh4= -golang.org/x/text v0.16.0/go.mod h1:GhwF1Be+LQoKShO3cGOHzqOgRrGaYc9AvblQOmPVHnI= -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/text v0.19.0 h1:kTxAhCbGbxhK0IwgSKiMO5awPoDQ0RpfiVYBfK860YM= +golang.org/x/text v0.19.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +golang.org/x/time v0.7.0 h1:ntUhktv3OPE6TgYxXWv9vKvUSJyIFJlyohwbkEwPrKQ= +golang.org/x/time v0.7.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= -golang.org/x/tools v0.23.0 h1:SGsXPZ+2l4JsgaCKkx+FQ9YZ5XEtA1GZYuoDjenLjvg= -golang.org/x/tools v0.23.0/go.mod h1:pnu6ufv6vQkll6szChhK3C3L/ruaIv5eBeztNG8wtsI= +golang.org/x/tools v0.26.0 h1:v/60pFQmzmT9ExmjDv2gGIfi3OqfKoEP6I5+umXlbnQ= +golang.org/x/tools v0.26.0/go.mod h1:TPVVj70c7JJ3WCazhD8OdXcZg/og+b9+tH/KxylGwH0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -golang.org/x/xerrors v0.0.0-20240716161551-93cc26a95ae9 h1:LLhsEBxRTBLuKlQxFBYUOU8xyFgXv6cOTp2HASDlsDk= -golang.org/x/xerrors v0.0.0-20240716161551-93cc26a95ae9/go.mod h1:NDW/Ps6MPRej6fsCIbMTohpP40sJ/P/vI1MoTEGwX90= +golang.org/x/xerrors v0.0.0-20240903120638-7835f813f4da h1:noIWHXmPHxILtqtCOPIhSt0ABwskkZKjD3bXGnZGpNY= +golang.org/x/xerrors v0.0.0-20240903120638-7835f813f4da/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.189.0 h1:equMo30LypAkdkLMBqfeIqtyAnlyig1JSZArl4XPwdI= -google.golang.org/api v0.189.0/go.mod h1:FLWGJKb0hb+pU2j+rJqwbnsF+ym+fQs73rbJ+KAUgy8= +google.golang.org/api v0.199.0 h1:aWUXClp+VFJmqE0JPvpZOK3LDQMyFKYIow4etYd9qxs= +google.golang.org/api v0.199.0/go.mod h1:ohG4qSztDJmZdjK/Ar6MhbAmb/Rpi4JHOqagsh90K28= 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-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= -google.golang.org/genproto v0.0.0-20240722135656-d784300faade h1:lKFsS7wpngDgSCeFn7MoLy+wBDQZ1UQIJD4UNM1Qvkg= -google.golang.org/genproto v0.0.0-20240722135656-d784300faade/go.mod h1:FfBgJBJg9GcpPvKIuHSZ/aE1g2ecGL74upMzGZjiGEY= -google.golang.org/genproto/googleapis/api v0.0.0-20240722135656-d784300faade h1:WxZOF2yayUHpHSbUE6NMzumUzBxYc3YGwo0YHnbzsJY= -google.golang.org/genproto/googleapis/api v0.0.0-20240722135656-d784300faade/go.mod h1:mw8MG/Qz5wfgYr6VqVCiZcHe/GJEfI+oGGDCohaVgB0= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240722135656-d784300faade h1:oCRSWfwGXQsqlVdErcyTt4A93Y8fo0/9D4b1gnI++qo= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240722135656-d784300faade/go.mod h1:Ue6ibwXGpU+dqIcODieyLOcgj7z8+IcskoNIgZxtrFY= +google.golang.org/genproto v0.0.0-20240930140551-af27646dc61f h1:mCJ6SGikSxVlt9scCayUl2dMq0msUgmBArqRY6umieI= +google.golang.org/genproto v0.0.0-20240930140551-af27646dc61f/go.mod h1:xtVODtPkMQRUZ4kqOTgp6JrXQrPevvfCSdk4mJtHUbM= +google.golang.org/genproto/googleapis/api v0.0.0-20240930140551-af27646dc61f h1:jTm13A2itBi3La6yTGqn8bVSrc3ZZ1r8ENHlIXBfnRA= +google.golang.org/genproto/googleapis/api v0.0.0-20240930140551-af27646dc61f/go.mod h1:CLGoBuH1VHxAUXVPP8FfPwPEVJB6lz3URE5mY2SuayE= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240930140551-af27646dc61f h1:cUMEy+8oS78BWIH9OWazBkzbr090Od9tWBNtZHkOhf0= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240930140551-af27646dc61f/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.33.2/go.mod h1:JMHMWHQWaTccqQQlmk3MJZS+GWXOdAesneDmEnv2fbc= -google.golang.org/grpc v1.65.0 h1:bs/cUb4lp1G5iImFFd3u5ixQzweKizoZJAwBNLR42lc= -google.golang.org/grpc v1.65.0/go.mod h1:WgYC2ypjlB0EiQi6wdKixMqukr6lBc0Vo+oOgjrM5ZQ= +google.golang.org/grpc v1.67.1 h1:zWnc1Vrcno+lHZCOofnIMvycFcc0QRGIzm9dhnDX68E= +google.golang.org/grpc v1.67.1/go.mod h1:1gLDyUQU7CTLJI90u3nXZ9ekeghjeM7pTDZlqFNg2AA= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= diff --git a/flow/go.mod b/flow/go.mod index f01a7e35c6..87201bebbe 100644 --- a/flow/go.mod +++ b/flow/go.mod @@ -4,9 +4,9 @@ go 1.23.1 require ( cloud.google.com/go v0.115.1 - cloud.google.com/go/bigquery v1.63.0 + cloud.google.com/go/bigquery v1.63.1 cloud.google.com/go/pubsub v1.43.0 - cloud.google.com/go/storage v1.43.0 + cloud.google.com/go/storage v1.44.0 github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.7.0 github.com/Azure/azure-sdk-for-go/sdk/messaging/azeventhubs v1.2.2 github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/eventhub/armeventhub v1.3.0 @@ -17,15 +17,15 @@ require ( github.com/PeerDB-io/gluajson v1.0.2 github.com/PeerDB-io/gluamsgpack v1.0.4 github.com/PeerDB-io/gluautf8 v1.0.0 - github.com/aws/aws-sdk-go-v2 v1.31.0 - github.com/aws/aws-sdk-go-v2/config v1.27.39 - github.com/aws/aws-sdk-go-v2/credentials v1.17.37 - github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.17.25 - github.com/aws/aws-sdk-go-v2/service/kms v1.36.3 - github.com/aws/aws-sdk-go-v2/service/s3 v1.63.3 - github.com/aws/aws-sdk-go-v2/service/ses v1.27.3 - github.com/aws/aws-sdk-go-v2/service/sns v1.32.3 - github.com/aws/smithy-go v1.21.0 + github.com/aws/aws-sdk-go-v2 v1.32.0 + github.com/aws/aws-sdk-go-v2/config v1.27.41 + github.com/aws/aws-sdk-go-v2/credentials v1.17.39 + github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.17.28 + github.com/aws/aws-sdk-go-v2/service/kms v1.37.0 + github.com/aws/aws-sdk-go-v2/service/s3 v1.65.0 + github.com/aws/aws-sdk-go-v2/service/ses v1.28.0 + github.com/aws/aws-sdk-go-v2/service/sns v1.33.0 + github.com/aws/smithy-go v1.22.0 github.com/cockroachdb/pebble v1.1.2 github.com/elastic/go-elasticsearch/v8 v8.15.0 github.com/google/uuid v1.6.0 @@ -44,7 +44,7 @@ require ( github.com/orcaman/concurrent-map/v2 v2.0.1 github.com/shopspring/decimal v1.4.0 github.com/slack-go/slack v0.14.0 - github.com/snowflakedb/gosnowflake v1.11.1 + github.com/snowflakedb/gosnowflake v1.11.2 github.com/stretchr/testify v1.9.0 github.com/twmb/franz-go v1.17.1 github.com/twmb/franz-go/pkg/kadm v1.13.0 @@ -62,32 +62,39 @@ require ( go.temporal.io/api v1.39.0 go.temporal.io/sdk v1.29.1 go.uber.org/automaxprocs v1.6.0 - golang.org/x/crypto v0.27.0 + golang.org/x/crypto v0.28.0 golang.org/x/mod v0.21.0 golang.org/x/sync v0.8.0 google.golang.org/api v0.199.0 - google.golang.org/genproto/googleapis/api v0.0.0-20240924160255-9d4c2d233b61 - google.golang.org/grpc v1.67.0 + google.golang.org/genproto/googleapis/api v0.0.0-20240930140551-af27646dc61f + google.golang.org/grpc v1.67.1 google.golang.org/protobuf v1.34.2 ) require ( - cloud.google.com/go/auth v0.9.5 // indirect + cel.dev/expr v0.16.2 // indirect + cloud.google.com/go/auth v0.9.7 // indirect cloud.google.com/go/auth/oauth2adapt v0.2.4 // indirect + cloud.google.com/go/monitoring v1.21.1 // indirect github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4 // indirect github.com/99designs/keyring v1.2.2 // indirect github.com/ClickHouse/ch-go v0.62.0 // indirect github.com/DataDog/zstd v1.5.6 // indirect + github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.24.2 // indirect + github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/metric v0.48.2 // indirect + github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/resourcemapping v0.48.2 // indirect github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c // indirect github.com/apache/arrow/go/v15 v15.0.2 // indirect - github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.14 // indirect + github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.15 // indirect github.com/aws/aws-sdk-go-v2/internal/ini v1.8.1 // indirect - github.com/aws/aws-sdk-go-v2/service/sso v1.23.3 // indirect - github.com/aws/aws-sdk-go-v2/service/ssooidc v1.27.3 // indirect - github.com/aws/aws-sdk-go-v2/service/sts v1.31.3 // indirect + github.com/aws/aws-sdk-go-v2/service/sso v1.24.0 // indirect + github.com/aws/aws-sdk-go-v2/service/ssooidc v1.28.0 // indirect + github.com/aws/aws-sdk-go-v2/service/sts v1.32.0 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/cenkalti/backoff/v4 v4.3.0 // indirect + github.com/census-instrumentation/opencensus-proto v0.4.1 // indirect github.com/cespare/xxhash/v2 v2.3.0 // indirect + github.com/cncf/xds/go v0.0.0-20240905190251-b4127c9b8d78 // indirect github.com/cockroachdb/errors v1.11.3 // indirect github.com/cockroachdb/fifo v0.0.0-20240816210425-c5d0cb0b6fc0 // indirect github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b // indirect @@ -97,6 +104,8 @@ require ( github.com/decred/dcrd/dcrec/secp256k1/v4 v4.3.0 // indirect github.com/dvsekhvalnov/jose2go v1.7.0 // indirect github.com/elastic/elastic-transport-go/v8 v8.6.0 // indirect + github.com/envoyproxy/go-control-plane v0.13.0 // indirect + github.com/envoyproxy/protoc-gen-validate v1.1.0 // indirect github.com/felixge/httpsnoop v1.0.4 // indirect github.com/gabriel-vasile/mimetype v1.4.5 // indirect github.com/getsentry/sentry-go v0.29.0 // indirect @@ -119,19 +128,22 @@ require ( github.com/nexus-rpc/sdk-go v0.0.10 // indirect github.com/paulmach/orb v0.11.1 // indirect github.com/pkg/errors v0.9.1 // indirect + github.com/planetscale/vtprotobuf v0.6.1-0.20240319094008-0393e58bdf10 // indirect github.com/prometheus/client_golang v1.20.4 // indirect github.com/prometheus/client_model v0.6.1 // indirect - github.com/prometheus/common v0.59.1 // indirect + github.com/prometheus/common v0.60.0 // indirect github.com/prometheus/procfs v0.15.1 // indirect github.com/rogpeppe/go-internal v1.13.1 // indirect github.com/segmentio/asm v1.2.0 // indirect github.com/sirupsen/logrus v1.9.3 // indirect github.com/twmb/franz-go/pkg/kmsg v1.8.0 // indirect + go.opentelemetry.io/contrib/detectors/gcp v1.30.0 // indirect go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.55.0 // indirect go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.55.0 // indirect go.opentelemetry.io/otel/trace v1.30.0 // indirect go.opentelemetry.io/proto/otlp v1.3.1 // indirect - golang.org/x/term v0.24.0 // indirect + golang.org/x/term v0.25.0 // indirect + google.golang.org/grpc/stats/opentelemetry v0.0.0-20241004113128-859602c14c6c // indirect ) require ( @@ -140,17 +152,17 @@ require ( github.com/Azure/azure-sdk-for-go/sdk/azcore v1.14.0 // indirect github.com/Azure/azure-sdk-for-go/sdk/internal v1.10.0 // indirect github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.4.1 // indirect - github.com/Azure/go-amqp v1.1.0 // indirect + github.com/Azure/go-amqp v1.2.0 // indirect github.com/AzureAD/microsoft-authentication-library-for-go v1.2.2 // indirect github.com/andybalholm/brotli v1.1.0 // indirect - github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.5 // indirect - github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.18 // indirect - github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.18 // indirect - github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.18 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.11.5 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.3.20 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.11.20 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.17.18 // indirect + github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.6 // indirect + github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.19 // indirect + github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.19 // indirect + github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.19 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.12.0 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.4.0 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.12.0 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.18.0 // indirect github.com/davecgh/go-spew v1.1.1 // indirect github.com/djherbis/buffer v1.2.0 github.com/djherbis/nio/v3 v3.0.1 @@ -185,15 +197,15 @@ 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-20240909161429-701f63a606c0 - golang.org/x/net v0.29.0 // indirect + golang.org/x/exp v0.0.0-20241004190924-225e2abe05e6 + golang.org/x/net v0.30.0 // indirect golang.org/x/oauth2 v0.23.0 // indirect - golang.org/x/sys v0.25.0 // indirect - golang.org/x/text v0.18.0 // indirect - golang.org/x/time v0.6.0 // indirect - golang.org/x/tools v0.25.0 // indirect + golang.org/x/sys v0.26.0 // indirect + golang.org/x/text v0.19.0 // indirect + golang.org/x/time v0.7.0 // indirect + golang.org/x/tools v0.26.0 // indirect golang.org/x/xerrors v0.0.0-20240903120638-7835f813f4da // indirect - google.golang.org/genproto v0.0.0-20240924160255-9d4c2d233b61 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240924160255-9d4c2d233b61 // indirect + google.golang.org/genproto v0.0.0-20240930140551-af27646dc61f // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240930140551-af27646dc61f // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/flow/go.sum b/flow/go.sum index 5fa7117f8d..0e3c31cf1e 100644 --- a/flow/go.sum +++ b/flow/go.sum @@ -1,12 +1,14 @@ +cel.dev/expr v0.16.2 h1:RwRhoH17VhAu9U5CMvMhH1PDVgf0tuz9FT+24AfMLfU= +cel.dev/expr v0.16.2/go.mod h1:gXngZQMkWJoSbE8mOzehJlXQyubn/Vg0vR9/F3W7iw8= cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.115.1 h1:Jo0SM9cQnSkYfp44+v+NQXHpcHqlnRJk2qxh6yvxxxQ= cloud.google.com/go v0.115.1/go.mod h1:DuujITeaufu3gL68/lOFIirVNJwQeyf5UXyi+Wbgknc= -cloud.google.com/go/auth v0.9.5 h1:4CTn43Eynw40aFVr3GpPqsQponx2jv0BQpjvajsbbzw= -cloud.google.com/go/auth v0.9.5/go.mod h1:Xo0n7n66eHyOWWCnitop6870Ilwo3PiZyodVkkH1xWM= +cloud.google.com/go/auth v0.9.7 h1:ha65jNwOfI48YmUzNfMaUDfqt5ykuYIUnSartpU1+BA= +cloud.google.com/go/auth v0.9.7/go.mod h1:Xo0n7n66eHyOWWCnitop6870Ilwo3PiZyodVkkH1xWM= cloud.google.com/go/auth/oauth2adapt v0.2.4 h1:0GWE/FUsXhf6C+jAkWgYm7X9tK8cuEIfy19DBn6B6bY= cloud.google.com/go/auth/oauth2adapt v0.2.4/go.mod h1:jC/jOpwFP6JBxhB3P5Rr0a9HLMC/Pe3eaL4NmdvqPtc= -cloud.google.com/go/bigquery v1.63.0 h1:yQFuJXdDukmBkiUUpjX0i1CtHLFU62HqPs/VDvSzaZo= -cloud.google.com/go/bigquery v1.63.0/go.mod h1:TQto6OR4kw27bqjNTGkVk1Vo5PJlTgxvDJn6YEIZL/E= +cloud.google.com/go/bigquery v1.63.1 h1:/6syiWrSpardKNxdvldS5CUTRJX1iIkSPXCjLjiGL+g= +cloud.google.com/go/bigquery v1.63.1/go.mod h1:ufaITfroCk17WTqBhMpi8CRjsfHjMX07pDrQaRKKX2o= cloud.google.com/go/compute/metadata v0.5.2 h1:UxK4uu/Tn+I3p2dYWTfiX4wva7aYlKixAHn3fyqngqo= cloud.google.com/go/compute/metadata v0.5.2/go.mod h1:C66sj2AluDcIqakBq/M8lw8/ybHgOZqin2obFxa/E5k= cloud.google.com/go/datacatalog v1.22.1 h1:i0DyKb/o7j+0vgaFtimcRFjYsD6wFw1jpnODYUyiYRs= @@ -15,12 +17,18 @@ cloud.google.com/go/iam v1.2.1 h1:QFct02HRb7H12J/3utj0qf5tobFh9V4vR6h9eX5EBRU= cloud.google.com/go/iam v1.2.1/go.mod h1:3VUIJDPpwT6p/amXRC5GY8fCCh70lxPygguVtI0Z4/g= cloud.google.com/go/kms v1.20.0 h1:uKUvjGqbBlI96xGE669hcVnEMw1Px/Mvfa62dhM5UrY= cloud.google.com/go/kms v1.20.0/go.mod h1:/dMbFF1tLLFnQV44AoI2GlotbjowyUfgVwezxW291fM= +cloud.google.com/go/logging v1.11.0 h1:v3ktVzXMV7CwHq1MBF65wcqLMA7i+z3YxbUsoK7mOKs= +cloud.google.com/go/logging v1.11.0/go.mod h1:5LDiJC/RxTt+fHc1LAt20R9TKiUTReDg6RuuFOZ67+A= cloud.google.com/go/longrunning v0.6.1 h1:lOLTFxYpr8hcRtcwWir5ITh1PAKUD/sG2lKrTSYjyMc= cloud.google.com/go/longrunning v0.6.1/go.mod h1:nHISoOZpBcmlwbJmiVk5oDRz0qG/ZxPynEGs1iZ79s0= +cloud.google.com/go/monitoring v1.21.1 h1:zWtbIoBMnU5LP9A/fz8LmWMGHpk4skdfeiaa66QdFGc= +cloud.google.com/go/monitoring v1.21.1/go.mod h1:Rj++LKrlht9uBi8+Eb530dIrzG/cU/lB8mt+lbeFK1c= cloud.google.com/go/pubsub v1.43.0 h1:s3Qx+F96J7Kwey/uVHdK3QxFLIlOvvw4SfMYw2jFjb4= cloud.google.com/go/pubsub v1.43.0/go.mod h1:LNLfqItblovg7mHWgU5g84Vhza4J8kTxx0YqIeTzcXY= -cloud.google.com/go/storage v1.43.0 h1:CcxnSohZwizt4LCzQHWvBf1/kvtHUn7gk9QERXPyXFs= -cloud.google.com/go/storage v1.43.0/go.mod h1:ajvxEa7WmZS1PxvKRq4bq0tFT3vMd502JwstCcYv0Q0= +cloud.google.com/go/storage v1.44.0 h1:abBzXf4UJKMmQ04xxJf9dYM/fNl24KHoTuBjyJDX2AI= +cloud.google.com/go/storage v1.44.0/go.mod h1:wpPblkIuMP5jCB/E48Pz9zIo2S/zD8g+ITmxKkPCITE= +cloud.google.com/go/trace v1.11.1 h1:UNqdP+HYYtnm6lb91aNA5JQ0X14GnxkABGlfz2PzPew= +cloud.google.com/go/trace v1.11.1/go.mod h1:IQKNQuBzH72EGaXEodKlNJrWykGZxet2zgjtS60OtjA= 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= @@ -49,8 +57,8 @@ github.com/Azure/azure-sdk-for-go/sdk/security/keyvault/internal v1.0.0 h1:D3occ github.com/Azure/azure-sdk-for-go/sdk/security/keyvault/internal v1.0.0/go.mod h1:bTSOgj05NGRuHHhQwAdPnYr9TOdNmKlZTgGLL6nyAdI= github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.4.1 h1:cf+OIKbkmMHBaC3u78AXomweqM0oxQSgBXRZf3WH4yM= github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.4.1/go.mod h1:ap1dmS6vQKJxSMNiGJcq4QuUQkOynyD93gLw6MDF7ek= -github.com/Azure/go-amqp v1.1.0 h1:XUhx5f4lZFVf6LQc5kBUFECW0iJW9VLxKCYrBeGwl0U= -github.com/Azure/go-amqp v1.1.0/go.mod h1:vZAogwdrkbyK3Mla8m/CxSc/aKdnTZ4IbPxl51Y5WZE= +github.com/Azure/go-amqp v1.2.0 h1:NNyfN3/cRszWzMvjmm64yaPZDHX/2DJkowv8Ub9y01I= +github.com/Azure/go-amqp v1.2.0/go.mod h1:vZAogwdrkbyK3Mla8m/CxSc/aKdnTZ4IbPxl51Y5WZE= github.com/AzureAD/microsoft-authentication-library-for-go v1.2.2 h1:XHOnouVk1mxXfQidrMEnLlPk9UMeRtyBTnEFtxkV0kU= github.com/AzureAD/microsoft-authentication-library-for-go v1.2.2/go.mod h1:wP83P5OoQ5p6ip3ScPr0BAq0BvuPAvacpEuSzyouqAI= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= @@ -60,6 +68,14 @@ github.com/ClickHouse/clickhouse-go/v2 v2.29.0 h1:Dj1w59RssRyLgGHXtYaWU0eIM1pJsu github.com/ClickHouse/clickhouse-go/v2 v2.29.0/go.mod h1:bLookq6qZJ4Ush/6tOAnJGh1Sf3Sa/nQoMn71p7ZCUE= github.com/DataDog/zstd v1.5.6 h1:LbEglqepa/ipmmQJUDnSsfvA8e8IStVcGaFWDuxvGOY= github.com/DataDog/zstd v1.5.6/go.mod h1:g4AWEaM3yOg3HYfnJ3YIawPnVdXJh9QME85blwSAmyw= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.24.2 h1:cZpsGsWTIFKymTA0je7IIvi1O7Es7apb9CF3EQlOcfE= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.24.2/go.mod h1:itPGVDKf9cC/ov4MdvJ2QZ0khw4bfoo9jzwTJlaxy2k= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/metric v0.48.2 h1:RopCq1mZTydpZpWfeYDvsnKR5L8VeaNt5JR5wiMfh7Q= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/metric v0.48.2/go.mod h1:tlLrnqq33OLuNnYbqswyI5ckZ0QjuM2DFIuaraxxDEU= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/cloudmock v0.48.2 h1:ffI2ensdT33alWXmBDi/7cvCV7K3o7TF5oE44g8tiN0= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/cloudmock v0.48.2/go.mod h1:pNP/L2wDlaQnQlFvkDKGSruDoYRpmAxB6drgsskfYwg= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/resourcemapping v0.48.2 h1:th/AQTVtV5u0WVQln/ks+jxhkZ433MeOevmka55fkeg= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/resourcemapping v0.48.2/go.mod h1:wRbFgBQUVm1YXrvWKofAEmq9HNJTDphbAaJSSX01KUI= github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c h1:RGWPOewvKIROun94nF7v2cua9qP+thov/7M50KEoeSU= github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c/go.mod h1:X0CRv0ky0k6m906ixxpzmDRLvX58TFUKS2eePweuyxk= github.com/PeerDB-io/glua64 v1.0.1 h1:biXLlFF/L5pnJCwDon7hkWkuQPozC8NjKS3J7Wzi69I= @@ -82,60 +98,64 @@ github.com/andybalholm/brotli v1.1.0 h1:eLKJA0d02Lf0mVpIDgYnqXcUn0GqVmEFny3VuID1 github.com/andybalholm/brotli v1.1.0/go.mod h1:sms7XGricyQI9K10gOSf56VKKWS4oLer58Q+mhRPtnY= 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.31.0 h1:3V05LbxTSItI5kUqNwhJrrrY1BAXxXt0sN0l72QmG5U= -github.com/aws/aws-sdk-go-v2 v1.31.0/go.mod h1:ztolYtaEUtdpf9Wftr31CJfLVjOnD/CVRkKOOYgF8hA= -github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.5 h1:xDAuZTn4IMm8o1LnBZvmrL8JA1io4o3YWNXgohbf20g= -github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.5/go.mod h1:wYSv6iDS621sEFLfKvpPE2ugjTuGlAG7iROg0hLOkfc= -github.com/aws/aws-sdk-go-v2/config v1.27.39 h1:FCylu78eTGzW1ynHcongXK9YHtoXD5AiiUqq3YfJYjU= -github.com/aws/aws-sdk-go-v2/config v1.27.39/go.mod h1:wczj2hbyskP4LjMKBEZwPRO1shXY+GsQleab+ZXT2ik= -github.com/aws/aws-sdk-go-v2/credentials v1.17.37 h1:G2aOH01yW8X373JK419THj5QVqu9vKEwxSEsGxihoW0= -github.com/aws/aws-sdk-go-v2/credentials v1.17.37/go.mod h1:0ecCjlb7htYCptRD45lXJ6aJDQac6D2NlKGpZqyTG6A= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.14 h1:C/d03NAmh8C4BZXhuRNboF/DqhBkBCeDiJDcaqIT5pA= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.14/go.mod h1:7I0Ju7p9mCIdlrfS+JCgqcYD0VXz/N4yozsox+0o078= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.17.25 h1:HkpHeZMM39sGtMHVYG1buAg93vhj5d7F81y6G0OAbGc= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.17.25/go.mod h1:j3Vz04ZjaWA6kygOsZRpmWe4CyGqfqq2u3unDTU0QGA= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.18 h1:kYQ3H1u0ANr9KEKlGs/jTLrBFPo8P8NaH/w7A01NeeM= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.18/go.mod h1:r506HmK5JDUh9+Mw4CfGJGSSoqIiLCndAuqXuhbv67Y= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.18 h1:Z7IdFUONvTcvS7YuhtVxN99v2cCoHRXOS4mTr0B/pUc= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.18/go.mod h1:DkKMmksZVVyat+Y+r1dEOgJEfUeA7UngIHWeKsi0yNc= +github.com/aws/aws-sdk-go-v2 v1.32.0 h1:GuHp7GvMN74PXD5C97KT5D87UhIy4bQPkflQKbfkndg= +github.com/aws/aws-sdk-go-v2 v1.32.0/go.mod h1:2SK5n0a2karNTv5tbP1SjsX0uhttou00v/HpXKM1ZUo= +github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.6 h1:pT3hpW0cOHRJx8Y0DfJUEQuqPild8jRGmSFmBgvydr0= +github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.6/go.mod h1:j/I2++U0xX+cr44QjHay4Cvxj6FUbnxrgmqN3H1jTZA= +github.com/aws/aws-sdk-go-v2/config v1.27.41 h1:esG3WpmEuNJ6F4kVFLumN8nCfA5VBav1KKb3JPx83O4= +github.com/aws/aws-sdk-go-v2/config v1.27.41/go.mod h1:haUg09ebP+ClvPjU3EB/xe0HF9PguO19PD2fdjM2X14= +github.com/aws/aws-sdk-go-v2/credentials v1.17.39 h1:tmVexAhoGqJxNE2oc4/SJqL+Jz1x1iCPt5ts9XcqZCU= +github.com/aws/aws-sdk-go-v2/credentials v1.17.39/go.mod h1:zgOdbDI9epE608PdboJ87CYvPIejAgFevazeJW6iauQ= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.15 h1:kGjlNc2IXXcxPDcfMyCshNCjVgxUhC/vTJv7NvC9wKk= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.15/go.mod h1:rk/HmqPo+dX0Uv0Q1+4w3QKFdICEGSsTYz1hRWvH8UI= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.17.28 h1:yUPy1fwOKNZ9L52E9TCMomU+mKXNCgqi17dtYIdSolk= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.17.28/go.mod h1:bJJP1cGMO0fPBgCjqHAWbc0WRbKrxrWU4hQfc/0ciAA= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.19 h1:Q/k5wCeJkSWs+62kDfOillkNIJ5NqmE3iOfm48g/W8c= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.19/go.mod h1:Wns1C66VvtA2Bv/cUBuKZKQKdjo7EVMhp90aAa+8oTI= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.19 h1:AYLE0lUfKvN6icFTR/p+NmD1amYKTbqHQ1Nm+jwE6BM= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.19/go.mod h1:1giLakj64GjuH1NBzF/DXqly5DWHtMTaOzRZ53nFX0I= github.com/aws/aws-sdk-go-v2/internal/ini v1.8.1 h1:VaRN3TlFdd6KxX1x3ILT5ynH6HvKgqdiXoTxAF4HQcQ= github.com/aws/aws-sdk-go-v2/internal/ini v1.8.1/go.mod h1:FbtygfRFze9usAadmnGJNc8KsP346kEe+y2/oyhGAGc= -github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.18 h1:OWYvKL53l1rbsUmW7bQyJVsYU/Ii3bbAAQIIFNbM0Tk= -github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.18/go.mod h1:CUx0G1v3wG6l01tUB+j7Y8kclA8NSqK4ef0YG79a4cg= -github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.11.5 h1:QFASJGfT8wMXtuP3D5CRmMjARHv9ZmzFUMJznHDOY3w= -github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.11.5/go.mod h1:QdZ3OmoIjSX+8D1OPAzPxDfjXASbBMDsz9qvtyIhtik= -github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.3.20 h1:rTWjG6AvWekO2B1LHeM3ktU7MqyX9rzWQ7hgzneZW7E= -github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.3.20/go.mod h1:RGW2DDpVc8hu6Y6yG8G5CHVmVOAn1oV8rNKOHRJyswg= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.11.20 h1:Xbwbmk44URTiHNx6PNo0ujDE6ERlsCKJD3u1zfnzAPg= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.11.20/go.mod h1:oAfOFzUB14ltPZj1rWwRc3d/6OgD76R8KlvU3EqM9Fg= -github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.17.18 h1:eb+tFOIl9ZsUe2259/BKPeniKuz4/02zZFH/i4Nf8Rg= -github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.17.18/go.mod h1:GVCC2IJNJTmdlyEsSmofEy7EfJncP7DNnXDzRjJ5Keg= -github.com/aws/aws-sdk-go-v2/service/kms v1.36.3 h1:iHi6lC6LfW6SNvB2bixmlOW3WMyWFrHZCWX+P+CCxMk= -github.com/aws/aws-sdk-go-v2/service/kms v1.36.3/go.mod h1:OHmlX4+o0XIlJAQGAHPIy0N9yZcYS/vNG+T7geSNcFw= -github.com/aws/aws-sdk-go-v2/service/s3 v1.63.3 h1:3zt8qqznMuAZWDTDpcwv9Xr11M/lVj2FsRR7oYBt0OA= -github.com/aws/aws-sdk-go-v2/service/s3 v1.63.3/go.mod h1:NLTqRLe3pUNu3nTEHI6XlHLKYmc8fbHUdMxAB6+s41Q= -github.com/aws/aws-sdk-go-v2/service/ses v1.27.3 h1:i7k9FzwUfv9seln228iz+69OyXZ0FMqhwotSeAlRC7A= -github.com/aws/aws-sdk-go-v2/service/ses v1.27.3/go.mod h1:WJjeWePq/vToxtM4fKbGHiXvInPARrWn8XJ0NOu4KtY= -github.com/aws/aws-sdk-go-v2/service/sns v1.32.3 h1:LC5JBrEAdJ0SSRLfNcLzOLsfoc3xO/BAsHiUNcQfDI4= -github.com/aws/aws-sdk-go-v2/service/sns v1.32.3/go.mod h1:ZO606Jfatw51c8q29gHVVCnufg2dq3MnmkNLlTZFrkE= -github.com/aws/aws-sdk-go-v2/service/sso v1.23.3 h1:rs4JCczF805+FDv2tRhZ1NU0RB2H6ryAvsWPanAr72Y= -github.com/aws/aws-sdk-go-v2/service/sso v1.23.3/go.mod h1:XRlMvmad0ZNL+75C5FYdMvbbLkd6qiqz6foR1nA1PXY= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.27.3 h1:S7EPdMVZod8BGKQQPTBK+FcX9g7bKR7c4+HxWqHP7Vg= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.27.3/go.mod h1:FnvDM4sfa+isJ3kDXIzAB9GAwVSzFzSy97uZ3IsHo4E= -github.com/aws/aws-sdk-go-v2/service/sts v1.31.3 h1:VzudTFrDCIDakXtemR7l6Qzt2+JYsVqo2MxBPt5k8T8= -github.com/aws/aws-sdk-go-v2/service/sts v1.31.3/go.mod h1:yMWe0F+XG0DkRZK5ODZhG7BEFYhLXi2dqGsv6tX0cgI= -github.com/aws/smithy-go v1.21.0 h1:H7L8dtDRk0P1Qm6y0ji7MCYMQObJ5R9CRpyPhRUkLYA= -github.com/aws/smithy-go v1.21.0/go.mod h1:irrKGvNn1InZwb2d7fkIRNucdfwR8R+Ts3wxYa/cJHg= +github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.19 h1:FKdiFzTxlTRO71p0C7VrLbkkdW8qfMKF5+ej6bTmkT0= +github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.19/go.mod h1:abO3pCj7WLQPTllnSeYImqFfkGrmJV0JovWo/gqT5N0= +github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.12.0 h1:TToQNkvGguu209puTojY/ozlqy2d/SFNcoLIqTFi42g= +github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.12.0/go.mod h1:0jp+ltwkf+SwG2fm/PKo8t4y8pJSgOCO4D8Lz3k0aHQ= +github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.4.0 h1:FQNWhRuSq8QwW74GtU0MrveNhZbqvHsA4dkA9w8fTDQ= +github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.4.0/go.mod h1:j/zZ3zmWfGCK91K73YsfHP53BSTLSjL/y6YN39XbBLM= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.12.0 h1:AdbiDUgQZmM28rDIZbiSwFxz8+3B94aOXxzs6oH+EA0= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.12.0/go.mod h1:uV476Bd80tiDTX4X2redMtagQUg65aU/gzPojSJ4kSI= +github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.18.0 h1:1NKXS8XfhMM0bg5wVYa/eOH8AM2f6JijugbKEyQFTIg= +github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.18.0/go.mod h1:ph931DUfVfgrhZR7py9olSvHCiRpvaGxNvlWBcXxFds= +github.com/aws/aws-sdk-go-v2/service/kms v1.37.0 h1:ovrHGOiNu4S0GSMeexZlsMhBkUb3bCE3iOktFZ7rmBU= +github.com/aws/aws-sdk-go-v2/service/kms v1.37.0/go.mod h1:YLqfMkq9GWbICgqT5XMIzT8I2+MxVKodTnNBo3BONgE= +github.com/aws/aws-sdk-go-v2/service/s3 v1.65.0 h1:2dSm7frMrw2tdJ0QvyccQNJyPGaP24dyDgZ6h1QJMGU= +github.com/aws/aws-sdk-go-v2/service/s3 v1.65.0/go.mod h1:4XSVpw66upN8wND3JZA29eXl2NOZvfFVq7DIP6xvfuQ= +github.com/aws/aws-sdk-go-v2/service/ses v1.28.0 h1:q+uLTiRFjYikxHyKoWd93PIqUtMauFULviv2TSo9DBQ= +github.com/aws/aws-sdk-go-v2/service/ses v1.28.0/go.mod h1:hnsZ7hRQ9ia0fw8TkGq+str440nue1tymneiXecfZFI= +github.com/aws/aws-sdk-go-v2/service/sns v1.33.0 h1:QuttYvND/OmttAImqJtsZXYJ6bEoUC2qLi29lhw1lss= +github.com/aws/aws-sdk-go-v2/service/sns v1.33.0/go.mod h1:bZXJof3RK1G0NKSmE3NQGBFDIpQD/ayLu7ffN1cCW/E= +github.com/aws/aws-sdk-go-v2/service/sso v1.24.0 h1:71FvP6XFj53NK+YiAEGVzeiccLVeFnHOCvMig0zOHsE= +github.com/aws/aws-sdk-go-v2/service/sso v1.24.0/go.mod h1:UVJqtKXSd9YppRKgdBIkyv7qgbSGv5DchM3yX0BN2mU= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.28.0 h1:Uco4o19bi3AmBapImNzuMk+rfzlui52BDyVK1UfJeRA= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.28.0/go.mod h1:+HLFhCpnG08hBee8bUdfd1mBK+rFKPt4O5igR9lXDfk= +github.com/aws/aws-sdk-go-v2/service/sts v1.32.0 h1:GiQUjZM2KUZX68o/LpZ1xqxYMuvoxpRrOwYARYog3vc= +github.com/aws/aws-sdk-go-v2/service/sts v1.32.0/go.mod h1:dKnu7M4MAS2SDlng1ytxd03H+y0LoUfEQ5E2VaaSw/4= +github.com/aws/smithy-go v1.22.0 h1:uunKnWlcoL3zO7q+gG2Pk53joueEOsnNB28QdMsmiMM= +github.com/aws/smithy-go v1.22.0/go.mod h1:irrKGvNn1InZwb2d7fkIRNucdfwR8R+Ts3wxYa/cJHg= 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/census-instrumentation/opencensus-proto v0.4.1 h1:iKLQ0xPNFxR/2hzXZMrBo8f1j86j5WHzznCCQxV/b8g= +github.com/census-instrumentation/opencensus-proto v0.4.1/go.mod h1:4T9NM4+4Vw91VeyqjLS6ao50K5bOcLKN6Q42XnYaRYw= github.com/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs= github.com/cespare/xxhash/v2 v2.3.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= +github.com/cncf/xds/go v0.0.0-20240905190251-b4127c9b8d78 h1:QVw89YDxXxEe+l8gU8ETbOasdwEV+avkR75ZzsVV9WI= +github.com/cncf/xds/go v0.0.0-20240905190251-b4127c9b8d78/go.mod h1:W+zGtBO5Y1IgJhy4+A9GOqVhqLpfZi+vwmdNXUehLA8= 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.3 h1:5bA+k2Y6r+oz/6Z/RFlNeVCesGARKuC6YymtcDrbC/I= @@ -172,7 +192,11 @@ github.com/elastic/go-elasticsearch/v8 v8.15.0/go.mod h1:HCON3zj4btpqs2N1jjsAy4a github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= +github.com/envoyproxy/go-control-plane v0.13.0 h1:HzkeUz1Knt+3bK+8LG1bxOO/jzWZmdxpwC51i202les= +github.com/envoyproxy/go-control-plane v0.13.0/go.mod h1:GRaKG3dwvFoTg4nj7aXdZnvMg4d7nvT/wl9WgVXn3Q8= github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= +github.com/envoyproxy/protoc-gen-validate v1.1.0 h1:tntQDh69XqOCOZsDz0lVJQez/2L6Uu2PdjCQwWCJ3bM= +github.com/envoyproxy/protoc-gen-validate v1.1.0/go.mod h1:sXRDRVmzEbkM7CVcM06s9shE/m23dg3wzjl0UWqJ2q4= github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a h1:yDWHCSQ40h88yih2JAcL6Ls/kVkSE8GFACTGVnMPruw= github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a/go.mod h1:7Ga40egUymuWXxAe151lTNnCv97MddSOVsjpPPkityA= github.com/felixge/httpsnoop v1.0.4 h1:NFTV2Zj1bL4mc9sqWACXbQFVBBg2W3GPvqp8/ESS2Wg= @@ -361,6 +385,8 @@ github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsK github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= +github.com/planetscale/vtprotobuf v0.6.1-0.20240319094008-0393e58bdf10 h1:GFCKgmp0tecUJ0sJuv4pzYCqS9+RGSn52M3FUwPs+uo= +github.com/planetscale/vtprotobuf v0.6.1-0.20240319094008-0393e58bdf10/go.mod h1:t/avpk3KcrXxUnYOhZhMXJlSEyie6gQbtLq5NM3loB8= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/prashantv/gostub v1.1.0 h1:BTyx3RfQjRHnUWaGF9oQos79AlQ5k8WNktv7VGvVH4g= @@ -370,8 +396,8 @@ github.com/prometheus/client_golang v1.20.4/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/j github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= -github.com/prometheus/common v0.59.1 h1:LXb1quJHWm1P6wq/U824uxYi4Sg0oGvNeUm1z5dJoX0= -github.com/prometheus/common v0.59.1/go.mod h1:GpWM7dewqmVYcd7SmRaiWVe9SSqjf0UrwnYnpEZNuT0= +github.com/prometheus/common v0.60.0 h1:+V9PAREWNvJMAuJ1x1BaWl9dewMW4YrHZQbx0sJNllA= +github.com/prometheus/common v0.60.0/go.mod h1:h0LYf1R1deLSKtD4Vdg8gy4RuOvENW2J/h19V5NADQw= 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= @@ -388,8 +414,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.14.0 h1:6c0UTfbRnvRssZUsZ2qe0Iu07VAMPjRqOa6oX8ewF4k= github.com/slack-go/slack v0.14.0/go.mod h1:hlGi5oXA+Gt+yWTPP0plCdRKmjsDxecdHxYQdlMQKOw= -github.com/snowflakedb/gosnowflake v1.11.1 h1:E91s8vBOSroaSTLsyjO4QPkEuzGmZcCxEFQLg214mvk= -github.com/snowflakedb/gosnowflake v1.11.1/go.mod h1:WFe+8mpsapDaQjHX6BqJBKtfQCGlGD3lHKeDsKfpx2A= +github.com/snowflakedb/gosnowflake v1.11.2 h1:eAMsxrCiC6ij5wX3dHx1TQCBOdDmCK062Ir8rndUkRg= +github.com/snowflakedb/gosnowflake v1.11.2/go.mod h1:WFe+8mpsapDaQjHX6BqJBKtfQCGlGD3lHKeDsKfpx2A= 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= @@ -442,6 +468,8 @@ go.einride.tech/aip v0.68.0/go.mod h1:7y9FF8VtPWqpxuAxl0KQWqaULxW4zFIesD6zF5RIHH 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/detectors/gcp v1.30.0 h1:GF+YVnUeJwOy+Ag2cTEpVZq+r2Tnci42FIiNwA2gjME= +go.opentelemetry.io/contrib/detectors/gcp v1.30.0/go.mod h1:p5Av42vWKPezk67MQwLYZwlo/z6xLnN/upaIyQNWBGg= go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.55.0 h1:hCq2hNMwsegUvPzI7sPOvtO9cqyy5GbWt/Ybp2xrx8Q= go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.55.0/go.mod h1:LqaApwGx/oUmzsbqxkzuBvyoPpkxk3JQWnqfVrJ3wCA= go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.55.0 h1:ZIg3ZT/aQ7AfKqdwp7ECpOK6vHqquXXuyTjIO8ZdmPs= @@ -476,11 +504,11 @@ 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.27.0 h1:GXm2NjJrPaiv/h1tb2UH8QfgC/hOf/+z0p6PT8o1w7A= -golang.org/x/crypto v0.27.0/go.mod h1:1Xngt8kV6Dvbssa53Ziq6Eqn0HqbZi5Z6R0ZpwQzt70= +golang.org/x/crypto v0.28.0 h1:GBDwsMXVQi34v5CCYUm2jkJvu4cbtru2U4TN2PSyQnw= +golang.org/x/crypto v0.28.0/go.mod h1:rmgy+3RHxRZMyY0jjAJShp2zgEdOqj2AO7U0pYmeQ7U= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20240909161429-701f63a606c0 h1:e66Fs6Z+fZTbFBAxKfP3PALWBtpfqks2bwGcexMxgtk= -golang.org/x/exp v0.0.0-20240909161429-701f63a606c0/go.mod h1:2TbTHSBQa924w8M6Xs1QcRcFwyucIwBGpK1p2f1YFFY= +golang.org/x/exp v0.0.0-20241004190924-225e2abe05e6 h1:1wqE9dj9NpSm04INVsJhhEUzhuDVjbcyKH91sVyPATw= +golang.org/x/exp v0.0.0-20241004190924-225e2abe05e6/go.mod h1:NQtJDoLvd6faHhE7m4T/1IY708gDefGGjR/iUW8yQQ8= 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= @@ -501,8 +529,8 @@ 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.29.0 h1:5ORfpBpCs4HzDYoodCDBbwHzdR5UrLBZ3sOnUJmFoHo= -golang.org/x/net v0.29.0/go.mod h1:gLkgy8jTGERgjzMic6DS9+SP0ajcu6Xu3Orq/SpETg0= +golang.org/x/net v0.30.0 h1:AcW1SDZMkb8IpzCdQUaIq2sP4sZ4zw+55h6ynffypl4= +golang.org/x/net v0.30.0/go.mod h1:2wGyMJ5iFasEhkwi13ChkO/t1ECNC4X4eBKkVFyYFlU= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.23.0 h1:PbgcYx2W7i4LvjJWEbf0ngHV6qJYr86PkAV3bXdLEbs= golang.org/x/oauth2 v0.23.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= @@ -528,19 +556,19 @@ 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.25.0 h1:r+8e+loiHxRqhXVl6ML1nO3l1+oFoWbnlu2Ehimmi34= -golang.org/x/sys v0.25.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.26.0 h1:KHjCJyddX0LoSTb3J+vWpupP9p0oznkqVk/IfjymZbo= +golang.org/x/sys v0.26.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.24.0 h1:Mh5cbb+Zk2hqqXNO7S1iTjEphVL+jb8ZWaqh/g+JWkM= -golang.org/x/term v0.24.0/go.mod h1:lOBK/LVxemqiMij05LGJ0tzNr8xlmwBRJ81PX6wVLH8= +golang.org/x/term v0.25.0 h1:WtHI/ltw4NvSUig5KARz9h521QvRC8RmF/cuYqifU24= +golang.org/x/term v0.25.0/go.mod h1:RPyXicDX+6vLxogjjRxjgD2TKtmAO6NZBsBRfrOLu7M= 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.18.0 h1:XvMDiNzPAl0jr17s6W9lcaIhGUfUORdGCNsuLmPG224= -golang.org/x/text v0.18.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= -golang.org/x/time v0.6.0 h1:eTDhh4ZXt5Qf0augr54TN6suAUudPcawVZeIAPU7D4U= -golang.org/x/time v0.6.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= +golang.org/x/text v0.19.0 h1:kTxAhCbGbxhK0IwgSKiMO5awPoDQ0RpfiVYBfK860YM= +golang.org/x/text v0.19.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +golang.org/x/time v0.7.0 h1:ntUhktv3OPE6TgYxXWv9vKvUSJyIFJlyohwbkEwPrKQ= +golang.org/x/time v0.7.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= @@ -551,8 +579,8 @@ golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtn golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.1.1/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= -golang.org/x/tools v0.25.0 h1:oFU9pkj/iJgs+0DT+VMHrx+oBKs/LJMV+Uvg78sl+fE= -golang.org/x/tools v0.25.0/go.mod h1:/vtpO8WL1N9cQC3FN5zPqb//fRXskFHbLKk4OW1Q7rg= +golang.org/x/tools v0.26.0 h1:v/60pFQmzmT9ExmjDv2gGIfi3OqfKoEP6I5+umXlbnQ= +golang.org/x/tools v0.26.0/go.mod h1:TPVVj70c7JJ3WCazhD8OdXcZg/og+b9+tH/KxylGwH0= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -569,20 +597,22 @@ google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoA 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-20240924160255-9d4c2d233b61 h1:KipVMxePgXPFBzXOvpKbny3RVdVmJOD64R/Ob7GPWEs= -google.golang.org/genproto v0.0.0-20240924160255-9d4c2d233b61/go.mod h1:HiAZQz/G7n0EywFjmncAwsfnmFm2bjm7qPjwl8hyzjM= -google.golang.org/genproto/googleapis/api v0.0.0-20240924160255-9d4c2d233b61 h1:pAjq8XSSzXoP9ya73v/w+9QEAAJNluLrpmMq5qFJQNY= -google.golang.org/genproto/googleapis/api v0.0.0-20240924160255-9d4c2d233b61/go.mod h1:O6rP0uBq4k0mdi/b4ZEMAZjkhYWhS815kCvaMha4VN8= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240924160255-9d4c2d233b61 h1:N9BgCIAUvn/M+p4NJccWPWb3BWh88+zyL0ll9HgbEeM= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240924160255-9d4c2d233b61/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= +google.golang.org/genproto v0.0.0-20240930140551-af27646dc61f h1:mCJ6SGikSxVlt9scCayUl2dMq0msUgmBArqRY6umieI= +google.golang.org/genproto v0.0.0-20240930140551-af27646dc61f/go.mod h1:xtVODtPkMQRUZ4kqOTgp6JrXQrPevvfCSdk4mJtHUbM= +google.golang.org/genproto/googleapis/api v0.0.0-20240930140551-af27646dc61f h1:jTm13A2itBi3La6yTGqn8bVSrc3ZZ1r8ENHlIXBfnRA= +google.golang.org/genproto/googleapis/api v0.0.0-20240930140551-af27646dc61f/go.mod h1:CLGoBuH1VHxAUXVPP8FfPwPEVJB6lz3URE5mY2SuayE= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240930140551-af27646dc61f h1:cUMEy+8oS78BWIH9OWazBkzbr090Od9tWBNtZHkOhf0= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240930140551-af27646dc61f/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= google.golang.org/grpc v1.33.2/go.mod h1:JMHMWHQWaTccqQQlmk3MJZS+GWXOdAesneDmEnv2fbc= -google.golang.org/grpc v1.67.0 h1:IdH9y6PF5MPSdAntIcpjQ+tXO41pcQsfZV2RxtQgVcw= -google.golang.org/grpc v1.67.0/go.mod h1:1gLDyUQU7CTLJI90u3nXZ9ekeghjeM7pTDZlqFNg2AA= +google.golang.org/grpc v1.67.1 h1:zWnc1Vrcno+lHZCOofnIMvycFcc0QRGIzm9dhnDX68E= +google.golang.org/grpc v1.67.1/go.mod h1:1gLDyUQU7CTLJI90u3nXZ9ekeghjeM7pTDZlqFNg2AA= +google.golang.org/grpc/stats/opentelemetry v0.0.0-20241004113128-859602c14c6c h1:pJuL8NmW9MVLZijtjSba++KjWp4JSUzTqHYInzgdxwA= +google.golang.org/grpc/stats/opentelemetry v0.0.0-20241004113128-859602c14c6c/go.mod h1:xwT0YrcBcgR1ZSSLJtUgCjF5QlvTOhiwA/I9TcYf3Gg= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= diff --git a/nexus/Cargo.lock b/nexus/Cargo.lock index 85285fc1d0..0061c58251 100644 --- a/nexus/Cargo.lock +++ b/nexus/Cargo.lock @@ -4,9 +4,9 @@ version = 3 [[package]] name = "addr2line" -version = "0.24.1" +version = "0.24.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f5fb1d8e4442bd405fdfd1dacb42792696b0cf9cb15882e5d097b742a676d375" +checksum = "dfbe277e56a376000877090da837660b4427aad530e3028d44e0bffe4f89a1c1" dependencies = [ "gimli", ] @@ -180,9 +180,9 @@ checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" [[package]] name = "async-compression" -version = "0.4.12" +version = "0.4.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fec134f64e2bc57411226dfc4e52dec859ddfc7e711fc5e07b612584f000e4aa" +checksum = "7e614738943d3f68c628ae3dbce7c3daffb196665f82f8c8ea6b65de73c79429" dependencies = [ "flate2", "futures-core", @@ -204,9 +204,9 @@ dependencies = [ [[package]] name = "async-stream" -version = "0.3.5" +version = "0.3.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd56dd203fef61ac097dd65721a419ddccb106b2d2b70ba60a6b529f03961a51" +checksum = "0b5a71a6f37880a80d1d7f19efd781e4b5de42c88f0722cc13bcb6cc2cfe8476" dependencies = [ "async-stream-impl", "futures-core", @@ -215,9 +215,9 @@ dependencies = [ [[package]] name = "async-stream-impl" -version = "0.3.5" +version = "0.3.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "16e62a023e7c117e27523144c5d2459f4397fcc3cab0085af8e2224f643a0193" +checksum = "c7c24de15d275a1ecfd47a380fb4d5ec9bfe0933f309ed5e705b775596a3574d" dependencies = [ "proc-macro2", "quote", @@ -249,9 +249,9 @@ checksum = "ace50bade8e6234aa140d9a2f552bbee1db4d353f69b8217bc503490fc1a9f26" [[package]] name = "aws-config" -version = "1.5.7" +version = "1.5.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8191fb3091fa0561d1379ef80333c3c7191c6f0435d986e85821bcf7acbd1126" +checksum = "7198e6f03240fdceba36656d8be440297b6b82270325908c7381f37d826a74f6" dependencies = [ "aws-credential-types", "aws-runtime", @@ -316,9 +316,9 @@ dependencies = [ [[package]] name = "aws-sdk-kms" -version = "1.45.0" +version = "1.46.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0caf20b8855dbeb458552e6c8f8f9eb92b95e4a131725b93540ec73d60c38eb3" +checksum = "e33590e8d45206fdc4273ded8a1f292bcceaadd513037aa790fc67b237bc30ee" dependencies = [ "aws-credential-types", "aws-runtime", @@ -338,9 +338,9 @@ dependencies = [ [[package]] name = "aws-sdk-sso" -version = "1.44.0" +version = "1.45.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0b90cfe6504115e13c41d3ea90286ede5aa14da294f3fe077027a6e83850843c" +checksum = "e33ae899566f3d395cbf42858e433930682cc9c1889fa89318896082fef45efb" dependencies = [ "aws-credential-types", "aws-runtime", @@ -360,9 +360,9 @@ dependencies = [ [[package]] name = "aws-sdk-ssooidc" -version = "1.45.0" +version = "1.46.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "167c0fad1f212952084137308359e8e4c4724d1c643038ce163f06de9662c1d0" +checksum = "f39c09e199ebd96b9f860b0fce4b6625f211e064ad7c8693b72ecf7ef03881e0" dependencies = [ "aws-credential-types", "aws-runtime", @@ -382,9 +382,9 @@ dependencies = [ [[package]] name = "aws-sdk-sts" -version = "1.44.0" +version = "1.45.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2cb5f98188ec1435b68097daa2a37d74b9d17c9caa799466338a8d1544e71b9d" +checksum = "3d95f93a98130389eb6233b9d615249e543f6c24a68ca1f109af9ca5164a8765" dependencies = [ "aws-credential-types", "aws-runtime", @@ -877,9 +877,9 @@ dependencies = [ [[package]] name = "cc" -version = "1.1.23" +version = "1.1.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3bbb537bb4a30b90362caddba8f360c0a56bc13d3a5570028e7197204cb54a17" +checksum = "2e80e3b6a3ab07840e1cae9b0666a63970dc28e8ed5ffbcdacbfc760c281bfc1" dependencies = [ "jobserver", "libc", @@ -970,9 +970,9 @@ dependencies = [ [[package]] name = "clap" -version = "4.5.18" +version = "4.5.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b0956a43b323ac1afaffc053ed5c4b7c1f1800bacd1683c353aabbb752515dd3" +checksum = "7be5744db7978a28d9df86a214130d106a89ce49644cbc4e3f0c22c3fba30615" dependencies = [ "clap_builder", "clap_derive", @@ -980,9 +980,9 @@ dependencies = [ [[package]] name = "clap_builder" -version = "4.5.18" +version = "4.5.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4d72166dd41634086d5803a47eb71ae740e61d84709c36f3c34110173db3961b" +checksum = "a5fbc17d3ef8278f55b282b2a2e75ae6f6c7d4bb70ed3d0382375104bfafdb4b" dependencies = [ "anstream", "anstyle", @@ -1367,9 +1367,9 @@ checksum = "e6d5a32815ae3f33302d95fdcb2ce17862f8c65363dcfd29360480ba1001fc9c" [[package]] name = "futures" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "645c6916888f6cb6350d2550b80fb63e734897a8498abe35cfb732b6487804b0" +checksum = "65bc07b1a8bc7c85c5f2e110c476c7389b4554ba72af57d8445ea63a576b0876" dependencies = [ "futures-channel", "futures-core", @@ -1382,9 +1382,9 @@ dependencies = [ [[package]] name = "futures-channel" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eac8f7d7865dcb88bd4373ab671c8cf4508703796caa2b1985a9ca867b3fcb78" +checksum = "2dff15bf788c671c1934e366d07e30c1814a8ef514e1af724a602e8a2fbe1b10" dependencies = [ "futures-core", "futures-sink", @@ -1392,15 +1392,15 @@ dependencies = [ [[package]] name = "futures-core" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dfc6580bb841c5a68e9ef15c77ccc837b40a7504914d52e47b8b0e9bbda25a1d" +checksum = "05f29059c0c2090612e8d742178b0580d2dc940c837851ad723096f87af6663e" [[package]] name = "futures-executor" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a576fc72ae164fca6b9db127eaa9a9dda0d61316034f33a0a0d4eda41f02b01d" +checksum = "1e28d1d997f585e54aebc3f97d39e72338912123a67330d723fdbb564d646c9f" dependencies = [ "futures-core", "futures-task", @@ -1409,15 +1409,15 @@ dependencies = [ [[package]] name = "futures-io" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a44623e20b9681a318efdd71c299b6b222ed6f231972bfe2f224ebad6311f0c1" +checksum = "9e5c1b78ca4aae1ac06c48a526a655760685149f0d465d21f37abfe57ce075c6" [[package]] name = "futures-macro" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" +checksum = "162ee34ebcb7c64a8abebc059ce0fee27c2262618d7b60ed8faf72fef13c3650" dependencies = [ "proc-macro2", "quote", @@ -1426,21 +1426,21 @@ dependencies = [ [[package]] name = "futures-sink" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9fb8e00e87438d937621c1c6269e53f536c14d3fbd6a042bb24879e57d474fb5" +checksum = "e575fab7d1e0dcb8d0c7bcf9a63ee213816ab51902e6d244a95819acacf1d4f7" [[package]] name = "futures-task" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38d84fa142264698cdce1a9f9172cf383a0c82de1bddcf3092901442c4097004" +checksum = "f90f7dce0722e95104fcb095585910c0977252f286e354b5e3bd38902cd99988" [[package]] name = "futures-util" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3d6401deb83407ab3da39eba7e33987a73c3df0c82b4bb5813ee871c19c41d48" +checksum = "9fa08315bb612088cc391249efdc3bc77536f16c91f6cf495e6fbe85b20a4a81" dependencies = [ "futures-channel", "futures-core", @@ -1516,9 +1516,9 @@ dependencies = [ [[package]] name = "gimli" -version = "0.31.0" +version = "0.31.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "32085ea23f3234fc7846555e85283ba4de91e21016dc0455a16286d87a292d64" +checksum = "07e28edb80900c19c28f1072f2e8aeca7fa06b23cd4169cefe1af5aa3260783f" [[package]] name = "glob" @@ -1538,7 +1538,7 @@ dependencies = [ "futures-sink", "futures-util", "http 0.2.12", - "indexmap 2.5.0", + "indexmap 2.6.0", "slab", "tokio", "tokio-util", @@ -1557,7 +1557,7 @@ dependencies = [ "futures-core", "futures-sink", "http 1.1.0", - "indexmap 2.5.0", + "indexmap 2.6.0", "slab", "tokio", "tokio-util", @@ -1583,6 +1583,12 @@ dependencies = [ "allocator-api2", ] +[[package]] +name = "hashbrown" +version = "0.15.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e087f84d4f86bf4b218b927129862374b72199ae7d8657835f1e89000eea4fb" + [[package]] name = "heck" version = "0.4.1" @@ -1674,9 +1680,9 @@ dependencies = [ [[package]] name = "httparse" -version = "1.9.4" +version = "1.9.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0fcc0b4a115bf80b728eb8ea024ad5bd707b615bfed49e0665b6e0f86fd082d9" +checksum = "7d71d3574edd2771538b901e6549113b4006ece66150fb69c0fb6d9a2adae946" [[package]] name = "httpdate" @@ -1755,7 +1761,7 @@ dependencies = [ "http 1.1.0", "hyper 1.4.1", "hyper-util", - "rustls 0.23.13", + "rustls 0.23.14", "rustls-native-certs 0.8.0", "rustls-pki-types", "tokio", @@ -1841,12 +1847,12 @@ dependencies = [ [[package]] name = "indexmap" -version = "2.5.0" +version = "2.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "68b900aa2f7301e21c36462b170ee99994de34dff39a4a6a528e80e7376d07e5" +checksum = "707907fe3c25f5424cce2cb7e1cbcafee6bdbe735ca90ef77c29e84591e5b9da" dependencies = [ "equivalent", - "hashbrown 0.14.5", + "hashbrown 0.15.0", ] [[package]] @@ -1861,9 +1867,9 @@ dependencies = [ [[package]] name = "ipnet" -version = "2.10.0" +version = "2.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "187674a687eed5fe42285b40c6291f9a01517d415fad1c3cbc6a9f778af7fcd4" +checksum = "ddc24109865250148c2e0f3d25d4f0f479571723792d3802153c60922a4fb708" [[package]] name = "is_terminal_polyfill" @@ -1925,7 +1931,7 @@ version = "0.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4ee7893dab2e44ae5f9d0173f26ff4aa327c10b01b06a72b52dd9405b628640d" dependencies = [ - "indexmap 2.5.0", + "indexmap 2.6.0", ] [[package]] @@ -2129,7 +2135,7 @@ dependencies = [ "pin-project", "rand", "rustls 0.22.4", - "rustls-pemfile 2.1.3", + "rustls-pemfile 2.2.0", "serde", "serde_json", "socket2", @@ -2282,21 +2288,18 @@ dependencies = [ [[package]] name = "object" -version = "0.36.4" +version = "0.36.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "084f1a5821ac4c651660a94a7153d27ac9d8a53736203f58b31945ded098070a" +checksum = "aedf0a2d09c573ed1d8d85b30c119153926a2b36dce0ab28322c09a117a4683e" dependencies = [ "memchr", ] [[package]] name = "once_cell" -version = "1.20.1" +version = "1.20.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "82881c4be219ab5faaf2ad5e5e5ecdff8c66bd7402ca3160975c93b24961afd1" -dependencies = [ - "portable-atomic", -] +checksum = "1261fe7e33c73b354eab43b1273a57c8f967d0391e80353e51f764ac02cf6775" [[package]] name = "opaque-debug" @@ -2625,7 +2628,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b4c5cc86750666a3ed20bdaf5ca2a0344f9c67674cae0515bec2da16fbaa47db" dependencies = [ "fixedbitset", - "indexmap 2.5.0", + "indexmap 2.6.0", ] [[package]] @@ -2673,18 +2676,18 @@ dependencies = [ [[package]] name = "pin-project" -version = "1.1.5" +version = "1.1.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b6bf43b791c5b9e34c3d182969b4abb522f9343702850a2e57f460d00d09b4b3" +checksum = "baf123a161dde1e524adf36f90bc5d8d3462824a9c43553ad07a8183161189ec" dependencies = [ "pin-project-internal", ] [[package]] name = "pin-project-internal" -version = "1.1.5" +version = "1.1.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2f38a4412a78282e09a2cf38d195ea5420d15ba0602cb375210efbc877243965" +checksum = "a4502d8515ca9f32f1fb543d987f63d95a14934883db45bdb48060b6b69257f8" dependencies = [ "proc-macro2", "quote", @@ -2758,12 +2761,6 @@ dependencies = [ "universal-hash", ] -[[package]] -name = "portable-atomic" -version = "1.9.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cc9c68a3f6da06753e9335d63e27f6b9754dd1920d941135b7ea8224f141adb2" - [[package]] name = "postgres" version = "0.19.9" @@ -2784,7 +2781,7 @@ version = "0.1.0" dependencies = [ "anyhow", "pt", - "rustls 0.23.13", + "rustls 0.23.14", "tokio", "tokio-postgres", "tokio-postgres-rustls", @@ -3030,7 +3027,7 @@ dependencies = [ "quinn-proto", "quinn-udp", "rustc-hash 2.0.0", - "rustls 0.23.13", + "rustls 0.23.14", "socket2", "thiserror", "tokio", @@ -3047,7 +3044,7 @@ dependencies = [ "rand", "ring", "rustc-hash 2.0.0", - "rustls 0.23.13", + "rustls 0.23.14", "slab", "thiserror", "tinyvec", @@ -3245,9 +3242,9 @@ dependencies = [ [[package]] name = "reqwest" -version = "0.12.7" +version = "0.12.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f8f4955649ef5c38cc7f9e8aa41761d48fb9677197daea9984dc54f56aad5e63" +checksum = "f713147fbe92361e52392c73b8c9e48c04c6625bce969ef54dc901e58e042a7b" dependencies = [ "async-compression", "base64 0.22.1", @@ -3268,8 +3265,8 @@ dependencies = [ "percent-encoding", "pin-project-lite", "quinn", - "rustls 0.23.13", - "rustls-pemfile 2.1.3", + "rustls 0.23.14", + "rustls-pemfile 2.2.0", "rustls-pki-types", "serde", "serde_json", @@ -3436,9 +3433,9 @@ dependencies = [ [[package]] name = "rustls" -version = "0.23.13" +version = "0.23.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2dabaac7466917e566adb06783a81ca48944c6898a1b08b9374106dd671f4c8" +checksum = "415d9944693cb90382053259f89fbb077ea730ad7273047ec63b19bc9b160ba8" dependencies = [ "log", "once_cell", @@ -3468,7 +3465,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fcaf18a4f2be7326cd874a5fa579fae794320a0f388d365dca7e480e55f83f8a" dependencies = [ "openssl-probe", - "rustls-pemfile 2.1.3", + "rustls-pemfile 2.2.0", "rustls-pki-types", "schannel", "security-framework", @@ -3485,11 +3482,10 @@ dependencies = [ [[package]] name = "rustls-pemfile" -version = "2.1.3" +version = "2.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "196fe16b00e106300d3e45ecfcb764fa292a535d7326a29a5875c579c7417425" +checksum = "dce314e5fee3f39953d46bb63bb8a46d40c2f8fb7cc5a3b6cab2bde9721d6e50" dependencies = [ - "base64 0.22.1", "rustls-pki-types", ] @@ -3558,9 +3554,9 @@ checksum = "ece8e78b2f38ec51c51f5d475df0a7187ba5111b2a28bdc761ee05b075d40a71" [[package]] name = "schannel" -version = "0.1.24" +version = "0.1.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e9aaafd5a2b6e3d657ff009d82fbd630b6bd54dd4eb06f21693925cdf80f9b8b" +checksum = "01227be5826fa0690321a2ba6c5cd57a19cf3f6a09e76973b58e61de6ab9d1c1" dependencies = [ "windows-sys 0.59.0", ] @@ -4113,7 +4109,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "04fb792ccd6bbcd4bba408eb8a292f70fc4a3589e5d793626f45190e6454b6ab" dependencies = [ "ring", - "rustls 0.23.13", + "rustls 0.23.14", "tokio", "tokio-postgres", "tokio-rustls 0.26.0", @@ -4147,7 +4143,7 @@ version = "0.26.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0c7bc40d0e5a97695bb96e27995cd3a08538541b0a846f65bba7a359f36700d4" dependencies = [ - "rustls 0.23.13", + "rustls 0.23.14", "rustls-pki-types", "tokio", ] @@ -4203,7 +4199,7 @@ version = "0.22.22" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4ae48d6208a266e853d946088ed816055e556cc6028c5e8e2b84d9fa5dd7c7f5" dependencies = [ - "indexmap 2.5.0", + "indexmap 2.6.0", "serde", "serde_spanned", "toml_datetime", @@ -4232,7 +4228,7 @@ dependencies = [ "pin-project", "prost", "rustls-native-certs 0.8.0", - "rustls-pemfile 2.1.3", + "rustls-pemfile 2.2.0", "socket2", "tokio", "tokio-rustls 0.26.0", @@ -4436,9 +4432,9 @@ dependencies = [ [[package]] name = "unicode-bidi" -version = "0.3.15" +version = "0.3.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08f95100a766bf4f8f28f90d77e0a5461bbdb219042e7679bebe79004fed8d75" +checksum = "5ab17db44d7388991a428b2ee655ce0c212e862eff1768a455c58f9aad6e7893" [[package]] name = "unicode-ident" @@ -4457,9 +4453,9 @@ dependencies = [ [[package]] name = "unicode-properties" -version = "0.1.2" +version = "0.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "52ea75f83c0137a9b98608359a5f1af8144876eb67bcb1ce837368e906a9f524" +checksum = "e70f2a8b45122e719eb623c01822704c4e0907e7e426a05927e1a1cfff5b75d0" [[package]] name = "unicode-width" @@ -4494,7 +4490,7 @@ dependencies = [ "flate2", "log", "once_cell", - "rustls 0.23.13", + "rustls 0.23.14", "rustls-pki-types", "serde", "serde_json", @@ -4996,8 +4992,8 @@ dependencies = [ "hyper-util", "log", "percent-encoding", - "rustls 0.23.13", - "rustls-pemfile 2.1.3", + "rustls 0.23.14", + "rustls-pemfile 2.2.0", "seahash", "serde", "serde_json", diff --git a/ui/package-lock.json b/ui/package-lock.json index 8ce708fc6a..4aa461704b 100644 --- a/ui/package-lock.json +++ b/ui/package-lock.json @@ -9,33 +9,33 @@ "version": "0.1.0", "dependencies": { "@monaco-editor/react": "^4.6.0", - "@radix-ui/react-checkbox": "^1.1.1", - "@radix-ui/react-collapsible": "^1.1.0", - "@radix-ui/react-dialog": "^1.1.1", - "@radix-ui/react-dropdown-menu": "^2.1.1", + "@radix-ui/react-checkbox": "^1.1.2", + "@radix-ui/react-collapsible": "^1.1.1", + "@radix-ui/react-dialog": "^1.1.2", + "@radix-ui/react-dropdown-menu": "^2.1.2", "@radix-ui/react-form": "^0.1.0", "@radix-ui/react-icons": "^1.3.0", - "@radix-ui/react-popover": "^1.1.1", + "@radix-ui/react-popover": "^1.1.2", "@radix-ui/react-progress": "^1.1.0", - "@radix-ui/react-radio-group": "^1.2.0", - "@radix-ui/react-switch": "^1.1.0", - "@radix-ui/react-tabs": "^1.1.0", - "@radix-ui/react-toast": "^1.2.1", + "@radix-ui/react-radio-group": "^1.2.1", + "@radix-ui/react-switch": "^1.1.1", + "@radix-ui/react-tabs": "^1.1.1", + "@radix-ui/react-toast": "^1.2.2", "@radix-ui/react-toggle": "^1.0.3", "@radix-ui/react-toggle-group": "^1.1.0", - "@radix-ui/react-tooltip": "^1.1.2", - "@tremor/react": "^3.18.2", - "@types/node": "^22.6.1", - "@types/react": "^18.3.9", + "@radix-ui/react-tooltip": "^1.1.3", + "@tremor/react": "^3.18.3", + "@types/node": "^22.7.4", + "@types/react": "^18.3.11", "@types/react-dom": "^18.3.0", "classnames": "^2.5.1", "long": "^5.2.3", - "lucide-react": "^0.446.0", - "material-symbols": "^0.23.0", + "lucide-react": "^0.447.0", + "material-symbols": "^0.24.0", "moment": "^2.30.1", - "moment-timezone": "^0.5.45", - "next": "^14.2.13", - "next-auth": "^4.24.7", + "moment-timezone": "^0.5.46", + "next": "^14.2.14", + "next-auth": "^4.24.8", "prop-types": "^15.8.1", "protobufjs": "^7.4.0", "react": "^18.3.1", @@ -52,7 +52,7 @@ "autoprefixer": "^10.4.20", "copy-webpack-plugin": "^12.0.2", "eslint": "^8.57.1", - "eslint-config-next": "^14.2.13", + "eslint-config-next": "^14.2.14", "eslint-config-prettier": "^9.1.0", "gh-pages": "^6.1.1", "less": "^4.2.0", @@ -63,7 +63,7 @@ "tailwindcss": "^3.4.13", "tailwindcss-animate": "^1.0.7", "typescript": "^5.6.2", - "webpack": "^5.94.0" + "webpack": "^5.95.0" } }, "node_modules/@alloc/quick-lru": { @@ -79,12 +79,12 @@ } }, "node_modules/@babel/code-frame": { - "version": "7.24.7", - "resolved": "https://registry.npmjs.org/@babel/code-frame/-/code-frame-7.24.7.tgz", - "integrity": "sha512-BcYH1CVJBO9tvyIZ2jVeXgSIMvGZ2FDRvDdOIVQyuklNKSsx+eppDEBq/g47Ayw+RqNFE+URvOShmf+f/qwAlA==", + "version": "7.25.7", + "resolved": "https://registry.npmjs.org/@babel/code-frame/-/code-frame-7.25.7.tgz", + "integrity": "sha512-0xZJFNE5XMpENsgfHYTw8FbX4kv53mFLn2i3XPoq69LyhYSCBJtitaHx9QnsVTrsogI4Z3+HtEfZ2/GFPOtf5g==", "license": "MIT", "dependencies": { - "@babel/highlight": "^7.24.7", + "@babel/highlight": "^7.25.7", "picocolors": "^1.0.0" }, "engines": { @@ -92,58 +92,58 @@ } }, "node_modules/@babel/generator": { - "version": "7.25.6", - "resolved": "https://registry.npmjs.org/@babel/generator/-/generator-7.25.6.tgz", - "integrity": "sha512-VPC82gr1seXOpkjAAKoLhP50vx4vGNlF4msF64dSFq1P8RfB+QAuJWGHPXXPc8QyfVWwwB/TNNU4+ayZmHNbZw==", + "version": "7.25.7", + "resolved": "https://registry.npmjs.org/@babel/generator/-/generator-7.25.7.tgz", + "integrity": "sha512-5Dqpl5fyV9pIAD62yK9P7fcA768uVPUyrQmqpqstHWgMma4feF1x/oFysBCVZLY5wJ2GkMUCdsNDnGZrPoR6rA==", "license": "MIT", "dependencies": { - "@babel/types": "^7.25.6", + "@babel/types": "^7.25.7", "@jridgewell/gen-mapping": "^0.3.5", "@jridgewell/trace-mapping": "^0.3.25", - "jsesc": "^2.5.1" + "jsesc": "^3.0.2" }, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/helper-module-imports": { - "version": "7.24.7", - "resolved": "https://registry.npmjs.org/@babel/helper-module-imports/-/helper-module-imports-7.24.7.tgz", - "integrity": "sha512-8AyH3C+74cgCVVXow/myrynrAGv+nTVg5vKu2nZph9x7RcRwzmh0VFallJuFTZ9mx6u4eSdXZfcOzSqTUm0HCA==", + "version": "7.25.7", + "resolved": "https://registry.npmjs.org/@babel/helper-module-imports/-/helper-module-imports-7.25.7.tgz", + "integrity": "sha512-o0xCgpNmRohmnoWKQ0Ij8IdddjyBFE4T2kagL/x6M3+4zUgc+4qTOUBoNe4XxDskt1HPKO007ZPiMgLDq2s7Kw==", "license": "MIT", "dependencies": { - "@babel/traverse": "^7.24.7", - "@babel/types": "^7.24.7" + "@babel/traverse": "^7.25.7", + "@babel/types": "^7.25.7" }, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/helper-string-parser": { - "version": "7.24.8", - "resolved": "https://registry.npmjs.org/@babel/helper-string-parser/-/helper-string-parser-7.24.8.tgz", - "integrity": "sha512-pO9KhhRcuUyGnJWwyEgnRJTSIZHiT+vMD0kPeD+so0l7mxkMT19g3pjY9GTnHySck/hDzq+dtW/4VgnMkippsQ==", + "version": "7.25.7", + "resolved": "https://registry.npmjs.org/@babel/helper-string-parser/-/helper-string-parser-7.25.7.tgz", + "integrity": "sha512-CbkjYdsJNHFk8uqpEkpCvRs3YRp9tY6FmFY7wLMSYuGYkrdUi7r2lc4/wqsvlHoMznX3WJ9IP8giGPq68T/Y6g==", "license": "MIT", "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/helper-validator-identifier": { - "version": "7.24.7", - "resolved": "https://registry.npmjs.org/@babel/helper-validator-identifier/-/helper-validator-identifier-7.24.7.tgz", - "integrity": "sha512-rR+PBcQ1SMQDDyF6X0wxtG8QyLCgUB0eRAGguqRLfkCA87l7yAP7ehq8SNj96OOGTO8OBV70KhuFYcIkHXOg0w==", + "version": "7.25.7", + "resolved": "https://registry.npmjs.org/@babel/helper-validator-identifier/-/helper-validator-identifier-7.25.7.tgz", + "integrity": "sha512-AM6TzwYqGChO45oiuPqwL2t20/HdMC1rTPAesnBCgPCSF1x3oN9MVUwQV2iyz4xqWrctwK5RNC8LV22kaQCNYg==", "license": "MIT", "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/highlight": { - "version": "7.24.7", - "resolved": "https://registry.npmjs.org/@babel/highlight/-/highlight-7.24.7.tgz", - "integrity": "sha512-EStJpq4OuY8xYfhGVXngigBJRWxftKX9ksiGDnmlY3o7B/V7KIAc9X4oiK87uPJSc/vs5L869bem5fhZa8caZw==", + "version": "7.25.7", + "resolved": "https://registry.npmjs.org/@babel/highlight/-/highlight-7.25.7.tgz", + "integrity": "sha512-iYyACpW3iW8Fw+ZybQK+drQre+ns/tKpXbNESfrhNnPLIklLbXr7MYJ6gPEd0iETGLOK+SxMjVvKb/ffmk+FEw==", "license": "MIT", "dependencies": { - "@babel/helper-validator-identifier": "^7.24.7", + "@babel/helper-validator-identifier": "^7.25.7", "chalk": "^2.4.2", "js-tokens": "^4.0.0", "picocolors": "^1.0.0" @@ -224,12 +224,12 @@ } }, "node_modules/@babel/parser": { - "version": "7.25.6", - "resolved": "https://registry.npmjs.org/@babel/parser/-/parser-7.25.6.tgz", - "integrity": "sha512-trGdfBdbD0l1ZPmcJ83eNxB9rbEax4ALFTF7fN386TMYbeCQbyme5cOEXQhbGXKebwGaB/J52w1mrklMcbgy6Q==", + "version": "7.25.7", + "resolved": "https://registry.npmjs.org/@babel/parser/-/parser-7.25.7.tgz", + "integrity": "sha512-aZn7ETtQsjjGG5HruveUK06cU3Hljuhd9Iojm4M8WWv3wLE6OkE5PWbDUkItmMgegmccaITudyuW5RPYrYlgWw==", "license": "MIT", "dependencies": { - "@babel/types": "^7.25.6" + "@babel/types": "^7.25.7" }, "bin": { "parser": "bin/babel-parser.js" @@ -239,9 +239,9 @@ } }, "node_modules/@babel/runtime": { - "version": "7.25.6", - "resolved": "https://registry.npmjs.org/@babel/runtime/-/runtime-7.25.6.tgz", - "integrity": "sha512-VBj9MYyDb9tuLq7yzqjgzt6Q+IBQLrGZfdjOekyEirZPHxXWoTSGUTMrpsfi58Up73d13NfYLv8HT9vmznjzhQ==", + "version": "7.25.7", + "resolved": "https://registry.npmjs.org/@babel/runtime/-/runtime-7.25.7.tgz", + "integrity": "sha512-FjoyLe754PMiYsFaN5C94ttGiOmBNYTf6pLr4xXHAT5uctHb092PBszndLDR5XA/jghQvn4n7JMHl7dmTgbm9w==", "license": "MIT", "dependencies": { "regenerator-runtime": "^0.14.0" @@ -251,30 +251,30 @@ } }, "node_modules/@babel/template": { - "version": "7.25.0", - "resolved": "https://registry.npmjs.org/@babel/template/-/template-7.25.0.tgz", - "integrity": "sha512-aOOgh1/5XzKvg1jvVz7AVrx2piJ2XBi227DHmbY6y+bM9H2FlN+IfecYu4Xl0cNiiVejlsCri89LUsbj8vJD9Q==", + "version": "7.25.7", + "resolved": "https://registry.npmjs.org/@babel/template/-/template-7.25.7.tgz", + "integrity": "sha512-wRwtAgI3bAS+JGU2upWNL9lSlDcRCqD05BZ1n3X2ONLH1WilFP6O1otQjeMK/1g0pvYcXC7b/qVUB1keofjtZA==", "license": "MIT", "dependencies": { - "@babel/code-frame": "^7.24.7", - "@babel/parser": "^7.25.0", - "@babel/types": "^7.25.0" + "@babel/code-frame": "^7.25.7", + "@babel/parser": "^7.25.7", + "@babel/types": "^7.25.7" }, "engines": { "node": ">=6.9.0" } }, "node_modules/@babel/traverse": { - "version": "7.25.6", - "resolved": "https://registry.npmjs.org/@babel/traverse/-/traverse-7.25.6.tgz", - "integrity": "sha512-9Vrcx5ZW6UwK5tvqsj0nGpp/XzqthkT0dqIc9g1AdtygFToNtTF67XzYS//dm+SAK9cp3B9R4ZO/46p63SCjlQ==", + "version": "7.25.7", + "resolved": "https://registry.npmjs.org/@babel/traverse/-/traverse-7.25.7.tgz", + "integrity": "sha512-jatJPT1Zjqvh/1FyJs6qAHL+Dzb7sTb+xr7Q+gM1b+1oBsMsQQ4FkVKb6dFlJvLlVssqkRzV05Jzervt9yhnzg==", "license": "MIT", "dependencies": { - "@babel/code-frame": "^7.24.7", - "@babel/generator": "^7.25.6", - "@babel/parser": "^7.25.6", - "@babel/template": "^7.25.0", - "@babel/types": "^7.25.6", + "@babel/code-frame": "^7.25.7", + "@babel/generator": "^7.25.7", + "@babel/parser": "^7.25.7", + "@babel/template": "^7.25.7", + "@babel/types": "^7.25.7", "debug": "^4.3.1", "globals": "^11.1.0" }, @@ -292,13 +292,13 @@ } }, "node_modules/@babel/types": { - "version": "7.25.6", - "resolved": "https://registry.npmjs.org/@babel/types/-/types-7.25.6.tgz", - "integrity": "sha512-/l42B1qxpG6RdfYf343Uw1vmDjeNhneUXtzhojE7pDgfpEypmRhI6j1kr17XCVv4Cgl9HdAiQY2x0GwKm7rWCw==", + "version": "7.25.7", + "resolved": "https://registry.npmjs.org/@babel/types/-/types-7.25.7.tgz", + "integrity": "sha512-vwIVdXG+j+FOpkwqHRcBgHLYNL7XMkufrlaFvL9o6Ai9sJn9+PdyIL5qa0XzTZw084c+u9LOls53eoZWP/W5WQ==", "license": "MIT", "dependencies": { - "@babel/helper-string-parser": "^7.24.8", - "@babel/helper-validator-identifier": "^7.24.7", + "@babel/helper-string-parser": "^7.25.7", + "@babel/helper-validator-identifier": "^7.25.7", "to-fast-properties": "^2.0.0" }, "engines": { @@ -783,15 +783,15 @@ } }, "node_modules/@next/env": { - "version": "14.2.13", - "resolved": "https://registry.npmjs.org/@next/env/-/env-14.2.13.tgz", - "integrity": "sha512-s3lh6K8cbW1h5Nga7NNeXrbe0+2jIIYK9YaA9T7IufDWnZpozdFUp6Hf0d5rNWUKu4fEuSX2rCKlGjCrtylfDw==", + "version": "14.2.14", + "resolved": "https://registry.npmjs.org/@next/env/-/env-14.2.14.tgz", + "integrity": "sha512-/0hWQfiaD5//LvGNgc8PjvyqV50vGK0cADYzaoOOGN8fxzBn3iAiaq3S0tCRnFBldq0LVveLcxCTi41ZoYgAgg==", "license": "MIT" }, "node_modules/@next/eslint-plugin-next": { - "version": "14.2.13", - "resolved": "https://registry.npmjs.org/@next/eslint-plugin-next/-/eslint-plugin-next-14.2.13.tgz", - "integrity": "sha512-z8Mk0VljxhIzsSiZUSdt3wp+t2lKd+jk5a9Jsvh3zDGkItgDMfjv/ZbET6HsxEl/fSihVoHGsXV6VLyDH0lfTQ==", + "version": "14.2.14", + "resolved": "https://registry.npmjs.org/@next/eslint-plugin-next/-/eslint-plugin-next-14.2.14.tgz", + "integrity": "sha512-kV+OsZ56xhj0rnTn6HegyTGkoa16Mxjrpk7pjWumyB2P8JVQb8S9qtkjy/ye0GnTr4JWtWG4x/2qN40lKZ3iVQ==", "dev": true, "license": "MIT", "dependencies": { @@ -799,9 +799,9 @@ } }, "node_modules/@next/swc-darwin-arm64": { - "version": "14.2.13", - "resolved": "https://registry.npmjs.org/@next/swc-darwin-arm64/-/swc-darwin-arm64-14.2.13.tgz", - "integrity": "sha512-IkAmQEa2Htq+wHACBxOsslt+jMoV3msvxCn0WFSfJSkv/scy+i/EukBKNad36grRxywaXUYJc9mxEGkeIs8Bzg==", + "version": "14.2.14", + "resolved": "https://registry.npmjs.org/@next/swc-darwin-arm64/-/swc-darwin-arm64-14.2.14.tgz", + "integrity": "sha512-bsxbSAUodM1cjYeA4o6y7sp9wslvwjSkWw57t8DtC8Zig8aG8V6r+Yc05/9mDzLKcybb6EN85k1rJDnMKBd9Gw==", "cpu": [ "arm64" ], @@ -815,9 +815,9 @@ } }, "node_modules/@next/swc-darwin-x64": { - "version": "14.2.13", - "resolved": "https://registry.npmjs.org/@next/swc-darwin-x64/-/swc-darwin-x64-14.2.13.tgz", - "integrity": "sha512-Dv1RBGs2TTjkwEnFMVL5XIfJEavnLqqwYSD6LXgTPdEy/u6FlSrLBSSfe1pcfqhFEXRAgVL3Wpjibe5wXJzWog==", + "version": "14.2.14", + "resolved": "https://registry.npmjs.org/@next/swc-darwin-x64/-/swc-darwin-x64-14.2.14.tgz", + "integrity": "sha512-cC9/I+0+SK5L1k9J8CInahduTVWGMXhQoXFeNvF0uNs3Bt1Ub0Azb8JzTU9vNCr0hnaMqiWu/Z0S1hfKc3+dww==", "cpu": [ "x64" ], @@ -831,9 +831,9 @@ } }, "node_modules/@next/swc-linux-arm64-gnu": { - "version": "14.2.13", - "resolved": "https://registry.npmjs.org/@next/swc-linux-arm64-gnu/-/swc-linux-arm64-gnu-14.2.13.tgz", - "integrity": "sha512-yB1tYEFFqo4ZNWkwrJultbsw7NPAAxlPXURXioRl9SdW6aIefOLS+0TEsKrWBtbJ9moTDgU3HRILL6QBQnMevg==", + "version": "14.2.14", + "resolved": "https://registry.npmjs.org/@next/swc-linux-arm64-gnu/-/swc-linux-arm64-gnu-14.2.14.tgz", + "integrity": "sha512-RMLOdA2NU4O7w1PQ3Z9ft3PxD6Htl4uB2TJpocm+4jcllHySPkFaUIFacQ3Jekcg6w+LBaFvjSPthZHiPmiAUg==", "cpu": [ "arm64" ], @@ -847,9 +847,9 @@ } }, "node_modules/@next/swc-linux-arm64-musl": { - "version": "14.2.13", - "resolved": "https://registry.npmjs.org/@next/swc-linux-arm64-musl/-/swc-linux-arm64-musl-14.2.13.tgz", - "integrity": "sha512-v5jZ/FV/eHGoWhMKYrsAweQ7CWb8xsWGM/8m1mwwZQ/sutJjoFaXchwK4pX8NqwImILEvQmZWyb8pPTcP7htWg==", + "version": "14.2.14", + "resolved": "https://registry.npmjs.org/@next/swc-linux-arm64-musl/-/swc-linux-arm64-musl-14.2.14.tgz", + "integrity": "sha512-WgLOA4hT9EIP7jhlkPnvz49iSOMdZgDJVvbpb8WWzJv5wBD07M2wdJXLkDYIpZmCFfo/wPqFsFR4JS4V9KkQ2A==", "cpu": [ "arm64" ], @@ -863,9 +863,9 @@ } }, "node_modules/@next/swc-linux-x64-gnu": { - "version": "14.2.13", - "resolved": "https://registry.npmjs.org/@next/swc-linux-x64-gnu/-/swc-linux-x64-gnu-14.2.13.tgz", - "integrity": "sha512-aVc7m4YL7ViiRv7SOXK3RplXzOEe/qQzRA5R2vpXboHABs3w8vtFslGTz+5tKiQzWUmTmBNVW0UQdhkKRORmGA==", + "version": "14.2.14", + "resolved": "https://registry.npmjs.org/@next/swc-linux-x64-gnu/-/swc-linux-x64-gnu-14.2.14.tgz", + "integrity": "sha512-lbn7svjUps1kmCettV/R9oAvEW+eUI0lo0LJNFOXoQM5NGNxloAyFRNByYeZKL3+1bF5YE0h0irIJfzXBq9Y6w==", "cpu": [ "x64" ], @@ -879,9 +879,9 @@ } }, "node_modules/@next/swc-linux-x64-musl": { - "version": "14.2.13", - "resolved": "https://registry.npmjs.org/@next/swc-linux-x64-musl/-/swc-linux-x64-musl-14.2.13.tgz", - "integrity": "sha512-4wWY7/OsSaJOOKvMsu1Teylku7vKyTuocvDLTZQq0TYv9OjiYYWt63PiE1nTuZnqQ4RPvME7Xai+9enoiN0Wrg==", + "version": "14.2.14", + "resolved": "https://registry.npmjs.org/@next/swc-linux-x64-musl/-/swc-linux-x64-musl-14.2.14.tgz", + "integrity": "sha512-7TcQCvLQ/hKfQRgjxMN4TZ2BRB0P7HwrGAYL+p+m3u3XcKTraUFerVbV3jkNZNwDeQDa8zdxkKkw2els/S5onQ==", "cpu": [ "x64" ], @@ -895,9 +895,9 @@ } }, "node_modules/@next/swc-win32-arm64-msvc": { - "version": "14.2.13", - "resolved": "https://registry.npmjs.org/@next/swc-win32-arm64-msvc/-/swc-win32-arm64-msvc-14.2.13.tgz", - "integrity": "sha512-uP1XkqCqV2NVH9+g2sC7qIw+w2tRbcMiXFEbMihkQ8B1+V6m28sshBwAB0SDmOe0u44ne1vFU66+gx/28RsBVQ==", + "version": "14.2.14", + "resolved": "https://registry.npmjs.org/@next/swc-win32-arm64-msvc/-/swc-win32-arm64-msvc-14.2.14.tgz", + "integrity": "sha512-8i0Ou5XjTLEje0oj0JiI0Xo9L/93ghFtAUYZ24jARSeTMXLUx8yFIdhS55mTExq5Tj4/dC2fJuaT4e3ySvXU1A==", "cpu": [ "arm64" ], @@ -911,9 +911,9 @@ } }, "node_modules/@next/swc-win32-ia32-msvc": { - "version": "14.2.13", - "resolved": "https://registry.npmjs.org/@next/swc-win32-ia32-msvc/-/swc-win32-ia32-msvc-14.2.13.tgz", - "integrity": "sha512-V26ezyjPqQpDBV4lcWIh8B/QICQ4v+M5Bo9ykLN+sqeKKBxJVDpEc6biDVyluTXTC40f5IqCU0ttth7Es2ZuMw==", + "version": "14.2.14", + "resolved": "https://registry.npmjs.org/@next/swc-win32-ia32-msvc/-/swc-win32-ia32-msvc-14.2.14.tgz", + "integrity": "sha512-2u2XcSaDEOj+96eXpyjHjtVPLhkAFw2nlaz83EPeuK4obF+HmtDJHqgR1dZB7Gb6V/d55FL26/lYVd0TwMgcOQ==", "cpu": [ "ia32" ], @@ -927,9 +927,9 @@ } }, "node_modules/@next/swc-win32-x64-msvc": { - "version": "14.2.13", - "resolved": "https://registry.npmjs.org/@next/swc-win32-x64-msvc/-/swc-win32-x64-msvc-14.2.13.tgz", - "integrity": "sha512-WwzOEAFBGhlDHE5Z73mNU8CO8mqMNLqaG+AO9ETmzdCQlJhVtWZnOl2+rqgVQS+YHunjOWptdFmNfbpwcUuEsw==", + "version": "14.2.14", + "resolved": "https://registry.npmjs.org/@next/swc-win32-x64-msvc/-/swc-win32-x64-msvc-14.2.14.tgz", + "integrity": "sha512-MZom+OvZ1NZxuRovKt1ApevjiUJTcU2PmdJKL66xUPaJeRywnbGGRWUlaAOwunD6dX+pm83vj979NTC8QXjGWg==", "cpu": [ "x64" ], @@ -1100,15 +1100,15 @@ } }, "node_modules/@radix-ui/react-checkbox": { - "version": "1.1.1", - "resolved": "https://registry.npmjs.org/@radix-ui/react-checkbox/-/react-checkbox-1.1.1.tgz", - "integrity": "sha512-0i/EKJ222Afa1FE0C6pNJxDq1itzcl3HChE9DwskA4th4KRse8ojx8a1nVcOjwJdbpDLcz7uol77yYnQNMHdKw==", + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/@radix-ui/react-checkbox/-/react-checkbox-1.1.2.tgz", + "integrity": "sha512-/i0fl686zaJbDQLNKrkCbMyDm6FQMt4jg323k7HuqitoANm9sE23Ql8yOK3Wusk34HSLKDChhMux05FnP6KUkw==", "license": "MIT", "dependencies": { "@radix-ui/primitive": "1.1.0", "@radix-ui/react-compose-refs": "1.1.0", - "@radix-ui/react-context": "1.1.0", - "@radix-ui/react-presence": "1.1.0", + "@radix-ui/react-context": "1.1.1", + "@radix-ui/react-presence": "1.1.1", "@radix-ui/react-primitive": "2.0.0", "@radix-ui/react-use-controllable-state": "1.1.0", "@radix-ui/react-use-previous": "1.1.0", @@ -1130,16 +1130,16 @@ } }, "node_modules/@radix-ui/react-collapsible": { - "version": "1.1.0", - "resolved": "https://registry.npmjs.org/@radix-ui/react-collapsible/-/react-collapsible-1.1.0.tgz", - "integrity": "sha512-zQY7Epa8sTL0mq4ajSJpjgn2YmCgyrG7RsQgLp3C0LQVkG7+Tf6Pv1CeNWZLyqMjhdPkBa5Lx7wYBeSu7uCSTA==", + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/@radix-ui/react-collapsible/-/react-collapsible-1.1.1.tgz", + "integrity": "sha512-1///SnrfQHJEofLokyczERxQbWfCGQlQ2XsCZMucVs6it+lq9iw4vXy+uDn1edlb58cOZOWSldnfPAYcT4O/Yg==", "license": "MIT", "dependencies": { "@radix-ui/primitive": "1.1.0", "@radix-ui/react-compose-refs": "1.1.0", - "@radix-ui/react-context": "1.1.0", + "@radix-ui/react-context": "1.1.1", "@radix-ui/react-id": "1.1.0", - "@radix-ui/react-presence": "1.1.0", + "@radix-ui/react-presence": "1.1.1", "@radix-ui/react-primitive": "2.0.0", "@radix-ui/react-use-controllable-state": "1.1.0", "@radix-ui/react-use-layout-effect": "1.1.0" @@ -1185,6 +1185,21 @@ } } }, + "node_modules/@radix-ui/react-collection/node_modules/@radix-ui/react-context": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@radix-ui/react-context/-/react-context-1.1.0.tgz", + "integrity": "sha512-OKrckBy+sMEgYM/sMmqmErVn0kZqrHPJze+Ql3DzYsDDp0hl0L62nx/2122/Bvps1qz645jlcu2tD9lrRSdf8A==", + "license": "MIT", + "peerDependencies": { + "@types/react": "*", + "react": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc" + }, + "peerDependenciesMeta": { + "@types/react": { + "optional": true + } + } + }, "node_modules/@radix-ui/react-compose-refs": { "version": "1.1.0", "resolved": "https://registry.npmjs.org/@radix-ui/react-compose-refs/-/react-compose-refs-1.1.0.tgz", @@ -1201,9 +1216,9 @@ } }, "node_modules/@radix-ui/react-context": { - "version": "1.1.0", - "resolved": "https://registry.npmjs.org/@radix-ui/react-context/-/react-context-1.1.0.tgz", - "integrity": "sha512-OKrckBy+sMEgYM/sMmqmErVn0kZqrHPJze+Ql3DzYsDDp0hl0L62nx/2122/Bvps1qz645jlcu2tD9lrRSdf8A==", + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/@radix-ui/react-context/-/react-context-1.1.1.tgz", + "integrity": "sha512-UASk9zi+crv9WteK/NU4PLvOoL3OuE6BWVKNF6hPRBtYBDXQ2u5iu3O59zUlJiTVvkyuycnqrztsHVJwcK9K+Q==", "license": "MIT", "peerDependencies": { "@types/react": "*", @@ -1216,25 +1231,25 @@ } }, "node_modules/@radix-ui/react-dialog": { - "version": "1.1.1", - "resolved": "https://registry.npmjs.org/@radix-ui/react-dialog/-/react-dialog-1.1.1.tgz", - "integrity": "sha512-zysS+iU4YP3STKNS6USvFVqI4qqx8EpiwmT5TuCApVEBca+eRCbONi4EgzfNSuVnOXvC5UPHHMjs8RXO6DH9Bg==", + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/@radix-ui/react-dialog/-/react-dialog-1.1.2.tgz", + "integrity": "sha512-Yj4dZtqa2o+kG61fzB0H2qUvmwBA2oyQroGLyNtBj1beo1khoQ3q1a2AO8rrQYjd8256CO9+N8L9tvsS+bnIyA==", "license": "MIT", "dependencies": { "@radix-ui/primitive": "1.1.0", "@radix-ui/react-compose-refs": "1.1.0", - "@radix-ui/react-context": "1.1.0", - "@radix-ui/react-dismissable-layer": "1.1.0", - "@radix-ui/react-focus-guards": "1.1.0", + "@radix-ui/react-context": "1.1.1", + "@radix-ui/react-dismissable-layer": "1.1.1", + "@radix-ui/react-focus-guards": "1.1.1", "@radix-ui/react-focus-scope": "1.1.0", "@radix-ui/react-id": "1.1.0", - "@radix-ui/react-portal": "1.1.1", - "@radix-ui/react-presence": "1.1.0", + "@radix-ui/react-portal": "1.1.2", + "@radix-ui/react-presence": "1.1.1", "@radix-ui/react-primitive": "2.0.0", "@radix-ui/react-slot": "1.1.0", "@radix-ui/react-use-controllable-state": "1.1.0", "aria-hidden": "^1.1.1", - "react-remove-scroll": "2.5.7" + "react-remove-scroll": "2.6.0" }, "peerDependencies": { "@types/react": "*", @@ -1267,9 +1282,9 @@ } }, "node_modules/@radix-ui/react-dismissable-layer": { - "version": "1.1.0", - "resolved": "https://registry.npmjs.org/@radix-ui/react-dismissable-layer/-/react-dismissable-layer-1.1.0.tgz", - "integrity": "sha512-/UovfmmXGptwGcBQawLzvn2jOfM0t4z3/uKffoBlj724+n3FvBbZ7M0aaBOmkp6pqFYpO4yx8tSVJjx3Fl2jig==", + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/@radix-ui/react-dismissable-layer/-/react-dismissable-layer-1.1.1.tgz", + "integrity": "sha512-QSxg29lfr/xcev6kSz7MAlmDnzbP1eI/Dwn3Tp1ip0KT5CUELsxkekFEMVBEoykI3oV39hKT4TKZzBNMbcTZYQ==", "license": "MIT", "dependencies": { "@radix-ui/primitive": "1.1.0", @@ -1294,16 +1309,16 @@ } }, "node_modules/@radix-ui/react-dropdown-menu": { - "version": "2.1.1", - "resolved": "https://registry.npmjs.org/@radix-ui/react-dropdown-menu/-/react-dropdown-menu-2.1.1.tgz", - "integrity": "sha512-y8E+x9fBq9qvteD2Zwa4397pUVhYsh9iq44b5RD5qu1GMJWBCBuVg1hMyItbc6+zH00TxGRqd9Iot4wzf3OoBQ==", + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/@radix-ui/react-dropdown-menu/-/react-dropdown-menu-2.1.2.tgz", + "integrity": "sha512-GVZMR+eqK8/Kes0a36Qrv+i20bAPXSn8rCBTHx30w+3ECnR5o3xixAlqcVaYvLeyKUsm0aqyhWfmUcqufM8nYA==", "license": "MIT", "dependencies": { "@radix-ui/primitive": "1.1.0", "@radix-ui/react-compose-refs": "1.1.0", - "@radix-ui/react-context": "1.1.0", + "@radix-ui/react-context": "1.1.1", "@radix-ui/react-id": "1.1.0", - "@radix-ui/react-menu": "2.1.1", + "@radix-ui/react-menu": "2.1.2", "@radix-ui/react-primitive": "2.0.0", "@radix-ui/react-use-controllable-state": "1.1.0" }, @@ -1323,9 +1338,9 @@ } }, "node_modules/@radix-ui/react-focus-guards": { - "version": "1.1.0", - "resolved": "https://registry.npmjs.org/@radix-ui/react-focus-guards/-/react-focus-guards-1.1.0.tgz", - "integrity": "sha512-w6XZNUPVv6xCpZUqb/yN9DL6auvpGX3C/ee6Hdi16v2UUy25HV2Q5bcflsiDyT/g5RwbPQ/GIT1vLkeRb+ITBw==", + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/@radix-ui/react-focus-guards/-/react-focus-guards-1.1.1.tgz", + "integrity": "sha512-pSIwfrT1a6sIoDASCSpFwOasEwKTZWDw/iBdtnqKO7v6FeOzYJ7U53cPzYFVR3geGGXgVHaH+CdngrrAzqUGxg==", "license": "MIT", "peerDependencies": { "@types/react": "*", @@ -1390,6 +1405,21 @@ } } }, + "node_modules/@radix-ui/react-form/node_modules/@radix-ui/react-context": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@radix-ui/react-context/-/react-context-1.1.0.tgz", + "integrity": "sha512-OKrckBy+sMEgYM/sMmqmErVn0kZqrHPJze+Ql3DzYsDDp0hl0L62nx/2122/Bvps1qz645jlcu2tD9lrRSdf8A==", + "license": "MIT", + "peerDependencies": { + "@types/react": "*", + "react": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc" + }, + "peerDependenciesMeta": { + "@types/react": { + "optional": true + } + } + }, "node_modules/@radix-ui/react-icons": { "version": "1.3.0", "resolved": "https://registry.npmjs.org/@radix-ui/react-icons/-/react-icons-1.3.0.tgz", @@ -1441,29 +1471,29 @@ } }, "node_modules/@radix-ui/react-menu": { - "version": "2.1.1", - "resolved": "https://registry.npmjs.org/@radix-ui/react-menu/-/react-menu-2.1.1.tgz", - "integrity": "sha512-oa3mXRRVjHi6DZu/ghuzdylyjaMXLymx83irM7hTxutQbD+7IhPKdMdRHD26Rm+kHRrWcrUkkRPv5pd47a2xFQ==", + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/@radix-ui/react-menu/-/react-menu-2.1.2.tgz", + "integrity": "sha512-lZ0R4qR2Al6fZ4yCCZzu/ReTFrylHFxIqy7OezIpWF4bL0o9biKo0pFIvkaew3TyZ9Fy5gYVrR5zCGZBVbO1zg==", "license": "MIT", "dependencies": { "@radix-ui/primitive": "1.1.0", "@radix-ui/react-collection": "1.1.0", "@radix-ui/react-compose-refs": "1.1.0", - "@radix-ui/react-context": "1.1.0", + "@radix-ui/react-context": "1.1.1", "@radix-ui/react-direction": "1.1.0", - "@radix-ui/react-dismissable-layer": "1.1.0", - "@radix-ui/react-focus-guards": "1.1.0", + "@radix-ui/react-dismissable-layer": "1.1.1", + "@radix-ui/react-focus-guards": "1.1.1", "@radix-ui/react-focus-scope": "1.1.0", "@radix-ui/react-id": "1.1.0", "@radix-ui/react-popper": "1.2.0", - "@radix-ui/react-portal": "1.1.1", - "@radix-ui/react-presence": "1.1.0", + "@radix-ui/react-portal": "1.1.2", + "@radix-ui/react-presence": "1.1.1", "@radix-ui/react-primitive": "2.0.0", "@radix-ui/react-roving-focus": "1.1.0", "@radix-ui/react-slot": "1.1.0", "@radix-ui/react-use-callback-ref": "1.1.0", "aria-hidden": "^1.1.1", - "react-remove-scroll": "2.5.7" + "react-remove-scroll": "2.6.0" }, "peerDependencies": { "@types/react": "*", @@ -1481,26 +1511,26 @@ } }, "node_modules/@radix-ui/react-popover": { - "version": "1.1.1", - "resolved": "https://registry.npmjs.org/@radix-ui/react-popover/-/react-popover-1.1.1.tgz", - "integrity": "sha512-3y1A3isulwnWhvTTwmIreiB8CF4L+qRjZnK1wYLO7pplddzXKby/GnZ2M7OZY3qgnl6p9AodUIHRYGXNah8Y7g==", + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/@radix-ui/react-popover/-/react-popover-1.1.2.tgz", + "integrity": "sha512-u2HRUyWW+lOiA2g0Le0tMmT55FGOEWHwPFt1EPfbLly7uXQExFo5duNKqG2DzmFXIdqOeNd+TpE8baHWJCyP9w==", "license": "MIT", "dependencies": { "@radix-ui/primitive": "1.1.0", "@radix-ui/react-compose-refs": "1.1.0", - "@radix-ui/react-context": "1.1.0", - "@radix-ui/react-dismissable-layer": "1.1.0", - "@radix-ui/react-focus-guards": "1.1.0", + "@radix-ui/react-context": "1.1.1", + "@radix-ui/react-dismissable-layer": "1.1.1", + "@radix-ui/react-focus-guards": "1.1.1", "@radix-ui/react-focus-scope": "1.1.0", "@radix-ui/react-id": "1.1.0", "@radix-ui/react-popper": "1.2.0", - "@radix-ui/react-portal": "1.1.1", - "@radix-ui/react-presence": "1.1.0", + "@radix-ui/react-portal": "1.1.2", + "@radix-ui/react-presence": "1.1.1", "@radix-ui/react-primitive": "2.0.0", "@radix-ui/react-slot": "1.1.0", "@radix-ui/react-use-controllable-state": "1.1.0", "aria-hidden": "^1.1.1", - "react-remove-scroll": "2.5.7" + "react-remove-scroll": "2.6.0" }, "peerDependencies": { "@types/react": "*", @@ -1549,10 +1579,25 @@ } } }, + "node_modules/@radix-ui/react-popper/node_modules/@radix-ui/react-context": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@radix-ui/react-context/-/react-context-1.1.0.tgz", + "integrity": "sha512-OKrckBy+sMEgYM/sMmqmErVn0kZqrHPJze+Ql3DzYsDDp0hl0L62nx/2122/Bvps1qz645jlcu2tD9lrRSdf8A==", + "license": "MIT", + "peerDependencies": { + "@types/react": "*", + "react": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc" + }, + "peerDependenciesMeta": { + "@types/react": { + "optional": true + } + } + }, "node_modules/@radix-ui/react-portal": { - "version": "1.1.1", - "resolved": "https://registry.npmjs.org/@radix-ui/react-portal/-/react-portal-1.1.1.tgz", - "integrity": "sha512-A3UtLk85UtqhzFqtoC8Q0KvR2GbXF3mtPgACSazajqq6A41mEQgo53iPzY4i6BwDxlIFqWIhiQ2G729n+2aw/g==", + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/@radix-ui/react-portal/-/react-portal-1.1.2.tgz", + "integrity": "sha512-WeDYLGPxJb/5EGBoedyJbT0MpoULmwnIPMJMSldkuiMsBAv7N1cRdsTWZWht9vpPOiN3qyiGAtbK2is47/uMFg==", "license": "MIT", "dependencies": { "@radix-ui/react-primitive": "2.0.0", @@ -1574,9 +1619,9 @@ } }, "node_modules/@radix-ui/react-presence": { - "version": "1.1.0", - "resolved": "https://registry.npmjs.org/@radix-ui/react-presence/-/react-presence-1.1.0.tgz", - "integrity": "sha512-Gq6wuRN/asf9H/E/VzdKoUtT8GC9PQc9z40/vEr0VCJ4u5XvvhWIrSsCB6vD2/cH7ugTdSfYq9fLJCcM00acrQ==", + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/@radix-ui/react-presence/-/react-presence-1.1.1.tgz", + "integrity": "sha512-IeFXVi4YS1K0wVZzXNrbaaUvIJ3qdY+/Ih4eHFhWA9SwGR9UDX7Ck8abvL57C4cv3wwMvUE0OG69Qc3NCcTe/A==", "license": "MIT", "dependencies": { "@radix-ui/react-compose-refs": "1.1.0", @@ -1644,17 +1689,32 @@ } } }, + "node_modules/@radix-ui/react-progress/node_modules/@radix-ui/react-context": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@radix-ui/react-context/-/react-context-1.1.0.tgz", + "integrity": "sha512-OKrckBy+sMEgYM/sMmqmErVn0kZqrHPJze+Ql3DzYsDDp0hl0L62nx/2122/Bvps1qz645jlcu2tD9lrRSdf8A==", + "license": "MIT", + "peerDependencies": { + "@types/react": "*", + "react": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc" + }, + "peerDependenciesMeta": { + "@types/react": { + "optional": true + } + } + }, "node_modules/@radix-ui/react-radio-group": { - "version": "1.2.0", - "resolved": "https://registry.npmjs.org/@radix-ui/react-radio-group/-/react-radio-group-1.2.0.tgz", - "integrity": "sha512-yv+oiLaicYMBpqgfpSPw6q+RyXlLdIpQWDHZbUKURxe+nEh53hFXPPlfhfQQtYkS5MMK/5IWIa76SksleQZSzw==", + "version": "1.2.1", + "resolved": "https://registry.npmjs.org/@radix-ui/react-radio-group/-/react-radio-group-1.2.1.tgz", + "integrity": "sha512-kdbv54g4vfRjja9DNWPMxKvXblzqbpEC8kspEkZ6dVP7kQksGCn+iZHkcCz2nb00+lPdRvxrqy4WrvvV1cNqrQ==", "license": "MIT", "dependencies": { "@radix-ui/primitive": "1.1.0", "@radix-ui/react-compose-refs": "1.1.0", - "@radix-ui/react-context": "1.1.0", + "@radix-ui/react-context": "1.1.1", "@radix-ui/react-direction": "1.1.0", - "@radix-ui/react-presence": "1.1.0", + "@radix-ui/react-presence": "1.1.1", "@radix-ui/react-primitive": "2.0.0", "@radix-ui/react-roving-focus": "1.1.0", "@radix-ui/react-use-controllable-state": "1.1.0", @@ -1707,6 +1767,21 @@ } } }, + "node_modules/@radix-ui/react-roving-focus/node_modules/@radix-ui/react-context": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@radix-ui/react-context/-/react-context-1.1.0.tgz", + "integrity": "sha512-OKrckBy+sMEgYM/sMmqmErVn0kZqrHPJze+Ql3DzYsDDp0hl0L62nx/2122/Bvps1qz645jlcu2tD9lrRSdf8A==", + "license": "MIT", + "peerDependencies": { + "@types/react": "*", + "react": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc" + }, + "peerDependenciesMeta": { + "@types/react": { + "optional": true + } + } + }, "node_modules/@radix-ui/react-slot": { "version": "1.1.0", "resolved": "https://registry.npmjs.org/@radix-ui/react-slot/-/react-slot-1.1.0.tgz", @@ -1726,14 +1801,14 @@ } }, "node_modules/@radix-ui/react-switch": { - "version": "1.1.0", - "resolved": "https://registry.npmjs.org/@radix-ui/react-switch/-/react-switch-1.1.0.tgz", - "integrity": "sha512-OBzy5WAj641k0AOSpKQtreDMe+isX0MQJ1IVyF03ucdF3DunOnROVrjWs8zsXUxC3zfZ6JL9HFVCUlMghz9dJw==", + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/@radix-ui/react-switch/-/react-switch-1.1.1.tgz", + "integrity": "sha512-diPqDDoBcZPSicYoMWdWx+bCPuTRH4QSp9J+65IvtdS0Kuzt67bI6n32vCj8q6NZmYW/ah+2orOtMwcX5eQwIg==", "license": "MIT", "dependencies": { "@radix-ui/primitive": "1.1.0", "@radix-ui/react-compose-refs": "1.1.0", - "@radix-ui/react-context": "1.1.0", + "@radix-ui/react-context": "1.1.1", "@radix-ui/react-primitive": "2.0.0", "@radix-ui/react-use-controllable-state": "1.1.0", "@radix-ui/react-use-previous": "1.1.0", @@ -1755,16 +1830,16 @@ } }, "node_modules/@radix-ui/react-tabs": { - "version": "1.1.0", - "resolved": "https://registry.npmjs.org/@radix-ui/react-tabs/-/react-tabs-1.1.0.tgz", - "integrity": "sha512-bZgOKB/LtZIij75FSuPzyEti/XBhJH52ExgtdVqjCIh+Nx/FW+LhnbXtbCzIi34ccyMsyOja8T0thCzoHFXNKA==", + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/@radix-ui/react-tabs/-/react-tabs-1.1.1.tgz", + "integrity": "sha512-3GBUDmP2DvzmtYLMsHmpA1GtR46ZDZ+OreXM/N+kkQJOPIgytFWWTfDQmBQKBvaFS0Vno0FktdbVzN28KGrMdw==", "license": "MIT", "dependencies": { "@radix-ui/primitive": "1.1.0", - "@radix-ui/react-context": "1.1.0", + "@radix-ui/react-context": "1.1.1", "@radix-ui/react-direction": "1.1.0", "@radix-ui/react-id": "1.1.0", - "@radix-ui/react-presence": "1.1.0", + "@radix-ui/react-presence": "1.1.1", "@radix-ui/react-primitive": "2.0.0", "@radix-ui/react-roving-focus": "1.1.0", "@radix-ui/react-use-controllable-state": "1.1.0" @@ -1785,18 +1860,18 @@ } }, "node_modules/@radix-ui/react-toast": { - "version": "1.2.1", - "resolved": "https://registry.npmjs.org/@radix-ui/react-toast/-/react-toast-1.2.1.tgz", - "integrity": "sha512-5trl7piMXcZiCq7MW6r8YYmu0bK5qDpTWz+FdEPdKyft2UixkspheYbjbrLXVN5NGKHFbOP7lm8eD0biiSqZqg==", + "version": "1.2.2", + "resolved": "https://registry.npmjs.org/@radix-ui/react-toast/-/react-toast-1.2.2.tgz", + "integrity": "sha512-Z6pqSzmAP/bFJoqMAston4eSNa+ud44NSZTiZUmUen+IOZ5nBY8kzuU5WDBVyFXPtcW6yUalOHsxM/BP6Sv8ww==", "license": "MIT", "dependencies": { "@radix-ui/primitive": "1.1.0", "@radix-ui/react-collection": "1.1.0", "@radix-ui/react-compose-refs": "1.1.0", - "@radix-ui/react-context": "1.1.0", - "@radix-ui/react-dismissable-layer": "1.1.0", - "@radix-ui/react-portal": "1.1.1", - "@radix-ui/react-presence": "1.1.0", + "@radix-ui/react-context": "1.1.1", + "@radix-ui/react-dismissable-layer": "1.1.1", + "@radix-ui/react-portal": "1.1.2", + "@radix-ui/react-presence": "1.1.1", "@radix-ui/react-primitive": "2.0.0", "@radix-ui/react-use-callback-ref": "1.1.0", "@radix-ui/react-use-controllable-state": "1.1.0", @@ -1872,20 +1947,35 @@ } } }, + "node_modules/@radix-ui/react-toggle-group/node_modules/@radix-ui/react-context": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/@radix-ui/react-context/-/react-context-1.1.0.tgz", + "integrity": "sha512-OKrckBy+sMEgYM/sMmqmErVn0kZqrHPJze+Ql3DzYsDDp0hl0L62nx/2122/Bvps1qz645jlcu2tD9lrRSdf8A==", + "license": "MIT", + "peerDependencies": { + "@types/react": "*", + "react": "^16.8 || ^17.0 || ^18.0 || ^19.0 || ^19.0.0-rc" + }, + "peerDependenciesMeta": { + "@types/react": { + "optional": true + } + } + }, "node_modules/@radix-ui/react-tooltip": { - "version": "1.1.2", - "resolved": "https://registry.npmjs.org/@radix-ui/react-tooltip/-/react-tooltip-1.1.2.tgz", - "integrity": "sha512-9XRsLwe6Yb9B/tlnYCPVUd/TFS4J7HuOZW345DCeC6vKIxQGMZdx21RK4VoZauPD5frgkXTYVS5y90L+3YBn4w==", + "version": "1.1.3", + "resolved": "https://registry.npmjs.org/@radix-ui/react-tooltip/-/react-tooltip-1.1.3.tgz", + "integrity": "sha512-Z4w1FIS0BqVFI2c1jZvb/uDVJijJjJ2ZMuPV81oVgTZ7g3BZxobplnMVvXtFWgtozdvYJ+MFWtwkM5S2HnAong==", "license": "MIT", "dependencies": { "@radix-ui/primitive": "1.1.0", "@radix-ui/react-compose-refs": "1.1.0", - "@radix-ui/react-context": "1.1.0", - "@radix-ui/react-dismissable-layer": "1.1.0", + "@radix-ui/react-context": "1.1.1", + "@radix-ui/react-dismissable-layer": "1.1.1", "@radix-ui/react-id": "1.1.0", "@radix-ui/react-popper": "1.2.0", - "@radix-ui/react-portal": "1.1.1", - "@radix-ui/react-presence": "1.1.0", + "@radix-ui/react-portal": "1.1.2", + "@radix-ui/react-presence": "1.1.1", "@radix-ui/react-primitive": "2.0.0", "@radix-ui/react-slot": "1.1.0", "@radix-ui/react-use-controllable-state": "1.1.0", @@ -2248,9 +2338,9 @@ "license": "MIT" }, "node_modules/@types/react": { - "version": "18.3.10", - "resolved": "https://registry.npmjs.org/@types/react/-/react-18.3.10.tgz", - "integrity": "sha512-02sAAlBnP39JgXwkAq3PeU9DVaaGpZyF3MGcC0MKgQVkZor5IiiDAipVaxQHtDJAmO4GIy/rVBy/LzVj76Cyqg==", + "version": "18.3.11", + "resolved": "https://registry.npmjs.org/@types/react/-/react-18.3.11.tgz", + "integrity": "sha512-r6QZ069rFTjrEYgFdOck1gK7FLVsgJE7tTz0pQBczlBNUhBNk0MQH4UbnFSwjpQLMkLzgqvBBa+qGpLje16eTQ==", "license": "MIT", "dependencies": { "@types/prop-types": "*", @@ -2282,17 +2372,17 @@ "license": "MIT" }, "node_modules/@typescript-eslint/eslint-plugin": { - "version": "8.7.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/eslint-plugin/-/eslint-plugin-8.7.0.tgz", - "integrity": "sha512-RIHOoznhA3CCfSTFiB6kBGLQtB/sox+pJ6jeFu6FxJvqL8qRxq/FfGO/UhsGgQM9oGdXkV4xUgli+dt26biB6A==", + "version": "8.8.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/eslint-plugin/-/eslint-plugin-8.8.0.tgz", + "integrity": "sha512-wORFWjU30B2WJ/aXBfOm1LX9v9nyt9D3jsSOxC3cCaTQGCW5k4jNpmjFv3U7p/7s4yvdjHzwtv2Sd2dOyhjS0A==", "dev": true, "license": "MIT", "dependencies": { "@eslint-community/regexpp": "^4.10.0", - "@typescript-eslint/scope-manager": "8.7.0", - "@typescript-eslint/type-utils": "8.7.0", - "@typescript-eslint/utils": "8.7.0", - "@typescript-eslint/visitor-keys": "8.7.0", + "@typescript-eslint/scope-manager": "8.8.0", + "@typescript-eslint/type-utils": "8.8.0", + "@typescript-eslint/utils": "8.8.0", + "@typescript-eslint/visitor-keys": "8.8.0", "graphemer": "^1.4.0", "ignore": "^5.3.1", "natural-compare": "^1.4.0", @@ -2316,16 +2406,16 @@ } }, "node_modules/@typescript-eslint/parser": { - "version": "8.7.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/parser/-/parser-8.7.0.tgz", - "integrity": "sha512-lN0btVpj2unxHlNYLI//BQ7nzbMJYBVQX5+pbNXvGYazdlgYonMn4AhhHifQ+J4fGRYA/m1DjaQjx+fDetqBOQ==", + "version": "8.8.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/parser/-/parser-8.8.0.tgz", + "integrity": "sha512-uEFUsgR+tl8GmzmLjRqz+VrDv4eoaMqMXW7ruXfgThaAShO9JTciKpEsB+TvnfFfbg5IpujgMXVV36gOJRLtZg==", "dev": true, "license": "BSD-2-Clause", "dependencies": { - "@typescript-eslint/scope-manager": "8.7.0", - "@typescript-eslint/types": "8.7.0", - "@typescript-eslint/typescript-estree": "8.7.0", - "@typescript-eslint/visitor-keys": "8.7.0", + "@typescript-eslint/scope-manager": "8.8.0", + "@typescript-eslint/types": "8.8.0", + "@typescript-eslint/typescript-estree": "8.8.0", + "@typescript-eslint/visitor-keys": "8.8.0", "debug": "^4.3.4" }, "engines": { @@ -2345,14 +2435,14 @@ } }, "node_modules/@typescript-eslint/scope-manager": { - "version": "8.7.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/scope-manager/-/scope-manager-8.7.0.tgz", - "integrity": "sha512-87rC0k3ZlDOuz82zzXRtQ7Akv3GKhHs0ti4YcbAJtaomllXoSO8hi7Ix3ccEvCd824dy9aIX+j3d2UMAfCtVpg==", + "version": "8.8.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/scope-manager/-/scope-manager-8.8.0.tgz", + "integrity": "sha512-EL8eaGC6gx3jDd8GwEFEV091210U97J0jeEHrAYvIYosmEGet4wJ+g0SYmLu+oRiAwbSA5AVrt6DxLHfdd+bUg==", "dev": true, "license": "MIT", "dependencies": { - "@typescript-eslint/types": "8.7.0", - "@typescript-eslint/visitor-keys": "8.7.0" + "@typescript-eslint/types": "8.8.0", + "@typescript-eslint/visitor-keys": "8.8.0" }, "engines": { "node": "^18.18.0 || ^20.9.0 || >=21.1.0" @@ -2363,14 +2453,14 @@ } }, "node_modules/@typescript-eslint/type-utils": { - "version": "8.7.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/type-utils/-/type-utils-8.7.0.tgz", - "integrity": "sha512-tl0N0Mj3hMSkEYhLkjREp54OSb/FI6qyCzfiiclvJvOqre6hsZTGSnHtmFLDU8TIM62G7ygEa1bI08lcuRwEnQ==", + "version": "8.8.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/type-utils/-/type-utils-8.8.0.tgz", + "integrity": "sha512-IKwJSS7bCqyCeG4NVGxnOP6lLT9Okc3Zj8hLO96bpMkJab+10HIfJbMouLrlpyOr3yrQ1cA413YPFiGd1mW9/Q==", "dev": true, "license": "MIT", "dependencies": { - "@typescript-eslint/typescript-estree": "8.7.0", - "@typescript-eslint/utils": "8.7.0", + "@typescript-eslint/typescript-estree": "8.8.0", + "@typescript-eslint/utils": "8.8.0", "debug": "^4.3.4", "ts-api-utils": "^1.3.0" }, @@ -2388,9 +2478,9 @@ } }, "node_modules/@typescript-eslint/types": { - "version": "8.7.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/types/-/types-8.7.0.tgz", - "integrity": "sha512-LLt4BLHFwSfASHSF2K29SZ+ZCsbQOM+LuarPjRUuHm+Qd09hSe3GCeaQbcCr+Mik+0QFRmep/FyZBO6fJ64U3w==", + "version": "8.8.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/types/-/types-8.8.0.tgz", + "integrity": "sha512-QJwc50hRCgBd/k12sTykOJbESe1RrzmX6COk8Y525C9l7oweZ+1lw9JiU56im7Amm8swlz00DRIlxMYLizr2Vw==", "dev": true, "license": "MIT", "engines": { @@ -2402,14 +2492,14 @@ } }, "node_modules/@typescript-eslint/typescript-estree": { - "version": "8.7.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/typescript-estree/-/typescript-estree-8.7.0.tgz", - "integrity": "sha512-MC8nmcGHsmfAKxwnluTQpNqceniT8SteVwd2voYlmiSWGOtjvGXdPl17dYu2797GVscK30Z04WRM28CrKS9WOg==", + "version": "8.8.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/typescript-estree/-/typescript-estree-8.8.0.tgz", + "integrity": "sha512-ZaMJwc/0ckLz5DaAZ+pNLmHv8AMVGtfWxZe/x2JVEkD5LnmhWiQMMcYT7IY7gkdJuzJ9P14fRy28lUrlDSWYdw==", "dev": true, "license": "BSD-2-Clause", "dependencies": { - "@typescript-eslint/types": "8.7.0", - "@typescript-eslint/visitor-keys": "8.7.0", + "@typescript-eslint/types": "8.8.0", + "@typescript-eslint/visitor-keys": "8.8.0", "debug": "^4.3.4", "fast-glob": "^3.3.2", "is-glob": "^4.0.3", @@ -2457,16 +2547,16 @@ } }, "node_modules/@typescript-eslint/utils": { - "version": "8.7.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/utils/-/utils-8.7.0.tgz", - "integrity": "sha512-ZbdUdwsl2X/s3CiyAu3gOlfQzpbuG3nTWKPoIvAu1pu5r8viiJvv2NPN2AqArL35NCYtw/lrPPfM4gxrMLNLPw==", + "version": "8.8.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/utils/-/utils-8.8.0.tgz", + "integrity": "sha512-QE2MgfOTem00qrlPgyByaCHay9yb1+9BjnMFnSFkUKQfu7adBXDTnCAivURnuPPAG/qiB+kzKkZKmKfaMT0zVg==", "dev": true, "license": "MIT", "dependencies": { "@eslint-community/eslint-utils": "^4.4.0", - "@typescript-eslint/scope-manager": "8.7.0", - "@typescript-eslint/types": "8.7.0", - "@typescript-eslint/typescript-estree": "8.7.0" + "@typescript-eslint/scope-manager": "8.8.0", + "@typescript-eslint/types": "8.8.0", + "@typescript-eslint/typescript-estree": "8.8.0" }, "engines": { "node": "^18.18.0 || ^20.9.0 || >=21.1.0" @@ -2480,13 +2570,13 @@ } }, "node_modules/@typescript-eslint/visitor-keys": { - "version": "8.7.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/visitor-keys/-/visitor-keys-8.7.0.tgz", - "integrity": "sha512-b1tx0orFCCh/THWPQa2ZwWzvOeyzzp36vkJYOpVg0u8UVOIsfVrnuC9FqAw9gRKn+rG2VmWQ/zDJZzkxUnj/XQ==", + "version": "8.8.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/visitor-keys/-/visitor-keys-8.8.0.tgz", + "integrity": "sha512-8mq51Lx6Hpmd7HnA2fcHQo3YgfX1qbccxQOgZcb4tvasu//zXRaA1j5ZRFeCw/VRAdFi4mRM9DnZw0Nu0Q2d1g==", "dev": true, "license": "MIT", "dependencies": { - "@typescript-eslint/types": "8.7.0", + "@typescript-eslint/types": "8.8.0", "eslint-visitor-keys": "^3.4.3" }, "engines": { @@ -3283,9 +3373,9 @@ } }, "node_modules/caniuse-lite": { - "version": "1.0.30001664", - "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001664.tgz", - "integrity": "sha512-AmE7k4dXiNKQipgn7a2xg558IRqPN3jMQY/rOsbxDhrd0tyChwbITBfiwtnqz8bi2M5mIWbxAYBvk7W7QBUS2g==", + "version": "1.0.30001667", + "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001667.tgz", + "integrity": "sha512-7LTwJjcRkzKFmtqGsibMeuXmvFDfZq/nzIjnmgCGzKKRVzjD72selLDK1oPF/Oxzmt4fNcPvTDvGqSDG4tCALw==", "funding": [ { "type": "opencollective", @@ -3897,9 +3987,9 @@ "license": "MIT" }, "node_modules/electron-to-chromium": { - "version": "1.5.29", - "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.5.29.tgz", - "integrity": "sha512-PF8n2AlIhCKXQ+gTpiJi0VhcHDb69kYX4MtCiivctc2QD3XuNZ/XIOlbGzt7WAjjEev0TtaH6Cu3arZExm5DOw==", + "version": "1.5.32", + "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.5.32.tgz", + "integrity": "sha512-M+7ph0VGBQqqpTT2YrabjNKSQ2fEl9PVx6AK3N558gDH9NO8O6XN9SXXFWRo9u9PbEg/bWq+tjXQr+eXmxubCw==", "dev": true, "license": "ISC" }, @@ -4173,6 +4263,7 @@ "version": "8.57.1", "resolved": "https://registry.npmjs.org/eslint/-/eslint-8.57.1.tgz", "integrity": "sha512-ypowyDxpVSYpkXr9WPv2PAZCtNip1Mv5KTW0SCurXv/9iOpcrH9PaqUElksqEB6pChqHGDRCFTyrZlGhnLNGiA==", + "deprecated": "This version is no longer supported. Please see https://eslint.org/version-support for other options.", "dev": true, "license": "MIT", "dependencies": { @@ -4226,13 +4317,13 @@ } }, "node_modules/eslint-config-next": { - "version": "14.2.13", - "resolved": "https://registry.npmjs.org/eslint-config-next/-/eslint-config-next-14.2.13.tgz", - "integrity": "sha512-aro1EKAoyYchnO/3Tlo91hnNBO7QO7qnv/79MAFC+4Jq8TdUVKQlht5d2F+YjrePjdpOvfL+mV9JPfyYNwkk1g==", + "version": "14.2.14", + "resolved": "https://registry.npmjs.org/eslint-config-next/-/eslint-config-next-14.2.14.tgz", + "integrity": "sha512-TXwyjGICAlWC9O0OufS3koTsBKQH8l1xt3SY/aDuvtKHIwjTHplJKWVb1WOEX0OsDaxGbFXmfD2EY1sNfG0Y/w==", "dev": true, "license": "MIT", "dependencies": { - "@next/eslint-plugin-next": "14.2.13", + "@next/eslint-plugin-next": "14.2.14", "@rushstack/eslint-patch": "^1.3.3", "@typescript-eslint/eslint-plugin": "^5.4.2 || ^6.0.0 || ^7.0.0 || ^8.0.0", "@typescript-eslint/parser": "^5.4.2 || ^6.0.0 || ^7.0.0 || ^8.0.0", @@ -4353,9 +4444,9 @@ } }, "node_modules/eslint-plugin-import": { - "version": "2.30.0", - "resolved": "https://registry.npmjs.org/eslint-plugin-import/-/eslint-plugin-import-2.30.0.tgz", - "integrity": "sha512-/mHNE9jINJfiD2EKkg1BKyPyUk4zdnT54YgbOgfjSakWT5oyX/qQLVNTkehyfpcMxZXMy1zyonZ2v7hZTX43Yw==", + "version": "2.31.0", + "resolved": "https://registry.npmjs.org/eslint-plugin-import/-/eslint-plugin-import-2.31.0.tgz", + "integrity": "sha512-ixmkI62Rbc2/w8Vfxyh1jQRTdRTF52VxwRVHl/ykPAmqG+Nb7/kNn+byLP0LxPgI7zWA16Jt82SybJInmMia3A==", "dev": true, "license": "MIT", "dependencies": { @@ -4367,7 +4458,7 @@ "debug": "^3.2.7", "doctrine": "^2.1.0", "eslint-import-resolver-node": "^0.3.9", - "eslint-module-utils": "^2.9.0", + "eslint-module-utils": "^2.12.0", "hasown": "^2.0.2", "is-core-module": "^2.15.1", "is-glob": "^4.0.3", @@ -4376,13 +4467,14 @@ "object.groupby": "^1.0.3", "object.values": "^1.2.0", "semver": "^6.3.1", + "string.prototype.trimend": "^1.0.8", "tsconfig-paths": "^3.15.0" }, "engines": { "node": ">=4" }, "peerDependencies": { - "eslint": "^2 || ^3 || ^4 || ^5 || ^6 || ^7.2.0 || ^8" + "eslint": "^2 || ^3 || ^4 || ^5 || ^6 || ^7.2.0 || ^8 || ^9" } }, "node_modules/eslint-plugin-import/node_modules/debug": { @@ -4450,9 +4542,9 @@ } }, "node_modules/eslint-plugin-react": { - "version": "7.37.0", - "resolved": "https://registry.npmjs.org/eslint-plugin-react/-/eslint-plugin-react-7.37.0.tgz", - "integrity": "sha512-IHBePmfWH5lKhJnJ7WB1V+v/GolbB0rjS8XYVCSQCZKaQCAUhMoVoOEn1Ef8Z8Wf0a7l8KTJvuZg5/e4qrZ6nA==", + "version": "7.37.1", + "resolved": "https://registry.npmjs.org/eslint-plugin-react/-/eslint-plugin-react-7.37.1.tgz", + "integrity": "sha512-xwTnwDqzbDRA8uJ7BMxPs/EXRB3i8ZfnOIp8BsxEQkT0nHPp+WWceqGgo6rKb9ctNi8GJLDT4Go5HAWELa/WMg==", "dev": true, "license": "MIT", "dependencies": { @@ -5988,15 +6080,15 @@ } }, "node_modules/jsesc": { - "version": "2.5.2", - "resolved": "https://registry.npmjs.org/jsesc/-/jsesc-2.5.2.tgz", - "integrity": "sha512-OYu7XEzjkCQ3C5Ps3QIZsQfNpqoJyZZA99wd9aWd05NCtC5pWOkShK2mkL6HXQR6/Cy2lbNdPlZBpuQHXE63gA==", + "version": "3.0.2", + "resolved": "https://registry.npmjs.org/jsesc/-/jsesc-3.0.2.tgz", + "integrity": "sha512-xKqzzWXDttJuOcawBt4KnKHHIf5oQ/Cxax+0PWFG+DFDgHNAdi+TXECADI+RYiFUMmx8792xsMbbgXj4CwnP4g==", "license": "MIT", "bin": { "jsesc": "bin/jsesc" }, "engines": { - "node": ">=4" + "node": ">=6" } }, "node_modules/json-buffer": { @@ -6267,9 +6359,9 @@ } }, "node_modules/lucide-react": { - "version": "0.446.0", - "resolved": "https://registry.npmjs.org/lucide-react/-/lucide-react-0.446.0.tgz", - "integrity": "sha512-BU7gy8MfBMqvEdDPH79VhOXSEgyG8TSPOKWaExWGCQVqnGH7wGgDngPbofu+KdtVjPQBWbEmnfMTq90CTiiDRg==", + "version": "0.447.0", + "resolved": "https://registry.npmjs.org/lucide-react/-/lucide-react-0.447.0.tgz", + "integrity": "sha512-SZ//hQmvi+kDKrNepArVkYK7/jfeZ5uFNEnYmd45RKZcbGD78KLnrcNXmgeg6m+xNHFvTG+CblszXCy4n6DN4w==", "license": "ISC", "peerDependencies": { "react": "^16.5.1 || ^17.0.0 || ^18.0.0 || ^19.0.0-rc" @@ -6302,9 +6394,9 @@ } }, "node_modules/material-symbols": { - "version": "0.23.0", - "resolved": "https://registry.npmjs.org/material-symbols/-/material-symbols-0.23.0.tgz", - "integrity": "sha512-ciq9bPgw5bFGC807lT3KgbvU97bC7w6DeUQmqLXdIHD28L/FD9ajvX11swtQXcMV/nEjQ/fI6u5NVtli18qUqQ==", + "version": "0.24.0", + "resolved": "https://registry.npmjs.org/material-symbols/-/material-symbols-0.24.0.tgz", + "integrity": "sha512-PfQ6S+vuYmm9F8f4xUwF8RfahbKxBPFbkfIe6du66Lyh5jFohqC4s2J87PB8L8T76TwqsnJLza1Wnt08NKtBtw==", "license": "Apache-2.0" }, "node_modules/memoize-one": { @@ -6421,9 +6513,9 @@ } }, "node_modules/moment-timezone": { - "version": "0.5.45", - "resolved": "https://registry.npmjs.org/moment-timezone/-/moment-timezone-0.5.45.tgz", - "integrity": "sha512-HIWmqA86KcmCAhnMAN0wuDOARV/525R2+lOLotuGFzn4HO+FH+/645z2wx0Dt3iDv6/p61SIvKnDstISainhLQ==", + "version": "0.5.46", + "resolved": "https://registry.npmjs.org/moment-timezone/-/moment-timezone-0.5.46.tgz", + "integrity": "sha512-ZXm9b36esbe7OmdABqIWJuBBiLLwAjrN7CE+7sYdCCx82Nabt1wHDj8TVseS59QIlfFPbOoiBPm6ca9BioG4hw==", "license": "MIT", "dependencies": { "moment": "^2.29.4" @@ -6507,12 +6599,12 @@ "license": "MIT" }, "node_modules/next": { - "version": "14.2.13", - "resolved": "https://registry.npmjs.org/next/-/next-14.2.13.tgz", - "integrity": "sha512-BseY9YNw8QJSwLYD7hlZzl6QVDoSFHL/URN5K64kVEVpCsSOWeyjbIGK+dZUaRViHTaMQX8aqmnn0PHBbGZezg==", + "version": "14.2.14", + "resolved": "https://registry.npmjs.org/next/-/next-14.2.14.tgz", + "integrity": "sha512-Q1coZG17MW0Ly5x76shJ4dkC23woLAhhnDnw+DfTc7EpZSGuWrlsZ3bZaO8t6u1Yu8FVfhkqJE+U8GC7E0GLPQ==", "license": "MIT", "dependencies": { - "@next/env": "14.2.13", + "@next/env": "14.2.14", "@swc/helpers": "0.5.5", "busboy": "1.6.0", "caniuse-lite": "^1.0.30001579", @@ -6527,15 +6619,15 @@ "node": ">=18.17.0" }, "optionalDependencies": { - "@next/swc-darwin-arm64": "14.2.13", - "@next/swc-darwin-x64": "14.2.13", - "@next/swc-linux-arm64-gnu": "14.2.13", - "@next/swc-linux-arm64-musl": "14.2.13", - "@next/swc-linux-x64-gnu": "14.2.13", - "@next/swc-linux-x64-musl": "14.2.13", - "@next/swc-win32-arm64-msvc": "14.2.13", - "@next/swc-win32-ia32-msvc": "14.2.13", - "@next/swc-win32-x64-msvc": "14.2.13" + "@next/swc-darwin-arm64": "14.2.14", + "@next/swc-darwin-x64": "14.2.14", + "@next/swc-linux-arm64-gnu": "14.2.14", + "@next/swc-linux-arm64-musl": "14.2.14", + "@next/swc-linux-x64-gnu": "14.2.14", + "@next/swc-linux-x64-musl": "14.2.14", + "@next/swc-win32-arm64-msvc": "14.2.14", + "@next/swc-win32-ia32-msvc": "14.2.14", + "@next/swc-win32-x64-msvc": "14.2.14" }, "peerDependencies": { "@opentelemetry/api": "^1.1.0", @@ -7302,9 +7394,9 @@ "license": "MIT" }, "node_modules/preact": { - "version": "10.24.1", - "resolved": "https://registry.npmjs.org/preact/-/preact-10.24.1.tgz", - "integrity": "sha512-PnBAwFI3Yjxxcxw75n6VId/5TFxNW/81zexzWD9jn1+eSrOP84NdsS38H5IkF/UH3frqRPT+MvuCoVHjTDTnDw==", + "version": "10.24.2", + "resolved": "https://registry.npmjs.org/preact/-/preact-10.24.2.tgz", + "integrity": "sha512-1cSoF0aCC8uaARATfrlz4VCBqE8LwZwRfLgkxJOQwAlQt6ayTmi0D9OF7nXid1POI5SZidFuG9CnlXbDfLqY/Q==", "license": "MIT", "funding": { "type": "opencollective", @@ -7501,12 +7593,12 @@ "license": "MIT" }, "node_modules/react-remove-scroll": { - "version": "2.5.7", - "resolved": "https://registry.npmjs.org/react-remove-scroll/-/react-remove-scroll-2.5.7.tgz", - "integrity": "sha512-FnrTWO4L7/Bhhf3CYBNArEG/yROV0tKmTv7/3h9QCFvH6sndeFf1wPqOcbFVu5VAulS5dV1wGT3GZZ/1GawqiA==", + "version": "2.6.0", + "resolved": "https://registry.npmjs.org/react-remove-scroll/-/react-remove-scroll-2.6.0.tgz", + "integrity": "sha512-I2U4JVEsQenxDAKaVa3VZ/JeJZe0/2DxPWL8Tj8yLKctQJQiZM52pn/GWFpSp8dftjM3pSAHVJZscAnC/y+ySQ==", "license": "MIT", "dependencies": { - "react-remove-scroll-bar": "^2.3.4", + "react-remove-scroll-bar": "^2.3.6", "react-style-singleton": "^2.2.1", "tslib": "^2.1.0", "use-callback-ref": "^1.3.0", @@ -7737,16 +7829,16 @@ "license": "MIT" }, "node_modules/regexp.prototype.flags": { - "version": "1.5.2", - "resolved": "https://registry.npmjs.org/regexp.prototype.flags/-/regexp.prototype.flags-1.5.2.tgz", - "integrity": "sha512-NcDiDkTLuPR+++OCKB0nWafEmhg/Da8aUPLPMQbK+bxKKCm1/S5he+AqYa4PlMCVBalb4/yxIRub6qkEx5yJbw==", + "version": "1.5.3", + "resolved": "https://registry.npmjs.org/regexp.prototype.flags/-/regexp.prototype.flags-1.5.3.tgz", + "integrity": "sha512-vqlC04+RQoFalODCbCumG2xIOvapzVMHwsyIGM/SIE8fRhFFsXeH8/QQ+s0T0kDAhKc4k30s73/0ydkHQz6HlQ==", "dev": true, "license": "MIT", "dependencies": { - "call-bind": "^1.0.6", + "call-bind": "^1.0.7", "define-properties": "^1.2.1", "es-errors": "^1.3.0", - "set-function-name": "^2.0.1" + "set-function-name": "^2.0.2" }, "engines": { "node": ">= 0.4" @@ -8624,9 +8716,9 @@ "license": "MIT" }, "node_modules/tailwind-merge": { - "version": "2.5.2", - "resolved": "https://registry.npmjs.org/tailwind-merge/-/tailwind-merge-2.5.2.tgz", - "integrity": "sha512-kjEBm+pvD+6eAwzJL2Bi+02/9LFLal1Gs61+QB7HvTfQQ0aXwC5LGT8PEt1gS0CWKktKe6ysPTAy3cBC5MeiIg==", + "version": "2.5.3", + "resolved": "https://registry.npmjs.org/tailwind-merge/-/tailwind-merge-2.5.3.tgz", + "integrity": "sha512-d9ZolCAIzom1nf/5p4LdD5zvjmgSxY0BGgdSvmXIoMYAiPdAW/dSpP7joCDYFY7r/HkEa2qmPtkgsu0xjQeQtw==", "license": "MIT", "funding": { "type": "github", diff --git a/ui/package.json b/ui/package.json index a8db15001f..75fb427f08 100644 --- a/ui/package.json +++ b/ui/package.json @@ -11,33 +11,33 @@ }, "dependencies": { "@monaco-editor/react": "^4.6.0", - "@radix-ui/react-checkbox": "^1.1.1", - "@radix-ui/react-collapsible": "^1.1.0", - "@radix-ui/react-dialog": "^1.1.1", - "@radix-ui/react-dropdown-menu": "^2.1.1", + "@radix-ui/react-checkbox": "^1.1.2", + "@radix-ui/react-collapsible": "^1.1.1", + "@radix-ui/react-dialog": "^1.1.2", + "@radix-ui/react-dropdown-menu": "^2.1.2", "@radix-ui/react-form": "^0.1.0", "@radix-ui/react-icons": "^1.3.0", - "@radix-ui/react-popover": "^1.1.1", + "@radix-ui/react-popover": "^1.1.2", "@radix-ui/react-progress": "^1.1.0", - "@radix-ui/react-radio-group": "^1.2.0", - "@radix-ui/react-switch": "^1.1.0", - "@radix-ui/react-tabs": "^1.1.0", - "@radix-ui/react-toast": "^1.2.1", + "@radix-ui/react-radio-group": "^1.2.1", + "@radix-ui/react-switch": "^1.1.1", + "@radix-ui/react-tabs": "^1.1.1", + "@radix-ui/react-toast": "^1.2.2", "@radix-ui/react-toggle": "^1.0.3", "@radix-ui/react-toggle-group": "^1.1.0", - "@radix-ui/react-tooltip": "^1.1.2", - "@tremor/react": "^3.18.2", - "@types/node": "^22.6.1", - "@types/react": "^18.3.9", + "@radix-ui/react-tooltip": "^1.1.3", + "@tremor/react": "^3.18.3", + "@types/node": "^22.7.4", + "@types/react": "^18.3.11", "@types/react-dom": "^18.3.0", "classnames": "^2.5.1", "long": "^5.2.3", - "lucide-react": "^0.446.0", - "material-symbols": "^0.23.0", + "lucide-react": "^0.447.0", + "material-symbols": "^0.24.0", "moment": "^2.30.1", - "moment-timezone": "^0.5.45", - "next": "^14.2.13", - "next-auth": "^4.24.7", + "moment-timezone": "^0.5.46", + "next": "^14.2.14", + "next-auth": "^4.24.8", "prop-types": "^15.8.1", "protobufjs": "^7.4.0", "react": "^18.3.1", @@ -54,7 +54,7 @@ "autoprefixer": "^10.4.20", "copy-webpack-plugin": "^12.0.2", "eslint": "^8.57.1", - "eslint-config-next": "^14.2.13", + "eslint-config-next": "^14.2.14", "eslint-config-prettier": "^9.1.0", "gh-pages": "^6.1.1", "less": "^4.2.0", @@ -65,6 +65,6 @@ "tailwindcss": "^3.4.13", "tailwindcss-animate": "^1.0.7", "typescript": "^5.6.2", - "webpack": "^5.94.0" + "webpack": "^5.95.0" } } From fab4786ba93944b3ab56d6f5581f532a92c56452 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Tue, 8 Oct 2024 01:19:13 +0000 Subject: [PATCH 13/13] nexus: update secrecy (#2127) --- nexus/Cargo.lock | 4 ++-- nexus/flow-rs/src/grpc.rs | 2 +- nexus/peer-mysql/src/ast.rs | 5 +++-- nexus/peer-snowflake/Cargo.toml | 2 +- nexus/peer-snowflake/src/auth.rs | 13 +++++-------- nexus/peer-snowflake/src/lib.rs | 4 ++-- nexus/peer-snowflake/src/stream.rs | 2 +- 7 files changed, 15 insertions(+), 17 deletions(-) diff --git a/nexus/Cargo.lock b/nexus/Cargo.lock index 0061c58251..5788d0dc04 100644 --- a/nexus/Cargo.lock +++ b/nexus/Cargo.lock @@ -3596,9 +3596,9 @@ checksum = "1c107b6f4780854c8b126e228ea8869f4d7b71260f962fefb57b996b8959ba6b" [[package]] name = "secrecy" -version = "0.8.0" +version = "0.10.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9bd1c54ea06cfd2f6b63219704de0b9b4f72dcc2b8fdef820be6cd799780e91e" +checksum = "ba079fa568d52545cd70b334b2ce6f88f62b8fc2bda9290f48a0578388a49331" dependencies = [ "zeroize", ] diff --git a/nexus/flow-rs/src/grpc.rs b/nexus/flow-rs/src/grpc.rs index 958b31d515..8548b91152 100644 --- a/nexus/flow-rs/src/grpc.rs +++ b/nexus/flow-rs/src/grpc.rs @@ -306,7 +306,7 @@ impl FlowGrpcClient { pub async fn resync_mirror(&mut self, flow_job_name: &str) -> anyhow::Result<()> { let resync_mirror_req = pt::peerdb_route::ResyncMirrorRequest { flow_job_name: flow_job_name.to_owned(), - drop_stats: true + drop_stats: true, }; let response = self.client.resync_mirror(resync_mirror_req).await?; let resync_mirror_response = response.into_inner(); diff --git a/nexus/peer-mysql/src/ast.rs b/nexus/peer-mysql/src/ast.rs index 90fb837d38..b0b6c01630 100644 --- a/nexus/peer-mysql/src/ast.rs +++ b/nexus/peer-mysql/src/ast.rs @@ -108,13 +108,14 @@ pub fn rewrite_query(peername: &str, query: &mut Query) { } } Expr::Cast { - data_type: DataType::Time(_, ref mut tzinfo), .. + data_type: DataType::Time(_, ref mut tzinfo), + .. } => { *tzinfo = TimezoneInfo::None; } Expr::Cast { ref mut data_type, .. - } if matches!(data_type, DataType::Timestamp(..)) =>{ + } if matches!(data_type, DataType::Timestamp(..)) => { *data_type = DataType::Datetime(None); } _ => {} diff --git a/nexus/peer-snowflake/Cargo.toml b/nexus/peer-snowflake/Cargo.toml index 05dc416c5e..e74c543181 100644 --- a/nexus/peer-snowflake/Cargo.toml +++ b/nexus/peer-snowflake/Cargo.toml @@ -20,7 +20,7 @@ pgwire.workspace = true pt = { path = "../pt" } reqwest = { version = "0.12", default-features = false, features = ["json", "gzip", "rustls-tls"] } rsa = { version = "0.9.2", features = ["pem", "pkcs5"] } -secrecy = { version = "0.8.0" } +secrecy = "0.10" serde = { version = "1.0", features = ["derive"] } serde_json = "1.0" sha2 = "0.10" diff --git a/nexus/peer-snowflake/src/auth.rs b/nexus/peer-snowflake/src/auth.rs index 4ec1a90823..321d00da0b 100644 --- a/nexus/peer-snowflake/src/auth.rs +++ b/nexus/peer-snowflake/src/auth.rs @@ -1,7 +1,4 @@ -use std::{ - str::FromStr, - time::{SystemTime, UNIX_EPOCH}, -}; +use std::time::{SystemTime, UNIX_EPOCH}; use anyhow::Context; use base64::prelude::{Engine as _, BASE64_STANDARD}; @@ -9,7 +6,7 @@ use jsonwebtoken::{encode as jwt_encode, Algorithm, EncodingKey, Header}; use rsa::pkcs1::EncodeRsaPrivateKey; use rsa::pkcs8::{DecodePrivateKey, EncodePublicKey}; use rsa::RsaPrivateKey; -use secrecy::{Secret, SecretString}; +use secrecy::SecretString; use serde::Serialize; use sha2::{Digest, Sha256}; use tracing::info; @@ -32,7 +29,7 @@ pub struct SnowflakeAuth { refresh_threshold: u64, expiry_threshold: u64, last_refreshed: u64, - current_jwt: Option>, + current_jwt: Option, } impl SnowflakeAuth { @@ -128,14 +125,14 @@ impl SnowflakeAuth { let header: Header = Header::new(Algorithm::RS256); let encoded_jwt = jwt_encode(&header, &jwt_claims, &private_key_jwt)?; - let secret = SecretString::from_str(&encoded_jwt)?; + let secret = SecretString::from(encoded_jwt); self.current_jwt = Some(secret); Ok(()) } - pub fn get_jwt(&mut self) -> anyhow::Result<&Secret> { + pub fn get_jwt(&mut self) -> anyhow::Result<&SecretString> { if SystemTime::now().duration_since(UNIX_EPOCH)?.as_secs() >= (self.last_refreshed + self.refresh_threshold) { diff --git a/nexus/peer-snowflake/src/lib.rs b/nexus/peer-snowflake/src/lib.rs index d04a369d32..e0af65b58c 100644 --- a/nexus/peer-snowflake/src/lib.rs +++ b/nexus/peer-snowflake/src/lib.rs @@ -153,7 +153,7 @@ impl SnowflakeQueryExecutor { async fn process_query(&self, query_str: &str) -> anyhow::Result { let mut auth = self.auth.clone(); let jwt = auth.get_jwt()?; - let secret = jwt.expose_secret().clone(); + let secret = jwt.expose_secret(); // TODO: for things other than SELECTs, the robust way to handle retrys is by // generating a UUID from our end to mark the query as unique and then sending it with the request. // If we need to retry, send same UUID with retry=true parameter set and Snowflake should prevent duplicate execution. @@ -216,7 +216,7 @@ impl SnowflakeQueryExecutor { ) -> anyhow::Result { let mut auth = self.auth.clone(); let jwt = auth.get_jwt()?; - let secret = jwt.expose_secret().clone(); + let secret = jwt.expose_secret(); let response = self .reqwest_client .get(format!( diff --git a/nexus/peer-snowflake/src/stream.rs b/nexus/peer-snowflake/src/stream.rs index efac7b7e1f..5cd6e117b0 100644 --- a/nexus/peer-snowflake/src/stream.rs +++ b/nexus/peer-snowflake/src/stream.rs @@ -209,7 +209,7 @@ impl SnowflakeRecordStream { self.partition_number += 1; self.partition_index = 0; let partition_number = self.partition_number; - let secret = self.auth.get_jwt()?.expose_secret().clone(); + let secret = self.auth.get_jwt()?.expose_secret(); let statement_handle = self.result_set.statementHandle.clone(); let url = self.endpoint_url.clone(); println!("Secret: {:#?}", secret);