diff --git a/.eslintrc b/.eslintrc index b49d243aa..cfce8d755 100644 --- a/.eslintrc +++ b/.eslintrc @@ -1,8 +1,15 @@ { - "extends": "scality", + "extends": [ + "scality", + "plugin:jest/recommended" + ], "parserOptions": { "ecmaVersion": 2020 }, + "plugins": [ + "jest", + "mocha" + ], "rules": { "object-curly-newline": "off", "import/newline-after-import": "off", @@ -33,13 +40,40 @@ "space-unary-ops": "off", "no-undef-init": "off", "newline-per-chained-call": "off", - "no-useless-escape": "off" + "no-useless-escape": "off", + "mocha/no-exclusive-tests": "error", + "jest/no-done-callback": "off", + "jest/expect-expect": [ + "warn", + { + "assertFunctionNames": [ + "expect", + "assert", + "assert.*", + "_assertCredentials" + ] + } + ], + "import/extensions": [ + "error", + "ignorePackages", + { + "js": "never", + "jsx": "never", + "ts": "never", + "tsx": "never" + } + ] + }, + "env": { + "jest/globals": true }, "settings": { "import/resolver": { - "node": { - "paths": ["/backbeat/node_modules", "node_modules"] - } + "node": { + "paths": ["/backbeat/node_modules", "node_modules"], + "extensions": [".js", ".jsx", ".ts", ".tsx"] + } } - } + } } diff --git a/.github/dockerfiles/mongodb/Dockerfile b/.github/dockerfiles/mongodb/Dockerfile index d7afc7f37..861715174 100644 --- a/.github/dockerfiles/mongodb/Dockerfile +++ b/.github/dockerfiles/mongodb/Dockerfile @@ -1,4 +1,4 @@ -FROM mongo:4.2.24 +FROM mongo:4.4.21 ENV USER=scality \ HOME_DIR=/home/scality \ diff --git a/.github/workflows/tests.yaml b/.github/workflows/tests.yaml index 43ce2ad40..f9929feaf 100644 --- a/.github/workflows/tests.yaml +++ b/.github/workflows/tests.yaml @@ -44,8 +44,7 @@ jobs: uses: docker/build-push-action@v4 with: push: true - context: . - file: .github/dockerfiles/syntheticbucketd/Dockerfile + context: tests/utils/syntheticbucketd tags: "ghcr.io/scality/backbeat/syntheticbucketd:${{ github.sha }}" cache-from: type=gha,scope=syntheticbucketd cache-to: type=gha,mode=max,scope=syntheticbucketd @@ -105,6 +104,8 @@ jobs: cache: yarn - name: Install node dependencies run: yarn install --ignore-engines --frozen-lockfile --network-concurrency 1 + - name: Compile TypeScript + run: yarn build - name: Install ginkgo run: go get github.com/onsi/ginkgo/ginkgo@${GINKGO_VERSION} - name: Lint markdown @@ -147,11 +148,10 @@ jobs: BACKBEAT_CONFIG_FILE: "tests/config.json" - name: run backbeat notification feature tests run: yarn run ft_test:notification - - name: run ballooning tests for lifecycle conductor - run: yarn mocha tests/performance/lifecycle/conductor-check-memory-balloon.js + run: .github/scripts/run_ft_tests.bash perf_test:lifecycle env: - # Constrain heap long-lived heap size to 150MB, so that pushing 200K messages + # Constrain heap long-lived heap size to 200MB, so that pushing 200K messages # will crash if they end up in memory all at the same time (circuit breaking # ineffective) while waiting to be committed to the kafka topic. - NODE_OPTIONS: '--max-old-space-size=150' + NODE_OPTIONS: '--max-old-space-size=200' \ No newline at end of file diff --git a/.gitignore b/.gitignore index b8ee1ab34..d442c6829 100644 --- a/.gitignore +++ b/.gitignore @@ -15,3 +15,6 @@ node_modules # Redis *.rdb + +# TypeScript compilation output +dist/ diff --git a/Dockerfile b/Dockerfile index 79bf75fba..7fd39107e 100644 --- a/Dockerfile +++ b/Dockerfile @@ -29,16 +29,21 @@ RUN wget https://github.com/jwilder/dockerize/releases/download/$DOCKERIZE_VERSI && rm dockerize-linux-amd64-$DOCKERIZE_VERSION.tar.gz COPY package.json yarn.lock /usr/src/app/ -RUN yarn install --ignore-engines --frozen-lockfile --production --network-concurrency 1 \ - && rm -rf /var/lib/apt/lists/* \ - && rm -rf ~/.node-gyp \ - && rm -rf /tmp/yarn-* + +RUN yarn install --ignore-engines --frozen-lockfile --network-concurrency 1 + +COPY . /usr/src/app + +RUN yarn build + +RUN rm -rf node_modules \ + && yarn install --production --frozen-lockfile --ignore-optional ################################################################################ FROM node:${NODE_VERSION} -RUN apt-get update && \ - apt-get install -y --no-install-recommends \ +RUN apt-get update \ + && apt-get install -y --no-install-recommends \ jq \ tini \ && rm -rf /var/lib/apt/lists/* @@ -46,7 +51,8 @@ RUN apt-get update && \ WORKDIR /usr/src/app # Keep the .git directory in order to properly report version -COPY . /usr/src/app +COPY ./package.json ./docker-entrypoint.sh ./ +COPY --from=builder /usr/src/app/dist ./ COPY --from=builder /usr/src/app/node_modules ./node_modules/ COPY --from=builder /usr/local/bin/dockerize /usr/local/bin/ diff --git a/extensions/lifecycle/tasks/LifecycleUpdateExpirationTask.js b/extensions/lifecycle/tasks/LifecycleUpdateExpirationTask.js index f255ffeb7..df6d0249f 100644 --- a/extensions/lifecycle/tasks/LifecycleUpdateExpirationTask.js +++ b/extensions/lifecycle/tasks/LifecycleUpdateExpirationTask.js @@ -4,7 +4,6 @@ const { errors } = require('arsenal'); const ObjectMD = require('arsenal').models.ObjectMD; const BackbeatTask = require('../../../lib/tasks/BackbeatTask'); const ActionQueueEntry = require('../../../lib/models/ActionQueueEntry'); -const locations = require('../../../conf/locationConfig.json') || {}; class LifecycleUpdateExpirationTask extends BackbeatTask { /** @@ -144,13 +143,20 @@ class LifecycleUpdateExpirationTask extends BackbeatTask { }); async.waterfall([ - next => this._getMetadata(entry, log, next), - (objMD, next) => { - const coldLocation = entry.getAttribute('target.location') || - // If location not specified, use the first (and only) location - // This is a temporary fix, until Sorbet is fixed to provide the information - Object.keys(locations).find(name => locations[name].isCold); - + next => { + const coldLocation = entry.getAttribute('target.location'); + if (!coldLocation) { + // this should never happen as sorbet always sets the location attribute + log.error('missing target location', { + entry: entry.getLogInfo(), + method: 'LifecycleUpdateExpirationTask.processActionEntry', + }); + return next(errors.MissingParameter.customizeDescription('missing target location')); + } + return next(null, coldLocation); + }, + (coldLocation, next) => this._getMetadata(entry, log, (err, objMD) => next(err, coldLocation, objMD)), + (coldLocation, objMD, next) => { const archive = objMD.getArchive(); // Confirm the object has indeed expired: it can happen that the diff --git a/extensions/replication/queueProcessor/QueueProcessor.js b/extensions/replication/queueProcessor/QueueProcessor.js index 04dcc2b7c..66178ef1c 100644 --- a/extensions/replication/queueProcessor/QueueProcessor.js +++ b/extensions/replication/queueProcessor/QueueProcessor.js @@ -459,10 +459,10 @@ class QueueProcessor extends EventEmitter { */ _setupRedis(redisConfig) { // redis pub/sub for pause/resume - const redis = new Redis(redisConfig); + this._redis = new Redis(redisConfig); // redis subscribe to site specific channel const channelName = `${this.repConfig.topic}-${this.site}`; - redis.subscribe(channelName, err => { + this._redis.subscribe(channelName, err => { if (err) { this.logger.fatal('queue processor failed to subscribe to ' + `crr redis channel for location ${this.site}`, @@ -470,7 +470,7 @@ class QueueProcessor extends EventEmitter { error: err }); process.exit(1); } - redis.on('message', (channel, message) => { + this._redis.on('message', (channel, message) => { const validActions = { pauseService: this._pauseService.bind(this), resumeService: this._resumeService.bind(this), diff --git a/extensions/replication/tasks/ReplicateObject.js b/extensions/replication/tasks/ReplicateObject.js index 677acf696..4a81e7c9b 100644 --- a/extensions/replication/tasks/ReplicateObject.js +++ b/extensions/replication/tasks/ReplicateObject.js @@ -8,7 +8,7 @@ const ObjectMDLocation = require('arsenal').models.ObjectMDLocation; const BackbeatClient = require('../../../lib/clients/BackbeatClient'); const BackbeatMetadataProxy = require('../../../lib/BackbeatMetadataProxy'); -const mapLimitWaitPendingIfError = require('../../../lib/util/mapLimitWaitPendingIfError'); +const mapLimitWaitPendingIfError = require('../../../lib/util/mapLimitWaitPendingIfError').default; const { attachReqUids, TIMEOUT_MS } = require('../../../lib/clients/utils'); const getExtMetrics = require('../utils/getExtMetrics'); const BackbeatTask = require('../../../lib/tasks/BackbeatTask'); diff --git a/lib/BackbeatConsumer.js b/lib/BackbeatConsumer.js index 4ec6d0d3f..caec50071 100644 --- a/lib/BackbeatConsumer.js +++ b/lib/BackbeatConsumer.js @@ -8,7 +8,7 @@ const Logger = require('werelogs').Logger; const { BreakerState, CircuitBreaker } = require('breakbeat').CircuitBreaker; const BackbeatProducer = require('./BackbeatProducer'); -const OffsetLedger = require('./OffsetLedger'); +const OffsetLedger = require('./OffsetLedger').default; const KafkaBacklogMetrics = require('./KafkaBacklogMetrics'); const { startCircuitBreakerMetricsExport, diff --git a/lib/OffsetLedger.js b/lib/OffsetLedger.ts similarity index 63% rename from lib/OffsetLedger.js rename to lib/OffsetLedger.ts index d2f39c08c..e21d2d91c 100644 --- a/lib/OffsetLedger.js +++ b/lib/OffsetLedger.ts @@ -1,4 +1,11 @@ -const assert = require('assert'); +import assert from 'assert'; + +type PartitionOffsets = { + processing: number[], + latestConsumed: number | null +}; + +type TopicOffsets = { [key: number]: PartitionOffsets} /** * @class OffsetLedger @@ -8,16 +15,20 @@ const assert = require('assert'); * when messages are processed asynchronously and processing may end * out-of-order. */ -class OffsetLedger { + + export default class OffsetLedger { + + #ledger: { [topic: string]: TopicOffsets }; + constructor() { - this._ledger = {}; + this.#ledger = {}; } - _getPartitionOffsets(topic, partition) { - let topicOffsets = this._ledger[topic]; + _getPartitionOffsets(topic: string, partition: number) { + let topicOffsets = this.#ledger[topic]; if (!topicOffsets) { topicOffsets = {}; - this._ledger[topic] = topicOffsets; + this.#ledger[topic] = topicOffsets; } let partitionOffsets = topicOffsets[partition]; if (!partitionOffsets) { @@ -30,7 +41,7 @@ class OffsetLedger { return partitionOffsets; } - _getPartitionCommittableOffset(partitionOffsets) { + _getPartitionCommittableOffset(partitionOffsets: PartitionOffsets): number | null { // we can commit up to the lowest offset still being processed // since it means all lower offsets have already been // processed. If nothing is being processed, the latest @@ -49,13 +60,8 @@ class OffsetLedger { /** * Function to be called as soon as a new message is received from * a Kafka topic and about to start being processed. - * - * @param {string} topic - topic name - * @param {number} partition - partition number - * @param {number} offset - offset of consumed message - * @return {undefined} */ - onOffsetConsumed(topic, partition, offset) { + onOffsetConsumed(topic: string, partition: number, offset: number): undefined { // make sure offset is a positive number not to jeopardize // processing sanity assert(Number.isInteger(offset) && offset >= 0); @@ -78,13 +84,10 @@ class OffsetLedger { /** * Function to be called when a message is completely processed. * - * @param {string} topic - topic name - * @param {number} partition - partition number - * @param {number} offset - offset of processed message - * @return {number} - highest committable offset for this - * topic/partition (as returned by getCommittableOffset()) + * @return highest committable offset for this topic/partition + * (as returned by getCommittableOffset()) */ - onOffsetProcessed(topic, partition, offset) { + onOffsetProcessed(topic: string, partition: number, offset: number): number | null { const partitionOffsets = this._getPartitionOffsets(topic, partition); partitionOffsets.processing = partitionOffsets.processing.filter(pOff => pOff !== offset); @@ -94,12 +97,9 @@ class OffsetLedger { /** * Get the highest committable offset for a topic/partition * - * @param {string} topic - topic name - * @param {number} partition - partition number - * @param {number} offset - offset of processed message - * @return {number} - highest committable offset for this topic/partition + * @return highest committable offset for this topic/partition */ - getCommittableOffset(topic, partition) { + getCommittableOffset(topic: string, partition: number): number | null { const partitionOffsets = this._getPartitionOffsets(topic, partition); return this._getPartitionCommittableOffset(partitionOffsets); } @@ -108,28 +108,25 @@ class OffsetLedger { * Get how many entries have been consumed but not yet fully * processed/committable * - * @param {string} [topic] - topic name - * @param {number} [partition] - partition number - * @return {number} - number of consumed but not committable - * entries for this topic/partition, or for this topic (if no - * partition given), or for all topics and partitions (if none of - * topic and partition is provided) + * @return number of consumed but not committable entries for this + * topic/partition, or for this topic (if no partition given), or for + * all topics and partitions (if none of topic and partition is provided) */ - getProcessingCount(topic, partition) { - if (topic && !this._ledger[topic]) { + getProcessingCount(topic?: string, partition?: number): number { + if (topic && !this.#ledger[topic]) { return 0; } if (topic && partition !== undefined && - !this._ledger[topic][partition]) { + !this.#ledger[topic][partition]) { return 0; } let count = 0; - const topics = topic ? [topic] : Object.keys(this._ledger); + const topics = topic ? [topic] : Object.keys(this.#ledger); topics.forEach(t => { const partitions = partition !== undefined ? - [partition] : Object.keys(this._ledger[t]); + [partition] : Object.keys(this.#ledger[t]); partitions.forEach(p => { - const partitionOffsets = this._ledger[t][p]; + const partitionOffsets = this.#ledger[t][p]; count += partitionOffsets.processing.length; }); }); @@ -139,12 +136,10 @@ class OffsetLedger { /** * Export the ledger in JSON format (useful for debugging) * - * @return {string} a JSON-serialized representation of the + * @return a JSON-serialized representation of the * current state of the ledger */ - toString() { - return JSON.stringify(this._ledger); + toString(): string { + return JSON.stringify(this.#ledger); } } - -module.exports = OffsetLedger; diff --git a/lib/api/BackbeatAPI.js b/lib/api/BackbeatAPI.js index e23d1ba69..71f63530a 100644 --- a/lib/api/BackbeatAPI.js +++ b/lib/api/BackbeatAPI.js @@ -110,31 +110,31 @@ class BackbeatAPI { ingestionSites: this._ingestionSites, internalStart: this._internalStart, }; - const metrics = new Metrics(metricsConfig, logger); + this._metrics = new Metrics(metricsConfig, logger); Object.assign(this, { - _queryStats: metrics._queryStats, - _getData: metrics._getData, - _hasGlobalKey: metrics._hasGlobalKey, - _getMaxUptime: metrics._getMaxUptime, - getBacklog: metrics.getBacklog, - getCompletions: metrics.getCompletions, + _queryStats: this._metrics._queryStats, + _getData: this._metrics._getData, + _hasGlobalKey: this._metrics._hasGlobalKey, + _getMaxUptime: this._metrics._getMaxUptime, + getBacklog: this._metrics.getBacklog, + getCompletions: this._metrics.getCompletions, _sendCompletionsResponseReplication: - metrics._sendCompletionsResponseReplication, + this._metrics._sendCompletionsResponseReplication, _sendCompletionsResponseIngestion: - metrics._sendCompletionsResponseIngestion, - getObjectThroughput: metrics.getObjectThroughput, - getObjectProgress: metrics.getObjectProgress, - getThroughput: metrics.getThroughput, + this._metrics._sendCompletionsResponseIngestion, + getObjectThroughput: this._metrics.getObjectThroughput, + getObjectProgress: this._metrics.getObjectProgress, + getThroughput: this._metrics.getThroughput, _sendThroughputResponseReplication: - metrics._sendThroughputResponseReplication, + this._metrics._sendThroughputResponseReplication, _sendThroughputResponseIngestion: - metrics._sendThroughputResponseIngestion, - getAllMetrics: metrics.getAllMetrics, - getPending: metrics.getPending, + this._metrics._sendThroughputResponseIngestion, + getAllMetrics: this._metrics.getAllMetrics, + getPending: this._metrics.getPending, _sendPendingResponseReplication: - metrics._sendPendingResponseReplication, + this._metrics._sendPendingResponseReplication, _sendPendingResponseIngestion: - metrics._sendPendingResponseIngestion, + this._metrics._sendPendingResponseIngestion, }); } diff --git a/lib/util/mapLimitWaitPendingIfError.js b/lib/util/mapLimitWaitPendingIfError.ts similarity index 83% rename from lib/util/mapLimitWaitPendingIfError.js rename to lib/util/mapLimitWaitPendingIfError.ts index 10bb43bfc..7de1c9d44 100644 --- a/lib/util/mapLimitWaitPendingIfError.js +++ b/lib/util/mapLimitWaitPendingIfError.ts @@ -28,14 +28,24 @@ * * @return {undefined} */ -function mapLimitWaitPendingIfError(coll, limit, iteratee, callback) { + + +type ItemCallback = (err: Error | null, res: Res) => void; +type ResultCallback = (err: Error | null, res: ResList) => void; + +export default function mapLimitWaitPendingIfError( + coll: Item[], + limit: number, + iteratee: (item: Item, cb: ItemCallback) => Result, + callback: ResultCallback +): void | undefined { if (coll.length === 0) { return callback(null, []); } - const results = []; + const results: Result[] = []; let nPendingRequests = 0; let nextIdx = 0; - let pendingError = null; + let pendingError: Error | null = null; const processNext = () => { const idx = nextIdx; nextIdx += 1; @@ -64,5 +74,3 @@ function mapLimitWaitPendingIfError(coll, limit, iteratee, callback) { } return undefined; } - -module.exports = mapLimitWaitPendingIfError; diff --git a/package.json b/package.json index 5ea716bc1..6a4c167da 100644 --- a/package.json +++ b/package.json @@ -17,21 +17,23 @@ "oplog_populator": "node extensions/oplogPopulator/OplogPopulatorTask.js", "mongo_queue_processor": "node extensions/mongoProcessor/mongoProcessorTask.js", "garbage_collector": "node extensions/gc/service.js", - "test": "mocha --recursive tests/unit", - "ft_test": "mocha --recursive $(find tests/functional -name '*.js') --timeout 30000", - "ft_test:notification": "mocha --recursive $(find tests/functional/notification -name '*.js') --timeout 30000", - "ft_test:replication": "mocha --recursive $(find tests/functional/replication -name '*.js') --timeout 30000", - "ft_test:lib": "mocha --recursive $(find tests/functional/lib -name '*.js') --timeout 30000", - "ft_test:lifecycle": "mocha --recursive $(find tests/functional/lifecycle -name '*.js') --timeout 30000", - "ft_test:ingestion": "mocha --recursive $(find tests/functional/ingestion -name '*.js') --timeout 30000", - "ft_test:api:routes": "mocha tests/functional/api/routes.js --timeout 30000", - "ft_test:api:retry": "mocha tests/functional/api/retry.js --timeout 30000", - "bh_test": "mocha --recursive tests/behavior", + "test": "jest /unit --forceExit", + "ft_test": "jest /functional --forceExit --testTimeout 30000", + "ft_test:notification": "jest /functional/notification --forceExit --testTimeout 30000", + "ft_test:replication": "jest /functional/replication --forceExit --testTimeout 30000", + "ft_test:lib": "jest /functional/lib --forceExit --testTimeout 30000", + "ft_test:lifecycle": "jest /functional/lifecycle --forceExit --testTimeout 30000", + "ft_test:ingestion": "jest /functional/ingestion --forceExit --testTimeout 30000", + "ft_test:api:routes": "jest /functional/api/routes.spec.js --forceExit --testTimeout 30000", + "ft_test:api:retry": "jest /functional/api/retry.spec.js --forceExit --testTimeout 30000", + "perf_test:lifecycle": "jest /performance/lifecycle/ --forceExit --testTimeout 30000", + "bh_test": "jest /behavior", "lint": "eslint $(git ls-files '*.js')", "lint_md": "mdlint $(git ls-files '[^bucket-scanner/]*.md')", "start": "node bin/backbeat.js", "notification_populator": "BACKBEAT_QUEUEPOPULATOR_EXTENSIONS=notification BACKBEAT_QUEUEPOPULATOR_LOGSOURCE=kafka node bin/queuePopulator.js", - "notification_processor": "node extensions/notification/queueProcessor/task.js" + "notification_processor": "node extensions/notification/queueProcessor/task.js", + "build": "yarn tsc" }, "repository": { "type": "git", @@ -52,6 +54,7 @@ "breakbeat": "scality/breakbeat#v1.0.1", "bucketclient": "scality/bucketclient#8.1.9", "commander": "^2.11.0", + "eslint-plugin-mocha": "^10.2.0", "fcntl": "github:scality/node-fcntl#0.2.1", "httpagent": "git+https://github.com/scality/httpagent#1.0.6", "ioredis": "^4.9.5", @@ -63,7 +66,6 @@ "node-rdkafka-prometheus": "^1.0.0", "node-schedule": "^1.2.0", "node-zookeeper-client": "^1.1.3", - "prom-client": "14.2.0", "semver": "^5.6.0", "typescript": "^4.9.5", "uuid": "^3.1.0", @@ -75,14 +77,34 @@ }, "devDependencies": { "@zenko/cloudserver": "git+https://github.com/scality/cloudserver#8.6.8", - "eslint": "^8.14.0", + "eslint": "^8.25.0", "eslint-config-airbnb-base": "^13.1.0", "eslint-config-scality": "scality/Guidelines#8.2.0", "eslint-plugin-import": "^2.14.0", + "eslint-plugin-jest": "^27.1.1", + "jest": "^29.1.2", "lolex": "^2.7.5", "mocha": "^3.3.0", "sinon": "^10.0.0", - "zookeeper-mock": "^1.2.0" + "ts-jest": "^29.0.3", + "ts-node": "^10.9.1", + "zookeeper-mock": "^1.2.0", + "@types/node": "^18.11.9", + "@tsconfig/node16": "^1.0.3" + }, + "jest": { + "maxWorkers": 1, + "testPathIgnorePatterns": [ + "/node_modules/" + ], + "preset": "ts-jest", + "globals": { + "test-jest": { + "diagnostics": { + "warnOnly": true + } + } + } }, "engines": { "node": ">=16" diff --git a/tests/behavior/queuePopulator.js b/tests/behavior/queuePopulator.spec.js similarity index 98% rename from tests/behavior/queuePopulator.js rename to tests/behavior/queuePopulator.spec.js index aa81d6f6d..0aa116fab 100644 --- a/tests/behavior/queuePopulator.js +++ b/tests/behavior/queuePopulator.spec.js @@ -74,10 +74,7 @@ describe('queuePopulator', () => { let s3; let zkClient; - beforeEach(function setup(done) { - this.timeout(30000); // may take some time to keep up with the - // log entries - + beforeEach(done => { s3 = new S3(s3config); async.waterfall([ next => { @@ -141,7 +138,7 @@ describe('queuePopulator', () => { assert.ifError(err); done(); }); - }); + }, 30000); afterEach(done => s3.listObjectVersions({ Bucket: testBucket }, (err, data) => { @@ -237,9 +234,7 @@ describe('queuePopulator', () => { done(); }); }); - it('processLogEntries with 100 objects to replicate in 20 batches', - function test100objects(done) { - this.timeout(10000); + it('processLogEntries with 100 objects to replicate in 20 batches', done => { async.waterfall([ next => { let nbDone = 0; @@ -274,7 +269,7 @@ describe('queuePopulator', () => { assert.ifError(err); done(); }); - }); + }, 10000); describe('lifecycle extension', () => { const { zookeeperPath } = testConfig.extensions.lifecycle; diff --git a/tests/functional/api/Metrics.js b/tests/functional/api/Metrics.spec.js similarity index 100% rename from tests/functional/api/Metrics.js rename to tests/functional/api/Metrics.spec.js diff --git a/tests/functional/api/retry.js b/tests/functional/api/retry.spec.js similarity index 98% rename from tests/functional/api/retry.js rename to tests/functional/api/retry.spec.js index ae628b94c..db7c5da74 100644 --- a/tests/functional/api/retry.js +++ b/tests/functional/api/retry.spec.js @@ -151,7 +151,7 @@ describe('CRR Retry routes', () => { }); describe('CRR Retry feature', () => { - before(done => { + beforeAll(done => { const S3Mock = require('../utils/S3Mock'); const VaultMock = require('../utils/VaultMock'); const s3Mock = new S3Mock(); @@ -372,8 +372,7 @@ describe('CRR Retry feature', () => { it('should get correct data at scale for GET route: ' + '/_/crr/failed when failures occur across hours', - function f(done) { - this.timeout(300000); + done => { const hours = Array.from(Array(24).keys()); async.eachLimit(hours, 10, (hour, callback) => { const delta = (60 * 60 * 1000) * hour; @@ -418,12 +417,11 @@ describe('CRR Retry feature', () => { return next(); }), done); }); - }); + }, 300000); it('should get correct data at scale for GET route: ' + '/_/crr/failed when failures occur in the same hour', - function f(done) { - this.timeout(30000); + done => { const statsClient = new StatsModel(redisClient); const epoch = Date.now(); let twelveHoursAgo = epoch - (60 * 60 * 1000) * 12; @@ -465,7 +463,7 @@ describe('CRR Retry feature', () => { return next(); }), done); }); - }); + }, 30000); it('should get correct data for GET route: ' + '/_/crr/failed/// when there is no key', @@ -574,9 +572,9 @@ describe('CRR Retry feature', () => { done(); }); }); - }).timeout(10000); + }, 10000); - it.skip('should get correct data for POST route: /_/crr/failed ' + + it('should get correct data for POST route: /_/crr/failed ' + 'when there are multiple matching keys', done => { const member = getMember(`test-bucket:test-key:${testVersionId}`); async.series([ @@ -646,8 +644,7 @@ describe('CRR Retry feature', () => { }); it('should get correct data at scale for POST route: /_/crr/failed', - function f(done) { - this.timeout(30000); + done => { const reqBody = []; async.timesLimit(10, 10, (i, next) => { const Versions = addRoleProp([{ @@ -699,5 +696,5 @@ describe('CRR Retry feature', () => { }); }); }); - }); + }, 30000); }); diff --git a/tests/functional/api/routes.js b/tests/functional/api/routes.spec.js similarity index 98% rename from tests/functional/api/routes.js rename to tests/functional/api/routes.spec.js index c8f21d143..39594cfe0 100644 --- a/tests/functional/api/routes.js +++ b/tests/functional/api/routes.spec.js @@ -89,14 +89,14 @@ describe('API routes', () => { res.on('end', () => { data = JSON.parse(rawData); if (done) { - // only set in before() processing + // only set in beforeAll() processing done(); } }); }); } - before(done => { + beforeAll(done => { async.series([ next => { testProducer = new Producer({ @@ -129,7 +129,7 @@ describe('API routes', () => { ], done); }); - after(() => { + afterAll(() => { clearInterval(healthcheckTimer); }); @@ -161,11 +161,11 @@ describe('API routes', () => { return done(); } timer = setInterval(_checkValidKeys, 1000); - }).timeout(20000); + }, 20000); }); - describe('metrics routes', function dF() { - this.timeout(10000); + describe('metrics routes', () => { + jest.setTimeout(10000); const OPS = 'test:bb:ops'; const BYTES = 'test:bb:bytes'; const OBJECT_BYTES = 'test:bb:object:bytes'; @@ -214,7 +214,7 @@ describe('API routes', () => { `test-bucket:test-key:${testVersionId}1:${site2}`, ]; - before(done => + beforeAll(done => async.parallel([ next => addMembers(redisClient, site1, members, next), next => redisClient.incrby(`${site1}:${OPS_PENDING}`, 2, next), @@ -251,7 +251,7 @@ describe('API routes', () => { }, ], done)); - after(done => { + afterAll(done => { const redis = new Redis(); redis.flushall(done); }); @@ -755,8 +755,8 @@ describe('API routes', () => { }); }); - it(`should return all metrics for route: /_/metrics/crr/${site1}` + - `/progress/${BUCKET_NAME}/${OBJECT_KEY}?versionId=${VERSION_ID}`, + it('should return all metrics for route:' + + ` /_/metrics/crr/${site1}/progress/${BUCKET_NAME}/${OBJECT_KEY}?versionId=${VERSION_ID}`, done => getRequest(`/_/metrics/crr/${site1}/progress/${BUCKET_NAME}/` + `${OBJECT_KEY}?versionId=${VERSION_ID}`, (err, res) => { @@ -768,8 +768,8 @@ describe('API routes', () => { done(); })); - it(`should return all metrics for route: /_/metrics/crr/${site1}` + - `/throughput/${BUCKET_NAME}/${OBJECT_KEY}?versionId=${VERSION_ID}`, + it('should return all metrics for route:' + + `/_/metrics/crr/${site1}/throughput/${BUCKET_NAME}/${OBJECT_KEY}?versionId=${VERSION_ID}`, done => getRequest(`/_/metrics/crr/${site1}/throughput/${BUCKET_NAME}/` + `${OBJECT_KEY}?versionId=${VERSION_ID}`, (err, res) => { @@ -780,7 +780,7 @@ describe('API routes', () => { })); describe('No metrics data in Redis', () => { - before(done => { + beforeAll(done => { redis.keys('*:test:bb:*').then(keys => { const pipeline = redis.pipeline(); keys.forEach(key => { @@ -941,7 +941,7 @@ describe('API routes', () => { }); } - before(done => { + beforeAll(done => { redis1 = new Redis(); redis2 = new Redis(); @@ -964,7 +964,7 @@ describe('API routes', () => { cache2 = []; }); - after(() => { + afterAll(() => { if (zkClient) { zkClient.close(); zkClient = null; diff --git a/tests/functional/ingestion/IngestionProducer.js b/tests/functional/ingestion/IngestionProducer.spec.js similarity index 96% rename from tests/functional/ingestion/IngestionProducer.js rename to tests/functional/ingestion/IngestionProducer.spec.js index 4d3cdcdba..3564d7b6e 100644 --- a/tests/functional/ingestion/IngestionProducer.js +++ b/tests/functional/ingestion/IngestionProducer.spec.js @@ -21,7 +21,7 @@ describe('ingestion producer tests with mock', () => { let metadataMock; const bucket = sourceConfig.bucket; - before(done => { + beforeAll(done => { metadataMock = new MetadataMock(); httpServer = http.createServer( (req, res) => metadataMock.onRequest(req, res)).listen(7998); @@ -42,14 +42,13 @@ describe('ingestion producer tests with mock', () => { setupS3Mock(sourceConfig, done); }); - after(done => { + afterAll(done => { httpServer.close(); emptyAndDeleteVersionedBucket(sourceConfig, done); }); - // skipping because functionality currently not needed - it.skip('should be able to grab list of buckets for each raft session', + it('should be able to grab list of buckets for each raft session', done => { this.iProducer._getBuckets('1', (err, res) => { assert.ifError(err); diff --git a/tests/functional/ingestion/IngestionReader.js b/tests/functional/ingestion/IngestionReader.spec.js similarity index 99% rename from tests/functional/ingestion/IngestionReader.js rename to tests/functional/ingestion/IngestionReader.spec.js index 1366e3137..e87050610 100644 --- a/tests/functional/ingestion/IngestionReader.js +++ b/tests/functional/ingestion/IngestionReader.spec.js @@ -102,11 +102,11 @@ function checkEntryInQueue(kafkaEntries, expectedEntries, done) { } describe('ingestion reader tests with mock', function fD() { - this.timeout(40000); + jest.setTimeout(40000); let httpServer; let producer; - before(done => { + beforeAll(done => { testConfig.s3.port = testPort; const mongoUrl = `mongodb://${testConfig.queuePopulator.mongo.replicaSetHosts}` + @@ -195,7 +195,7 @@ describe('ingestion reader tests with mock', function fD() { ], done); }); - after(done => { + afterAll(done => { async.waterfall([ next => { httpServer.close(); @@ -321,8 +321,7 @@ describe('ingestion reader tests with mock', function fD() { done(); }); - // TODO: ZENKO-3420 - it.skip('should successfully ingest new bucket with existing object', + it('should successfully ingest new bucket with existing object', done => { // update zookeeper status to indicate snapshot phase const path = diff --git a/tests/functional/ingestion/MongoQueueProcessor.js b/tests/functional/ingestion/MongoQueueProcessor.spec.js similarity index 99% rename from tests/functional/ingestion/MongoQueueProcessor.js rename to tests/functional/ingestion/MongoQueueProcessor.spec.js index e37abea75..2b543b4f6 100644 --- a/tests/functional/ingestion/MongoQueueProcessor.js +++ b/tests/functional/ingestion/MongoQueueProcessor.spec.js @@ -188,13 +188,13 @@ class MongoQueueProcessorMock extends MongoQueueProcessor { } } -describe('MongoQueueProcessor', function mqp() { - this.timeout(5000); +describe('MongoQueueProcessor', () => { + jest.setTimeout(5000); let mqp; let mongoClient; - before(() => { + beforeAll(() => { mqp = new MongoQueueProcessorMock(kafkaConfig, mongoProcessorConfig, mongoClientConfig, mConfig); mqp.start(); diff --git a/tests/functional/ingestion/pauseResumeState.js b/tests/functional/ingestion/pauseResumeState.spec.js similarity index 99% rename from tests/functional/ingestion/pauseResumeState.js rename to tests/functional/ingestion/pauseResumeState.spec.js index f795a861c..c99cca887 100644 --- a/tests/functional/ingestion/pauseResumeState.js +++ b/tests/functional/ingestion/pauseResumeState.spec.js @@ -203,11 +203,11 @@ function deleteScheduledResumeLocation(iPopulator, location, done) { */ describe('Ingestion Pause/Resume', function d() { - this.timeout(10000); + jest.setTimeout(10000); this.mockAPI = new MockAPI(ingestionConfig); - before(done => { + beforeAll(done => { this.zkHelper = new ZKStateHelper(zkConfig, ZK_TEST_STATE_PATH, firstBucket.locationConstraint, secondBucket.locationConstraint, futureDate); @@ -259,7 +259,7 @@ describe('Ingestion Pause/Resume', function d() { } }); - after(() => { + afterAll(() => { this.zkHelper.close(); }); diff --git a/tests/functional/lib/BackbeatClient.js b/tests/functional/lib/BackbeatClient.spec.js similarity index 91% rename from tests/functional/lib/BackbeatClient.js rename to tests/functional/lib/BackbeatClient.spec.js index 39c9249a8..9b99300a7 100644 --- a/tests/functional/lib/BackbeatClient.js +++ b/tests/functional/lib/BackbeatClient.spec.js @@ -30,7 +30,7 @@ const serverMock = new MetadataMock(); describe('BackbeatClient unit tests with mock server', () => { let httpServer; - before(done => { + beforeAll(done => { expectedLogs.log.forEach((log, i) => { log.entries.forEach((entry, j) => { expectedLogs.log[i].entries[j].value.attributes = @@ -48,10 +48,9 @@ describe('BackbeatClient unit tests with mock server', () => { .listen(backbeatClientTestPort, done); }); - after(() => httpServer.close()); + afterAll(() => httpServer.close()); - // skipping this test because ingestion does not need list bucket per raft - it.skip('should get list of buckets managed by raft session', done => { + it('should get list of buckets managed by raft session', done => { const destReq = backbeatClient.getRaftBuckets({ LogId: '1', }); @@ -69,7 +68,7 @@ describe('BackbeatClient unit tests with mock server', () => { const destReq = backbeatClient.getRaftId({ Bucket: bucketName, }); - return destReq.send((err, data) => { + destReq.send((err, data) => { assert.ifError(err); assert.strictEqual(data[0], '1'); return done(); @@ -80,7 +79,7 @@ describe('BackbeatClient unit tests with mock server', () => { const destReq = backbeatClient.getRaftLog({ LogId: '1', }); - return destReq.send((err, data) => { + destReq.send((err, data) => { assert.ifError(err); assert.deepStrictEqual(data, expectedLogs); return done(); @@ -91,7 +90,7 @@ describe('BackbeatClient unit tests with mock server', () => { const destReq = backbeatClient.getBucketMetadata({ Bucket: bucketName, }); - return destReq.send((err, data) => { + destReq.send((err, data) => { assert.ifError(err); const bucketMd = dummyBucketMD[bucketName]; const expectedBucketMD = new BucketInfo(bucketMd.name, @@ -119,7 +118,7 @@ describe('BackbeatClient unit tests with mock server', () => { const destReq = backbeatClient.getObjectList({ Bucket: bucketName, }); - return destReq.send((err, data) => { + destReq.send((err, data) => { assert.ifError(err); assert.deepStrictEqual(data, expectedObjectList); return done(); @@ -131,7 +130,7 @@ describe('BackbeatClient unit tests with mock server', () => { Bucket: bucketName, Key: objectName, }); - return destReq.send((err, data) => { + destReq.send((err, data) => { assert.ifError(err); assert(data.Body); const dataValue = JSON.parse(data.Body); @@ -144,7 +143,7 @@ describe('BackbeatClient unit tests with mock server', () => { const destReq = backbeatClient.getBucketCseq({ Bucket: bucketName, }); - return destReq.send((err, data) => { + destReq.send((err, data) => { assert.ifError(err); assert(data[0] && data[0].cseq); assert.strictEqual(data[0].cseq, 7); diff --git a/tests/functional/lib/BackbeatConsumer.js b/tests/functional/lib/BackbeatConsumer.spec.js similarity index 92% rename from tests/functional/lib/BackbeatConsumer.js rename to tests/functional/lib/BackbeatConsumer.spec.js index b5b5703ef..cff6537df 100644 --- a/tests/functional/lib/BackbeatConsumer.js +++ b/tests/functional/lib/BackbeatConsumer.spec.js @@ -21,6 +21,8 @@ const consumerKafkaConf = { }, }; +jest.setTimeout(60000); + describe('BackbeatConsumer main tests', () => { const topic = 'backbeat-consumer-spec'; const groupId = `replication-group-${Math.random()}`; @@ -39,9 +41,7 @@ describe('BackbeatConsumer main tests', () => { process.nextTick(cb); } - before(function before(done) { - this.timeout(60000); - + beforeAll(done => { producer = new BackbeatProducer({ kafka: producerKafkaConf, topic, pollIntervalMs: 100 }); consumer = new BackbeatConsumer({ @@ -65,8 +65,7 @@ describe('BackbeatConsumer main tests', () => { consumedMessages = []; consumer.removeAllListeners('consumed'); }); - after(function after(done) { - this.timeout(10000); + afterAll(done => { async.parallel([ innerDone => producer.close(innerDone), innerDone => consumer.close(innerDone), @@ -75,7 +74,7 @@ describe('BackbeatConsumer main tests', () => { innerDone(); }, ], done); - }); + }, 10000); it('should be able to read messages sent to the topic and publish ' + 'topic metrics', done => { @@ -124,7 +123,7 @@ describe('BackbeatConsumer main tests', () => { setTimeout(() => { _checkZkMetrics(() => { consumeCb(); - consumer.unsubscribe(); + consumer.pause(); }); }, 5000); assert.deepStrictEqual( @@ -142,13 +141,13 @@ describe('BackbeatConsumer main tests', () => { // Check that rdkafka metrics are indeed exported assert(metrics.ZenkoMetrics.getMetric('rdkafka_cgrp_assignment_size') !== undefined); - }).timeout(30000); + }, 30000); it('should not consume messages when paused and when resumed, consume ' + 'messages from the previous offset', done => { let totalConsumed = 0; const kafkaConsumer = consumer._consumer; - consumer.subscribe(); + consumer.resume(); async.series([ next => { @@ -200,7 +199,7 @@ describe('BackbeatConsumer main tests', () => { assert.equal(totalConsumed, 4); done(); }); - }).timeout(30000); + }, 30000); }); describe('BackbeatConsumer rebalance tests', () => { @@ -249,9 +248,7 @@ describe('BackbeatConsumer rebalance tests', () => { }, consumedMessages++ ? 4000 : 2000); } - before(function before(done) { - this.timeout(60000); - + beforeAll(done => { // Bootstrap just once at the beginning of the test suite consumer = new BackbeatConsumer({ zookeeper: zookeeperConf, @@ -262,9 +259,7 @@ describe('BackbeatConsumer rebalance tests', () => { consumer.on('ready', () => consumer.close(done)); }); - beforeEach(function before(done) { - this.timeout(60000); - + beforeEach(done => { consumedMessages = 0; processedMessages = []; producer = new BackbeatProducer({ @@ -297,8 +292,7 @@ describe('BackbeatConsumer rebalance tests', () => { ], done); }); - afterEach(function after(done) { - this.timeout(10000); + afterEach(done => { if (timer) { clearInterval(timer); timer = null; @@ -308,7 +302,7 @@ describe('BackbeatConsumer rebalance tests', () => { innerDone => consumer.close(innerDone), innerDone => (consumer2 ? consumer2.close(innerDone) : innerDone()), ], done); - }); + }, 10000); it('should handle rebalance when no task in progress', done => { consumer.on('processed.all', () => { @@ -328,7 +322,7 @@ describe('BackbeatConsumer rebalance tests', () => { producer.send(messages, err => { assert.ifError(err); }); - }).timeout(40000); + }, 40000); it('should commit current tasks during rebalance', done => { consumer.on('processed.all', () => { @@ -350,7 +344,7 @@ describe('BackbeatConsumer rebalance tests', () => { producer.send(messages, err => { assert.ifError(err); }); - }).timeout(40000); + }, 40000); it('should fail healthcheck on rebalance timeout', done => { assert(consumer.isReady()); @@ -377,7 +371,7 @@ describe('BackbeatConsumer rebalance tests', () => { done(); } }, 1000); - }).timeout(60000); + }, 60000); }); describe('BackbeatConsumer concurrency tests', () => { @@ -396,9 +390,7 @@ describe('BackbeatConsumer concurrency tests', () => { taskStuckCallbacks.push(cb); } } - before(function before(done) { - this.timeout(60000); - + beforeAll(done => { producer = new BackbeatProducer({ kafka: producerKafkaConf, topic: topicConc, @@ -415,7 +407,7 @@ describe('BackbeatConsumer concurrency tests', () => { innerDone => producer.on('ready', innerDone), innerDone => consumer.on('ready', innerDone), ], done); - }); + }, 60000); afterEach(() => { consumedMessages = []; consumer.removeAllListeners('consumed'); @@ -423,7 +415,7 @@ describe('BackbeatConsumer concurrency tests', () => { taskStuckCallbacks.map(cb => cb()); taskStuckCallbacks = []; }); - after(done => { + afterAll(done => { async.parallel([ innerDone => producer.close(innerDone), innerDone => consumer.close(innerDone), @@ -533,9 +525,7 @@ describe('BackbeatConsumer "deferred committable" tests', () => { process.nextTick(cb); } } - before(function before(done) { - this.timeout(60000); - + beforeAll(done => { producer = new BackbeatProducer({ kafka: producerKafkaConf, topic: topicConc, @@ -552,12 +542,12 @@ describe('BackbeatConsumer "deferred committable" tests', () => { innerDone => producer.on('ready', innerDone), innerDone => consumer.on('ready', innerDone), ], done); - }); + }, 60000); afterEach(() => { consumedMessages = []; consumer.removeAllListeners('consumed'); }); - after(done => { + afterAll(done => { async.parallel([ innerDone => producer.close(innerDone), innerDone => consumer.close(innerDone), @@ -611,9 +601,7 @@ describe('BackbeatConsumer with circuit breaker', () => { process.nextTick(cb); } - beforeEach(function before(done) { - this.timeout(60000); - + beforeEach(done => { groupIdBreaker = `replication-group-breaker-${Math.random()}`; producer = new BackbeatProducer({ @@ -621,18 +609,7 @@ describe('BackbeatConsumer with circuit breaker', () => { topic: topicBreaker, pollIntervalMs: 100, }); - consumer = new BackbeatConsumer({ - zookeeper: zookeeperConf, - kafka: consumerKafkaConf, groupId: groupIdBreaker, topic: topicBreaker, - queueProcessor, - concurrency: 10, - bootstrap: true, - circuitBreaker: this.currentTest.breakerConf, - }); - async.parallel([ - innerDone => producer.on('ready', innerDone), - innerDone => consumer.on('ready', innerDone), - ], done); + producer.on('ready', done); }); afterEach(done => { @@ -652,9 +629,9 @@ describe('BackbeatConsumer with circuit breaker', () => { const nMessages = 50; - const testCases = [ + test.each([ { - description: 'should consume if breaker state nominal', + description: 'consume if breaker state nominal', expectedMessages: nMessages, breakerConf: { probes: [ @@ -666,7 +643,7 @@ describe('BackbeatConsumer with circuit breaker', () => { }, }, { - description: 'should not consume if breaker state not nominal', + description: 'not consume if breaker state not nominal', expectedMessages: 0, breakerConf: { nominalEvaluateIntervalMs: 1, @@ -678,10 +655,8 @@ describe('BackbeatConsumer with circuit breaker', () => { ], }, }, - ]; - - testCases.forEach(t => { - const test = it(t.description, done => { + ])('should $description', + ({ expectedMessages, breakerConf }, done) => { const boatloadOfMessages = []; for (let i = 0; i < nMessages; ++i) { boatloadOfMessages.push({ @@ -693,6 +668,17 @@ describe('BackbeatConsumer with circuit breaker', () => { let totalConsumed = 0; async.series([ + next => { + consumer = new BackbeatConsumer({ + zookeeper: zookeeperConf, + kafka: consumerKafkaConf, groupId: groupIdBreaker, topic: topicBreaker, + queueProcessor, + concurrency: 10, + bootstrap: true, + circuitBreaker: breakerConf, + }); + consumer.on('ready', next); + }, next => { setTimeout(() => producer.send(boatloadOfMessages, err => { assert.ifError(err); @@ -704,15 +690,12 @@ describe('BackbeatConsumer with circuit breaker', () => { }); }, next => { - assert.strictEqual(totalConsumed, t.expectedMessages); + assert.strictEqual(totalConsumed, expectedMessages); next(); }, ], done); - }); - - // Attach breakerConf to the test, so it can be used from the hooks - test.breakerConf = t.breakerConf; - }); + } + ); }); describe('BackbeatConsumer shutdown tests', () => { @@ -732,8 +715,7 @@ describe('BackbeatConsumer shutdown tests', () => { } } - before(function before(done) { - this.timeout(60000); + beforeAll(done => { producer = new BackbeatProducer({ topic, kafka: producerKafkaConf, @@ -750,8 +732,7 @@ describe('BackbeatConsumer shutdown tests', () => { ], done); }); - beforeEach(function beforeEach(done) { - this.timeout(60000); + beforeEach(done => { consumer = new BackbeatConsumer({ zookeeper: zookeeperConf, kafka: { @@ -774,8 +755,7 @@ describe('BackbeatConsumer shutdown tests', () => { consumer.removeAllListeners('consumed'); }); - after(function after(done) { - this.timeout(10000); + afterAll(done => { async.parallel([ innerDone => producer.close(innerDone), innerDone => { @@ -783,7 +763,7 @@ describe('BackbeatConsumer shutdown tests', () => { innerDone(); }, ], done); - }); + }, 10000); it('should stop consuming and wait for current jobs to end before shutting down', done => { setTimeout(() => { @@ -815,9 +795,9 @@ describe('BackbeatConsumer shutdown tests', () => { }); }, ], done); - }).timeout(30000); + }, 30000); - it('should immediatly shuttdown when no in progress tasks', done => { + it('should immediately shutdown when no in progress tasks', done => { setTimeout(() => { producer.send([messages[0]], assert.ifError); }, 3000); @@ -840,9 +820,9 @@ describe('BackbeatConsumer shutdown tests', () => { }); }, ], done); - }).timeout(30000); + }, 30000); - it('should shuttdown when consumer has been disconnected', done => { + it('should shutdown when consumer has been disconnected', done => { async.series([ next => { consumer._consumer.disconnect(); @@ -850,7 +830,7 @@ describe('BackbeatConsumer shutdown tests', () => { }, next => consumer.close(next), ], done); - }).timeout(30000); + }, 30000); it('should close even when a job is stuck', done => { setTimeout(() => { @@ -874,5 +854,5 @@ describe('BackbeatConsumer shutdown tests', () => { }); }, ], done); - }).timeout(60000); + }); }); diff --git a/tests/functional/lib/BackbeatProducer.js b/tests/functional/lib/BackbeatProducer.spec.js similarity index 86% rename from tests/functional/lib/BackbeatProducer.js rename to tests/functional/lib/BackbeatProducer.spec.js index e1225fb56..e0b7b4597 100644 --- a/tests/functional/lib/BackbeatProducer.js +++ b/tests/functional/lib/BackbeatProducer.spec.js @@ -21,15 +21,13 @@ const oneBigMessage = [{ key: 'large-foo', config: { kafka: kafkaConf, topic, pollIntervalMs: 100 }, }, ].forEach(item => { - describe(`BackbeatProducer - ${item.type}`, function backbeatProducer() { - this.timeout(10000); - + describe(`BackbeatProducer - ${item.type}`, () => { let producer; - before(done => { + beforeAll(done => { producer = new BackbeatProducer(item.config); producer.on('ready', () => done()); }); - after(() => { producer = null; }); + afterAll(() => { producer = null; }); it('should be able to send one message and get delivery reports back', done => { @@ -49,7 +47,7 @@ const oneBigMessage = [{ key: 'large-foo', latestPublishedMetricValues[0].value >= beforeSend / 1000); done(); }); - }).timeout(30000); + }, 30000); it('should be able to send a batch of messages and get delivery ' + 'reports back', done => { @@ -59,7 +57,7 @@ const oneBigMessage = [{ key: 'large-foo', assert.strictEqual(reports.length, 3); done(); }); - }).timeout(30000); + }, 30000); it('should be able to send a big ' + `${oneBigMessage[0].message.length / 1000000}MB message`, done => { @@ -67,28 +65,26 @@ const oneBigMessage = [{ key: 'large-foo', assert.ifError(err); done(); }); - }).timeout(30000); - }); + }, 30000); + }, 10000); }); -describe('BackbeatProducer - Error case', function backbeatProducerErrors() { - this.timeout(10000); - +describe('BackbeatProducer - Error case', () => { let producer; - before(done => { + beforeAll(done => { producer = new BackbeatProducer({ kafka: kafkaConf, topic }); producer .on('ready', () => producer.close(done)) .on('error', done); }); - after(() => { producer = null; }); + afterAll(() => { producer = null; }); it('should get an error if producer is not ready', done => { producer.send(oneMessage, err => { assert.deepStrictEqual(errors.InternalError, err); done(); }); - }).timeout(30000); -}); + }, 30000); +}, 10000); diff --git a/tests/functional/lib/KafkaBacklogMetrics.spec.js b/tests/functional/lib/KafkaBacklogMetrics.spec.js index 43422a106..c4d4b02e9 100644 --- a/tests/functional/lib/KafkaBacklogMetrics.spec.js +++ b/tests/functional/lib/KafkaBacklogMetrics.spec.js @@ -30,16 +30,16 @@ const CONSUMER_TIMEOUT = 60000; werelogs.configure({ level: 'info', dump: 'error' }); -describe('KafkaBacklogMetrics class', function kafkaBacklogMetrics() { +describe('KafkaBacklogMetrics class', () => { let zkPath; let zkClient; let kafkaBacklogMetrics; let consumer; let producer; - this.timeout(TIMEOUT); + jest.setTimeout(TIMEOUT); - before(done => { + beforeAll(done => { zkPath = kafkaConfig.backlogMetrics.zkPath; async.series([ next => { @@ -83,7 +83,7 @@ describe('KafkaBacklogMetrics class', function kafkaBacklogMetrics() { ], done); }); - after(done => { + afterAll(done => { async.waterfall([ next => zkClient.removeRecur(zkPath, next), next => producer.close(next), diff --git a/tests/functional/lifecycle/LifecycleBucketProcessor.spec.js b/tests/functional/lifecycle/LifecycleBucketProcessor.spec.js index 5e37a2171..928414657 100644 --- a/tests/functional/lifecycle/LifecycleBucketProcessor.spec.js +++ b/tests/functional/lifecycle/LifecycleBucketProcessor.spec.js @@ -42,8 +42,8 @@ const bucketEntryMessage = { werelogs.configure({ level: 'warn', dump: 'error' }); -describe('Lifecycle Bucket Processor', function lifecycleBucketProcessor() { - this.timeout(testTimeout); +describe('Lifecycle Bucket Processor', () => { + jest.setTimeout(testTimeout); function generateRetryTest(s3Client, shouldRetry = true) { const lbp = new LifecycleBucketProcessor( diff --git a/tests/functional/lifecycle/LifecycleConductor.spec.js b/tests/functional/lifecycle/LifecycleConductor.spec.js index 95289881c..a98ddbfd6 100644 --- a/tests/functional/lifecycle/LifecycleConductor.spec.js +++ b/tests/functional/lifecycle/LifecycleConductor.spec.js @@ -137,8 +137,8 @@ const CONSUMER_TIMEOUT = 60000; werelogs.configure({ level: 'info', dump: 'error' }); -describe('lifecycle conductor', function lifecycleConductor() { - this.timeout(TIMEOUT); +describe('lifecycle conductor', () => { + jest.setTimeout(TIMEOUT); describe('backlog control', () => { const bucketdPort = 14344; @@ -318,20 +318,13 @@ describe('lifecycle conductor', function lifecycleConductor() { function describeConductorSpec(opts) { const { - description, lifecycleConfig, transformExpectedMessages, mockBucketd, mockVault, setupZookeeper, - skip, } = opts; - if (skip) { - return describe.skip(`skipped: ${description} ${skip}`, () => { - }); - } - const bucketdPort = 14345; const vaultPort = 14346; const maxKeys = 2; @@ -400,6 +393,7 @@ describe('lifecycle conductor', function lifecycleConductor() { if (mockVault) { lifecycleConfig.auth.vault.port = vaultPort; + lifecycleConfig.conductor.vaultAdmin = lifecycleConfig.auth.vault; } lifecycleConfig.conductor.concurrency = maxKeys; @@ -428,162 +422,161 @@ describe('lifecycle conductor', function lifecycleConductor() { }; } - return describe(description, () => { - beforeEach(done => { - bucketdListing = []; + beforeEach(done => { + bucketdListing = []; - lcConductor = new LifecycleConductor(zkConfig.zookeeper, - kafkaConfig, validatedLifecycleConfig, repConfig, s3Config); + lcConductor = new LifecycleConductor(zkConfig.zookeeper, + kafkaConfig, validatedLifecycleConfig, repConfig, s3Config); - async.series([ - next => lcConductor.init(next), - next => { - consumer = new BackbeatTestConsumer({ - kafka: { hosts: kafkaConfig.hosts }, - topic: validatedLifecycleConfig.bucketTasksTopic, - groupId: 'test-consumer-group', + async.series([ + next => lcConductor.init(next), + next => { + consumer = new BackbeatTestConsumer({ + kafka: { hosts: kafkaConfig.hosts }, + topic: validatedLifecycleConfig.bucketTasksTopic, + groupId: 'test-consumer-group', + }); + consumer.on('ready', next); + }, + next => { + consumer.subscribe(); + // it seems the consumer needs some extra time to + // start consuming the first messages + setTimeout(next, 2000); + }, + next => { + if (mockBucketd) { + bucketd = http.createServer(bucketdHandler); + bucketd.listen(bucketdPort, next); + } else { + process.nextTick(next); + } + }, + next => { + if (mockVault) { + vault = http.createServer(vaultHandler); + vault.listen(vaultPort, next); + } else { + process.nextTick(next); + } + }, + next => { + if (setupZookeeper) { + zkClient = zookeeper.createClient( + zkConfig.zookeeper.connectionString, + zkConfig.zookeeper); + zkClient.connect(); + zkClient.once('ready', () => { + lcConductor.initZkPaths(next); }); - consumer.on('ready', next); - }, - next => { - consumer.subscribe(); - // it seems the consumer needs some extra time to - // start consuming the first messages - setTimeout(next, 2000); - }, - next => { - if (mockBucketd) { - bucketd = http.createServer(bucketdHandler); - bucketd.listen(bucketdPort, next); - } else { - process.nextTick(next); - } - }, - next => { - if (mockVault) { - vault = http.createServer(vaultHandler); - vault.listen(vaultPort, next); - } else { - process.nextTick(next); - } - }, - next => { - if (setupZookeeper) { - zkClient = zookeeper.createClient( - zkConfig.zookeeper.connectionString, - zkConfig.zookeeper); - zkClient.connect(); - zkClient.once('ready', () => { - lcConductor.initZkPaths(next); - }); - } else { - process.nextTick(next); - } - }, - ], done); - }); + } else { + process.nextTick(next); + } + }, + ], done); + }); - afterEach(done => { - async.series([ - next => { - if (mockBucketd) { - bucketd.close(next); - } else { - process.nextTick(next); - } - }, - next => { - if (mockVault) { - vault.close(next); - } else { - process.nextTick(next); - } - }, - next => { - if (setupZookeeper) { - zkClient.removeRecur(validatedLifecycleConfig.zookeeperPath, next); - } else { - process.nextTick(next); - } - }, - next => consumer.close(next), - next => lcConductor.stop(next), - ], done); - }); + afterEach(done => { + async.series([ + next => { + if (mockBucketd && bucketd) { + bucketd.close(next); + } else { + process.nextTick(next); + } + }, + next => { + if (mockVault) { + vault.close(next); + } else { + process.nextTick(next); + } + }, + next => { + if (setupZookeeper) { + zkClient.removeRecur(validatedLifecycleConfig.zookeeperPath, next); + } else { + process.nextTick(next); + } + }, + next => consumer.close(next), + next => lcConductor.stop(next), + ], done); + }); - it('should populate queue', done => { - async.waterfall([ - bucketPopulatorStep1, - next => { - lcConductor.processBuckets(); - consumer.expectUnorderedMessages(transformExpectedMessages(expected2Messages), - CONSUMER_TIMEOUT, next); - }, - bucketPopulatorStep2, - next => { - lcConductor.processBuckets(); - consumer.expectUnorderedMessages(transformExpectedMessages(expected4Messages), - CONSUMER_TIMEOUT, next); - }, - ], err => { - assert.ifError(err); - done(); - }); + it('should populate queue', done => { + async.waterfall([ + bucketPopulatorStep1, + next => { + lcConductor.processBuckets(); + consumer.expectUnorderedMessages(transformExpectedMessages(expected2Messages), + CONSUMER_TIMEOUT, next); + }, + bucketPopulatorStep2, + next => { + lcConductor.processBuckets(); + consumer.expectUnorderedMessages(transformExpectedMessages(expected4Messages), + CONSUMER_TIMEOUT, next); + }, + ], err => { + assert.ifError(err); + done(); }); }); } - describeConductorSpec({ - description: 'with auth `account` and buckets from bucketd', - lifecycleConfig: { - ...baseLCConfig, - conductor: { - ...baseLCConfig.conductor, - bucketSource: 'bucketd', - bucketd: { - host: '127.0.0.1', + describe.each([ + { + description: 'with auth `account` and buckets from bucketd', + lifecycleConfig: { + ...baseLCConfig, + conductor: { + ...baseLCConfig.conductor, + bucketSource: 'bucketd', + bucketd: { + host: '127.0.0.1', + }, }, }, + mockBucketd: true, + transformExpectedMessages: identity, }, - mockBucketd: true, - transformExpectedMessages: identity, - }); - - describeConductorSpec({ - description: 'with auth `account` and buckets from zookeeper (compat mode)', - lifecycleConfig: baseLCConfig, - setupZookeeper: true, - transformExpectedMessages: identity, - }); - - describeConductorSpec({ - description: 'with auth `assumeRole` and buckets from bucketd', - lifecycleConfig: { - ...baseLCConfig, - conductor: { - ...baseLCConfig.conductor, - bucketSource: 'bucketd', - bucketd: { - host: '127.0.0.1', - }, - }, - auth: { - type: 'assumeRole', - roleName: 'lc', - sts: { - host: '127.0.0.1', - port: 8650, - accessKey: 'ak', - secretKey: 'sk', + { + description: 'with auth `account` and buckets from zookeeper (compat mode)', + lifecycleConfig: baseLCConfig, + setupZookeeper: true, + transformExpectedMessages: identity, + }, + { + description: 'with auth `assumeRole` and buckets from bucketd', + lifecycleConfig: { + ...baseLCConfig, + conductor: { + ...baseLCConfig.conductor, + bucketSource: 'bucketd', + bucketd: { + host: '127.0.0.1', + }, }, - vault: { - host: '127.0.0.1', + auth: { + type: 'assumeRole', + roleName: 'lc', + sts: { + host: '127.0.0.1', + port: 8650, + accessKey: 'ak', + secretKey: 'sk', + }, + vault: { + host: '127.0.0.1', + }, }, }, - }, - mockBucketd: true, - mockVault: true, - transformExpectedMessages: withAccountIds, - skip: 'to be reintroduced with https://scality.atlassian.net/browse/BB-126', + mockBucketd: true, + mockVault: true, + transformExpectedMessages: withAccountIds, + } + ])('$description', options => { + describeConductorSpec(options); }); }); diff --git a/tests/functional/lifecycle/LifecycleObjectProcessor.spec.js b/tests/functional/lifecycle/LifecycleObjectProcessor.spec.js index 461bf845c..6d8632952 100644 --- a/tests/functional/lifecycle/LifecycleObjectProcessor.spec.js +++ b/tests/functional/lifecycle/LifecycleObjectProcessor.spec.js @@ -23,8 +23,8 @@ werelogs.configure({ level: 'warn', dump: 'error' }); const backbeatRoutes = ['deleteObject']; -describe('Lifecycle Object Processor', function lifecycleObjectProcessor() { - this.timeout(testTimeout); +describe('Lifecycle Object Processor', () => { + jest.setTimeout(testTimeout); function generateRetryTest(name, failures, message) { return function testRetries(done) { diff --git a/tests/functional/lifecycle/LifecycleTask.js b/tests/functional/lifecycle/LifecycleTask.spec.js similarity index 90% rename from tests/functional/lifecycle/LifecycleTask.js rename to tests/functional/lifecycle/LifecycleTask.spec.js index 4b6377f95..0417e153e 100644 --- a/tests/functional/lifecycle/LifecycleTask.js +++ b/tests/functional/lifecycle/LifecycleTask.spec.js @@ -472,15 +472,15 @@ s3mock, params, cb) { }); } -describe('lifecycle task functional tests', function dF() { - this.timeout(10000); +describe('lifecycle task functional tests', () => { + jest.setTimeout(10000); let lcp; let lcTask; let s3; let s3Helper; - before(() => { + beforeAll(() => { lcp = new LifecycleBucketProcessorMock(); s3 = new S3(s3config); lcTask = new LifecycleTask(lcp); @@ -798,7 +798,7 @@ describe('lifecycle task functional tests', function dF() { }); }); - [ + it.each([ // expire: pagination, prefix { message: 'should verify that EXPIRED objects are sent to ' + @@ -901,40 +901,38 @@ describe('lifecycle task functional tests', function dF() { objectCount: 2, }, }, - ].forEach(item => { - it(item.message, done => { - const params = { - lcTask, - lcp, - counter: 0, - }; - const bucket = item.bucketEntry.target.bucket; - async.waterfall([ - next => s3Helper.setAndCreateBucket(bucket, next), - next => s3Helper.setBucketLifecycleConfigurations( - item.bucketLCRules, next), - (data, next) => s3Helper.createObjects(item.scenario, next), - next => s3.getBucketLifecycleConfiguration({ - Bucket: bucket, - }, next), - (data, next) => { - wrapProcessBucketEntry(data.Rules, item.bucketEntry, s3, + ])('$message', (item, done) => { + const params = { + lcTask, + lcp, + counter: 0, + }; + const bucket = item.bucketEntry.target.bucket; + async.waterfall([ + next => s3Helper.setAndCreateBucket(bucket, next), + next => s3Helper.setBucketLifecycleConfigurations( + item.bucketLCRules, next), + (data, next) => s3Helper.createObjects(item.scenario, next), + next => s3.getBucketLifecycleConfiguration({ + Bucket: bucket, + }, next), + (data, next) => { + wrapProcessBucketEntry(data.Rules, item.bucketEntry, s3, params, (err, data) => { assert.ifError(err); - assert.equal(data.count.bucket, - item.expected.bucketCount); - assert.equal(data.count.object, - item.expected.objectCount); - assert.deepStrictEqual(data.entries.object.sort(), - item.expected.objects.sort()); - next(); - }); - }, - ], err => { - assert.ifError(err); - done(); - }); + assert.equal(data.count.bucket, + item.expected.bucketCount); + assert.equal(data.count.object, + item.expected.objectCount); + assert.deepStrictEqual(data.entries.object.sort(), + item.expected.objects.sort()); + next(); + }); + }, + ], err => { + assert.ifError(err); + done(); }); }); }); // end non-versioned describe block @@ -1242,7 +1240,7 @@ describe('lifecycle task functional tests', function dF() { }); // 1 Day Expiration rule - handling the IsLatest versions - [ + it.each([ { message: 'should expire a version in a versioning enabled ' + 'bucket with 0 non-current versions using basic ' + @@ -1331,83 +1329,81 @@ describe('lifecycle task functional tests', function dF() { objectCount: 0, }, }, - ].forEach(item => { - it(item.message, done => { - const Bucket = 'test-bucket'; - const Key = 'test-key1'; - const bucketEntry = { - action: 'testing-islatest', - target: { - bucket: Bucket, - owner: OWNER, - }, - details: {}, - }; - const params = { - lcTask, - lcp, - counter: 0, - }; + ])('$message', (item, done) => { + const Bucket = 'test-bucket'; + const Key = 'test-key1'; + const bucketEntry = { + action: 'testing-islatest', + target: { + bucket: Bucket, + owner: OWNER, + }, + details: {}, + }; + const params = { + lcTask, + lcp, + counter: 0, + }; - async.waterfall([ - next => s3Helper.setAndCreateBucket(Bucket, next), - next => s3Helper.setBucketVersioning('Enabled', next), - (data, next) => s3.putObject({ Bucket, Key, Body: '' }, - next), - (data, next) => { - if (item.isDeleteMarker) { - return async.series([ - cb => s3.deleteObject({ Bucket, Key }, - err => { - if (err) { - return cb(err); - } - return cb(); - }), - cb => { - if (!item.hasNonCurrentVersions) { - return s3.deleteObject({ - Bucket, Key, - VersionId: data.VersionId, - }, cb); + async.waterfall([ + next => s3Helper.setAndCreateBucket(Bucket, next), + next => s3Helper.setBucketVersioning('Enabled', next), + (data, next) => s3.putObject({ Bucket, Key, Body: '' }, + next), + (data, next) => { + if (item.isDeleteMarker) { + return async.series([ + cb => s3.deleteObject({ Bucket, Key }, + err => { + if (err) { + return cb(err); } return cb(); - }, - ], next); - } - if (item.hasNonCurrentVersions) { - return s3.putObject({ Bucket, Key, Body: '' }, - next); - } - return next(null, null); - }, - (data, next) => s3Helper.setBucketVersioning( - item.versionStatus, next), - (data, next) => { - s3Helper.setBucketLifecycleConfigurations([ - new LifecycleRule().addID('task-1') - .addExpiration('Date', PAST).build(), + }), + cb => { + if (!item.hasNonCurrentVersions) { + return s3.deleteObject({ + Bucket, Key, + VersionId: data.VersionId, + }, cb); + } + return cb(); + }, ], next); - }, - (data, next) => s3.getBucketLifecycleConfiguration( - { Bucket }, next), - (data, next) => { - wrapProcessBucketEntry(data.Rules, bucketEntry, s3, + } + if (item.hasNonCurrentVersions) { + return s3.putObject({ Bucket, Key, Body: '' }, + next); + } + return next(null, null); + }, + (data, next) => s3Helper.setBucketVersioning( + item.versionStatus, next), + (data, next) => { + s3Helper.setBucketLifecycleConfigurations([ + new LifecycleRule().addID('task-1') + .addExpiration('Date', PAST).build(), + ], next); + }, + (data, next) => s3.getBucketLifecycleConfiguration( + { Bucket }, next), + (data, next) => { + wrapProcessBucketEntry(data.Rules, bucketEntry, s3, params, (err, data) => { assert.ifError(err); assert.equal(data.count.object, item.expected.objectCount); next(); }); - }, - ], err => { - assert.ifError(err); - done(); - }); + }, + ], err => { + assert.ifError(err); + done(); }); }); - [ + it.each([ { message: 'should apply ExpiredObjectDeleteMarker rule on ' + 'only a delete marker in a versioning enabled bucket ' + @@ -1450,32 +1446,31 @@ describe('lifecycle task functional tests', function dF() { objectCount: 1, }, }, - ].forEach(item => { - it(item.message, done => { - const bucket = 'test-bucket'; - const keyName = 'test-key'; - const bucketEntry = { - action: 'testing-eodm', - target: { - bucket, - owner: item.owner, - }, - details: {}, - }; - const params = { - lcTask, - lcp, - counter: 0, - }; - async.waterfall([ - next => s3Helper.setupEODM(bucket, keyName, next), - (data, next) => s3Helper.setBucketLifecycleConfigurations( - item.bucketLCRules, next), - (data, next) => s3.getBucketLifecycleConfiguration({ - Bucket: bucket, - }, next), - (data, next) => { - wrapProcessBucketEntry(data.Rules, bucketEntry, s3, + ])('$message', (item, done) => { + const bucket = 'test-bucket'; + const keyName = 'test-key'; + const bucketEntry = { + action: 'testing-eodm', + target: { + bucket, + owner: item.owner, + }, + details: {}, + }; + const params = { + lcTask, + lcp, + counter: 0, + }; + async.waterfall([ + next => s3Helper.setupEODM(bucket, keyName, next), + (data, next) => s3Helper.setBucketLifecycleConfigurations( + item.bucketLCRules, next), + (data, next) => s3.getBucketLifecycleConfiguration({ + Bucket: bucket, + }, next), + (data, next) => { + wrapProcessBucketEntry(data.Rules, bucketEntry, s3, params, (err, data) => { assert.ifError(err); @@ -1483,15 +1478,14 @@ describe('lifecycle task functional tests', function dF() { item.expected.objectCount); next(); }); - }, - ], err => { - assert.ifError(err); - done(); - }); + }, + ], err => { + assert.ifError(err); + done(); }); }); - [ + it.each([ // ncve: basic 1 day rule should expire, no pagination { message: 'should verify that NoncurrentVersionExpiration rule' + @@ -1715,26 +1709,25 @@ describe('lifecycle task functional tests', function dF() { objectCount: 0, }, }, - ].forEach(item => { - it(item.message, done => { - const params = { - lcTask, - lcp, - counter: 0, - queuedEntries: [], - }; - const bucket = item.bucketEntry.target.bucket; - async.waterfall([ - next => s3Helper.setAndCreateBucket(bucket, next), - next => s3Helper.setBucketLifecycleConfigurations( - item.bucketLCRules, next), - (data, next) => s3Helper[item.scenarioFxn](item.scenario, - next), - next => s3.getBucketLifecycleConfiguration({ - Bucket: bucket, - }, next), - (data, next) => { - wrapProcessBucketEntry(data.Rules, item.bucketEntry, s3, + ])('$message', (item, done) => { + const params = { + lcTask, + lcp, + counter: 0, + queuedEntries: [], + }; + const bucket = item.bucketEntry.target.bucket; + async.waterfall([ + next => s3Helper.setAndCreateBucket(bucket, next), + next => s3Helper.setBucketLifecycleConfigurations( + item.bucketLCRules, next), + (data, next) => s3Helper[item.scenarioFxn](item.scenario, + next), + next => s3.getBucketLifecycleConfiguration({ + Bucket: bucket, + }, next), + (data, next) => { + wrapProcessBucketEntry(data.Rules, item.bucketEntry, s3, params, (err, data) => { assert.ifError(err); @@ -1747,11 +1740,10 @@ describe('lifecycle task functional tests', function dF() { next(); }); - }, - ], err => { - assert.ifError(err); - done(); - }); + }, + ], err => { + assert.ifError(err); + done(); }); }); }); // end versioned describe block @@ -1759,7 +1751,7 @@ describe('lifecycle task functional tests', function dF() { describe('incomplete mpu objects', () => { const bucketName = 'test-mpu-bucket'; - before(done => { + beforeAll(done => { s3Helper.setAndCreateBucket(bucketName, done); }); @@ -1783,7 +1775,7 @@ describe('lifecycle task functional tests', function dF() { }); }); - after(done => { + afterAll(done => { s3.deleteBucket({ Bucket: bucketName }, done); }); diff --git a/tests/functional/lifecycle/LifecycleTaskV2-versioned.js b/tests/functional/lifecycle/LifecycleTaskV2-versioned.js index a30385034..a6903f79f 100644 --- a/tests/functional/lifecycle/LifecycleTaskV2-versioned.js +++ b/tests/functional/lifecycle/LifecycleTaskV2-versioned.js @@ -58,7 +58,7 @@ describe('LifecycleTaskV2 with bucket versioned', () => { let backbeatMetadataProxy; let s3; - before(() => { + beforeAll(() => { const producer = { sendToTopic: (topicName, entry, cb) => { kafkaEntries.push({ topicName, entry }); diff --git a/tests/functional/lifecycle/LifecycleTaskV2.js b/tests/functional/lifecycle/LifecycleTaskV2.js index 573d3c129..615af86c8 100644 --- a/tests/functional/lifecycle/LifecycleTaskV2.js +++ b/tests/functional/lifecycle/LifecycleTaskV2.js @@ -59,7 +59,7 @@ describe('LifecycleTaskV2 with bucket non-versioned', () => { let backbeatMetadataProxy; let s3; - before(() => { + beforeAll(() => { const producer = { sendToTopic: (topicName, entry, cb) => { kafkaEntries.push({ topicName, entry }); diff --git a/tests/functional/notification/NotificationConfigManager.js b/tests/functional/notification/NotificationConfigManager.spec.js similarity index 95% rename from tests/functional/notification/NotificationConfigManager.js rename to tests/functional/notification/NotificationConfigManager.spec.js index ed3d2f224..ffafcb07d 100644 --- a/tests/functional/notification/NotificationConfigManager.js +++ b/tests/functional/notification/NotificationConfigManager.spec.js @@ -31,6 +31,21 @@ const notificationConfigurationVariant = { ], }; +class MockChangeStream extends events.EventEmitter { + constructor() { + super(); + this.closed = false; + } + + isClosed() { + return this.closed; + } + + close() { + this.closed = true; + } +} + describe('NotificationConfigManager ::', () => { const params = { mongoConfig, @@ -42,7 +57,7 @@ describe('NotificationConfigManager ::', () => { manager = new NotificationConfigManager(params); const getCollectionStub = sinon.stub().returns({ // mock change stream - watch: () => new events.EventEmitter(), + watch: () => new MockChangeStream(), // mock bucket notification configuration findOne: () => ( { diff --git a/tests/functional/replication/pauseResumeState.js b/tests/functional/replication/pauseResumeState.spec.js similarity index 99% rename from tests/functional/replication/pauseResumeState.js rename to tests/functional/replication/pauseResumeState.spec.js index f0b05bbbc..580a28362 100644 --- a/tests/functional/replication/pauseResumeState.js +++ b/tests/functional/replication/pauseResumeState.spec.js @@ -44,8 +44,8 @@ function isConsumerActive(consumer) { return consumer.getServiceStatus(); } -describe('CRR Pause/Resume status updates', function d() { - this.timeout(10000); +describe('CRR Pause/Resume status updates', () => { + jest.setTimeout(10000); let zkHelper; let mockAPI; const firstSite = destConfig.bootstrapList[0].site; @@ -61,7 +61,7 @@ describe('CRR Pause/Resume status updates', function d() { let replayProcessor2; let zkReplayHelper; - before(done => { + beforeAll(done => { mockAPI = new MockAPI(repConfig); zkHelper = new ZKStateHelper(config.zookeeper, ZK_TEST_CRR_STATE_PATH, firstSite, secondSite, futureDate); @@ -97,7 +97,7 @@ describe('CRR Pause/Resume status updates', function d() { }); }); - before(done => { + beforeAll(done => { zkReplayHelper = new ZKStateHelper(config.zookeeper, ZK_TEST_CRR_REPLAY_STATE_PATH, firstSite, secondSite, futureDate); zkReplayHelper.init(err => { @@ -165,7 +165,7 @@ describe('CRR Pause/Resume status updates', function d() { }); }); - after(() => { + afterAll(() => { zkHelper.close(); zkReplayHelper.close(); }); diff --git a/tests/functional/replication/queueProcessor.js b/tests/functional/replication/queueProcessor.spec.js similarity index 84% rename from tests/functional/replication/queueProcessor.js rename to tests/functional/replication/queueProcessor.spec.js index 38b7c26c3..304d3c843 100644 --- a/tests/functional/replication/queueProcessor.js +++ b/tests/functional/replication/queueProcessor.spec.js @@ -749,7 +749,7 @@ function sendCopyLocationAction(s3mock, queueProcessor, resultsCb) { }); } -/* eslint-enable max-len */ +jest.setTimeout(60000); describe('queue processor functional tests with mocking', () => { let queueProcessorSF; @@ -763,8 +763,7 @@ describe('queue processor functional tests with mocking', () => { const ReplicationStatusProcessor = require('../../../extensions/replication' + '/replicationStatusProcessor/ReplicationStatusProcessor'); - before(function before(done) { - this.timeout(60000); + beforeAll(done => { const serverList = constants.target.hosts.map(h => `${h.host}:${h.port}`); @@ -877,9 +876,9 @@ describe('queue processor functional tests with mocking', () => { httpServer.listen(7777); }); - after(done => { + afterAll(done => { httpServer.close(); - async.parallel([ + async.series([ done => queueProcessorSF.stop(done), done => queueProcessorAzure.stop(done), done => replicationStatusProcessor.stop(done), @@ -891,10 +890,9 @@ describe('queue processor functional tests with mocking', () => { s3mock.resetTest(); }); - describe('success path tests', function successPath() { - this.timeout(30000); - - [{ caption: 'object with single part', + describe('success path tests', () => { + describe.each([ + { caption: 'object with single part', nbParts: 1 }, { caption: 'encrypted object with single part', nbParts: 1, @@ -905,45 +903,46 @@ describe('queue processor functional tests with mocking', () => { nbParts: 2, encrypted: true }, { caption: 'empty object', - nbParts: 0 }].forEach(testCase => describe(testCase.caption, () => { - before(() => { - s3mock.setParam('nbParts', testCase.nbParts); - if (testCase.encrypted) { - s3mock.setParam('target.bucketIsEncrypted', true); - } - }); - it('should complete a replication end-to-end', done => { - async.parallel([ - done => { - s3mock.onPutSourceMd = done; - }, - done => queueProcessorSF.processReplicationEntry( - s3mock.getParam('kafkaEntry'), err => { - assert.ifError(err); - assert.strictEqual(s3mock.hasPutTargetData, - testCase.nbParts > 0); - assert(s3mock.hasPutTargetMd); - assert.strictEqual(s3mock.partsDeleted.length, 0); - done(); - }), - ], done); - }); - - it('should complete a "copy location" action', done => { - sendCopyLocationAction( - s3mock, queueProcessorSF, response => { - assert.strictEqual(response.getError(), undefined); - assert.strictEqual(response.getStatus(), 'success'); - const results = response.getResults(); - assert.strictEqual( - Array.isArray(results.location), true); - assert.strictEqual(results.location.length, 1); - // 0-byte objects must also be copied - assert.strictEqual(s3mock.hasPutTargetData, true); + nbParts: 0 }, + ])('$caption', testCase => { + beforeAll(() => { + s3mock.setParam('nbParts', testCase.nbParts); + if (testCase.encrypted) { + s3mock.setParam('target.bucketIsEncrypted', true); + } + }); + it('should complete a replication end-to-end', done => { + async.parallel([ + done => { + s3mock.onPutSourceMd = done; + }, + done => queueProcessorSF.processReplicationEntry( + s3mock.getParam('kafkaEntry'), err => { + assert.ifError(err); + assert.strictEqual(s3mock.hasPutTargetData, + testCase.nbParts > 0); + assert(s3mock.hasPutTargetMd); + assert.strictEqual(s3mock.partsDeleted.length, 0); done(); - }); - }); - })); + }), + ], done); + }); + + it('should complete a "copy location" action', done => { + sendCopyLocationAction( + s3mock, queueProcessorSF, response => { + assert.strictEqual(response.getError(), undefined); + assert.strictEqual(response.getStatus(), 'success'); + const results = response.getResults(); + assert.strictEqual( + Array.isArray(results.location), true); + assert.strictEqual(results.location.length, 1); + // 0-byte objects must also be copied + assert.strictEqual(s3mock.hasPutTargetData, true); + done(); + }); + }); + }); it('should replicate metadata in metadata-only mode', done => { s3mock.setParam('nbParts', 2); @@ -984,38 +983,33 @@ describe('queue processor functional tests with mocking', () => { }), ], done); }); - }); - - describe('error paths', function errorPaths() { - this.timeout(20000); // give more time to leave room for retry - // delays and timeout + }, 30000); + describe('error paths', () => { describe('source Vault errors', () => { - ['assumeRoleBackbeat'].forEach(action => { - [errors.AccessDenied, errors.NoSuchEntity].forEach(error => { - it(`should skip processing on ${error.code} ` + - `(${error.message}) from source Vault on ${action}`, - done => { - s3mock.installVaultErrorResponder( - `source.vault.${action}`, error); + it.each([ + { action: 'assumeRoleBackbeat', error: errors.NoSuchEntity }, + { action: 'assumeRoleBackbeat', error: errors.AccessDenied }, + ])('should skip processing on $error from source Vault on $action', + ({ action, error }, done) => { + s3mock.installVaultErrorResponder( + `source.vault.${action}`, error); - queueProcessorSF.processReplicationEntry( - s3mock.getParam('kafkaEntry'), err => { - assert.ifError(err); - assert(!s3mock.hasPutTargetData); - assert(!s3mock.hasPutTargetMd); - assert.strictEqual(s3mock.partsDeleted.length, 0); - done(); - }); - }); + queueProcessorSF.processReplicationEntry( + s3mock.getParam('kafkaEntry'), err => { + assert.ifError(err); + assert(!s3mock.hasPutTargetData); + assert(!s3mock.hasPutTargetMd); + assert.strictEqual(s3mock.partsDeleted.length, 0); + done(); + }); }); - }); }); describe('source S3 errors', () => { - [errors.AccessDenied, errors.ObjNotFound].forEach(error => { - it(`replication should skip on ${error.code} ` + - `(${error.message}) from source S3 on getObject`, done => { + it.each([errors.AccessDenied, errors.ObjNotFound])( + 'replication should skip on $error from source S3 on getObject', + (error, done) => { s3mock.installS3ErrorResponder('source.s3.getObject', error); queueProcessorSF.processReplicationEntry( @@ -1028,8 +1022,9 @@ describe('queue processor functional tests with mocking', () => { }); }); - it(`copy location should skip on ${error.code} ` + - `(${error.message}) from source S3 on getObject`, done => { + it.each([errors.AccessDenied, errors.ObjNotFound])( + 'copy location should skip on $error from source S3 on getObject', + (error, done) => { s3mock.installS3ErrorResponder('source.s3.getObject', error); sendCopyLocationAction( @@ -1052,7 +1047,6 @@ describe('queue processor functional tests with mocking', () => { done(); }); }); - }); it('should fail if replication is disabled in bucket replication ' + 'configuration', done => { @@ -1115,29 +1109,28 @@ describe('queue processor functional tests with mocking', () => { }); }); - ['getBucketReplication', 'getObject'].forEach(action => { - [errors.InternalError].forEach(error => { - it(`replication should retry on ${error.code} ` + - `(${error.message}) from source S3 on ${action}`, done => { - s3mock.installS3ErrorResponder( - `source.s3.${action}`, error, { once: true }); - - async.parallel([ - done => { - s3mock.onPutSourceMd = done; - }, - done => queueProcessorSF.processReplicationEntry( - s3mock.getParam('kafkaEntry'), err => { - assert.ifError(err); - assert(s3mock.hasPutTargetData); - assert(s3mock.hasPutTargetMd); - assert.strictEqual(s3mock.partsDeleted.length, 0); - done(); - }), - ], done); - }); + it.each([ + { error: errors.InternalError, action: 'getBucketReplication' }, + { error: errors.InternalError, action: 'getObject' }, + ])('replication should retry on $error from source S3 on $action', + ({ error, action }, done) => { + s3mock.installS3ErrorResponder( + `source.s3.${action}`, error, { once: true }); + + async.parallel([ + done => { + s3mock.onPutSourceMd = done; + }, + done => queueProcessorSF.processReplicationEntry( + s3mock.getParam('kafkaEntry'), err => { + assert.ifError(err); + assert(s3mock.hasPutTargetData); + assert(s3mock.hasPutTargetMd); + assert.strictEqual(s3mock.partsDeleted.length, 0); + done(); + }), + ], done); }); - }); it('copy location should retry on 500 (InternalError) ' + 'from source S3 on getObject', done => { @@ -1197,123 +1190,115 @@ describe('queue processor functional tests with mocking', () => { }); describe('target Vault errors', () => { - ['getAccountsCanonicalIds', - 'assumeRoleBackbeat'].forEach(action => { - [errors.AccessDenied, errors.NoSuchEntity].forEach(err => { - it(`should fail on ${err.code} (${err.message}) ` + - `from target Vault on ${action}`, done => { - s3mock.installVaultErrorResponder( - `target.${action}`, err); - s3mock.setExpectedReplicationStatus('FAILED'); - - async.parallel([ - done => { - s3mock.onPutSourceMd = done; - }, - done => - queueProcessorSF.processReplicationEntry( - s3mock.getParam('kafkaEntry'), - error => { - assert.ifError(error); - assert(!s3mock.hasPutTargetData); - assert(!s3mock.hasPutTargetMd); - assert.strictEqual(s3mock.partsDeleted.length, 0); - done(); - }), - ], done); - }); - }); - }); + it.each([ + { err: errors.AccessDenied, action: 'getAccountsCanonicalIds' }, + { err: errors.NoSuchEntity, action: 'getAccountsCanonicalIds' }, + { err: errors.AccessDenied, action: 'assumeRoleBackbeat' }, + { err: errors.NoSuchEntity, action: 'assumeRoleBackbeat' }, + ])('should fail on $err from target Vault on $action', ({ err, action }, done) => { + s3mock.installVaultErrorResponder( + `target.${action}`, err); + s3mock.setExpectedReplicationStatus('FAILED'); - ['getAccountsCanonicalIds', - 'assumeRoleBackbeat'].forEach(action => { - [errors.InternalError].forEach(error => { - it(`should retry on ${error.code} (${error.message}) ` + - `from target Vault on ${action}`, done => { - s3mock.installVaultErrorResponder( - `target.${action}`, error, { once: true }); - - async.parallel([ - done => { - s3mock.onPutSourceMd = done; - }, - done => queueProcessorSF.processReplicationEntry( - s3mock.getParam('kafkaEntry'), err => { - assert.ifError(err); - assert(s3mock.hasPutTargetData); - assert(s3mock.hasPutTargetMd); - assert.strictEqual(s3mock.partsDeleted.length, 0); - // should have retried on other host - assert(s3mock.requestsPerHost['127.0.0.3'] - > 0); - assert(s3mock.requestsPerHost['127.0.0.4'] - > 0); - done(); - }), - ], done); - }); - }); + async.parallel([ + done => { + s3mock.onPutSourceMd = done; + }, + done => + queueProcessorSF.processReplicationEntry( + s3mock.getParam('kafkaEntry'), + error => { + assert.ifError(error); + assert(!s3mock.hasPutTargetData); + assert(!s3mock.hasPutTargetMd); + assert.strictEqual(s3mock.partsDeleted.length, 0); + done(); + }), + ], done); + }); + + it.each([ + { error: errors.InternalError, action: 'getAccountsCanonicalIds' }, + { error: errors.InternalError, action: 'assumeRoleBackbeat' }, + ])('should retry on $error from target Vault on $action', ({ error, action }, done) => { + s3mock.installVaultErrorResponder( + `target.${action}`, error, { once: true }); + + async.parallel([ + done => { + s3mock.onPutSourceMd = done; + }, + done => queueProcessorSF.processReplicationEntry( + s3mock.getParam('kafkaEntry'), err => { + assert.ifError(err); + assert(s3mock.hasPutTargetData); + assert(s3mock.hasPutTargetMd); + assert.strictEqual(s3mock.partsDeleted.length, 0); + // should have retried on other host + assert(s3mock.requestsPerHost['127.0.0.3'] + > 0); + assert(s3mock.requestsPerHost['127.0.0.4'] + > 0); + done(); + }), + ], done); }); }); describe('target S3 errors', () => { - ['putData', 'putMetadata'].forEach(action => { - [errors.AccessDenied].forEach(error => { - it(`should fail on ${error.code} (${error.message}) ` + - `from target S3 on ${action}`, done => { - s3mock.installS3ErrorResponder(`target.${action}`, - error); - s3mock.setExpectedReplicationStatus('FAILED'); - - async.parallel([ - done => { - s3mock.onPutSourceMd = done; - }, - done => queueProcessorSF.processReplicationEntry( - s3mock.getParam('kafkaEntry'), err => { - assert.ifError(err); - assert(!s3mock.hasPutTargetMd); - if (action === 'putMetadata') { - assert.deepStrictEqual(s3mock.partsDeleted, [{ - key: constants.target.dataPartsKeys[0], - dataStoreName: 'file', - }]); - } else { - assert.strictEqual(s3mock.partsDeleted.length, 0); - } - done(); - }), - ], done); - }); - }); + it.each([ + { error: errors.AccessDenied, action: 'putData' }, + { error: errors.AccessDenied, action: 'putMetadata' }, + ])('should fail on $error from target S3 on $action', ({ error, action }, done) => { + s3mock.installS3ErrorResponder(`target.${action}`, + error); + s3mock.setExpectedReplicationStatus('FAILED'); + + async.parallel([ + done => { + s3mock.onPutSourceMd = done; + }, + done => queueProcessorSF.processReplicationEntry( + s3mock.getParam('kafkaEntry'), err => { + assert.ifError(err); + assert(!s3mock.hasPutTargetMd); + if (action === 'putMetadata') { + assert.deepStrictEqual(s3mock.partsDeleted, [{ + key: constants.target.dataPartsKeys[0], + dataStoreName: 'file', + }]); + } else { + assert.strictEqual(s3mock.partsDeleted.length, 0); + } + done(); + }), + ], done); }); - ['putData', 'putMetadata'].forEach(action => { - [errors.InternalError].forEach(error => { - it(`should retry on ${error.code} (${error.message}) ` + - `from target S3 on ${action}`, done => { - s3mock.installS3ErrorResponder(`target.${action}`, - error, { once: true }); - async.parallel([ - done => { - s3mock.onPutSourceMd = done; - }, - done => queueProcessorSF.processReplicationEntry( - s3mock.getParam('kafkaEntry'), err => { - assert.ifError(err); - assert(s3mock.hasPutTargetData); - assert(s3mock.hasPutTargetMd); - assert.strictEqual(s3mock.partsDeleted.length, 0); - // should have retried on other host - assert(s3mock.requestsPerHost['127.0.0.3'] - > 0); - assert(s3mock.requestsPerHost['127.0.0.4'] - > 0); - done(); - }), - ], done); - }); - }); + it.each([ + { error: errors.InternalError, action: 'putData' }, + { error: errors.InternalError, action: 'putMetadata' }, + ])('should retry on $error from target S3 on $action', ({ error, action }, done) => { + s3mock.installS3ErrorResponder(`target.${action}`, + error, { once: true }); + async.parallel([ + done => { + s3mock.onPutSourceMd = done; + }, + done => queueProcessorSF.processReplicationEntry( + s3mock.getParam('kafkaEntry'), err => { + assert.ifError(err); + assert(s3mock.hasPutTargetData); + assert(s3mock.hasPutTargetMd); + assert.strictEqual(s3mock.partsDeleted.length, 0); + // should have retried on other host + assert(s3mock.requestsPerHost['127.0.0.3'] + > 0); + assert(s3mock.requestsPerHost['127.0.0.4'] + > 0); + done(); + }), + ], done); }); it('copy location should retry on 500 (InternalError) ' + diff --git a/tests/functional/utils/BackbeatTestConsumer.spec.js b/tests/functional/utils/BackbeatTestConsumer.spec.js index 6bb393971..a66774870 100644 --- a/tests/functional/utils/BackbeatTestConsumer.spec.js +++ b/tests/functional/utils/BackbeatTestConsumer.spec.js @@ -8,7 +8,7 @@ const TIMEOUT = 60000; const TOPIC = 'backbeat-test-consumer-spec'; const GROUP_ID = 'test-consumer-group'; -describe('BackbeatTestConsumer', function backbeatTestConsumer() { +describe('BackbeatTestConsumer', () => { let consumer; let producer; const messages = [ @@ -17,9 +17,9 @@ describe('BackbeatTestConsumer', function backbeatTestConsumer() { { message: '{"value":"m3"}' }, ]; - this.timeout(TIMEOUT); + jest.setTimeout(TIMEOUT); - before(done => { + beforeAll(done => { async.series([ next => { producer = new BackbeatProducer({ @@ -49,7 +49,7 @@ describe('BackbeatTestConsumer', function backbeatTestConsumer() { ], done); }); - after(done => { + afterAll(done => { async.waterfall([ next => producer.close(next), next => consumer.close(next), diff --git a/tests/performance/lifecycle/conductor-check-memory-balloon.js b/tests/performance/lifecycle/conductor-check-memory-balloon.spec.js similarity index 98% rename from tests/performance/lifecycle/conductor-check-memory-balloon.js rename to tests/performance/lifecycle/conductor-check-memory-balloon.spec.js index 9933349fc..429b789e6 100644 --- a/tests/performance/lifecycle/conductor-check-memory-balloon.js +++ b/tests/performance/lifecycle/conductor-check-memory-balloon.spec.js @@ -9,8 +9,8 @@ const { BackbeatMetadataProxyMock } = require('../../functional/lifecycle/utils' const nBuckets = 200000; -describe('Lifecycle Conductor', function testBackpressure() { - this.timeout(10 * 60 * 1000); +describe('Lifecycle Conductor', () => { + jest.setTimeout(10 * 60 * 1000); describe('with bucketd', () => { const lcConfig = { diff --git a/tests/unit/OffsetLedger.spec.js b/tests/unit/OffsetLedger.spec.js index dfa041181..1cc037edc 100644 --- a/tests/unit/OffsetLedger.spec.js +++ b/tests/unit/OffsetLedger.spec.js @@ -1,6 +1,6 @@ const assert = require('assert'); -const OffsetLedger = require('../../lib/OffsetLedger'); +const OffsetLedger = require('../../lib/OffsetLedger').default; describe('OffsetLedger', () => { it('should get no committable offset if no offset registered yet', () => { diff --git a/tests/unit/ProvisionDispatcher.spec.js b/tests/unit/ProvisionDispatcher.spec.js index 7d06bdafa..f8a520c7f 100644 --- a/tests/unit/ProvisionDispatcher.spec.js +++ b/tests/unit/ProvisionDispatcher.spec.js @@ -11,19 +11,19 @@ const ProvisionDispatcher = const ZK_TEST_PATH = '/tests/prov-test'; describe('provision dispatcher based on zookeeper recipes', -function testDispatch() { +() => { const zkConf = { connectionString: `localhost:2181${ZK_TEST_PATH}` }; const provisionList = ['0', '1', '2', '3', '4', '5', '6', '7']; let clients = []; - this.timeout(60000); + jest.setTimeout(60000); const zk = new ZookeeperMock({ doLog: false, maxRandomDelay: 100, }); - before(done => { + beforeAll(done => { const zkClient = zk.createClient('localhost:2181'); zkClient.connect(); zkClient.on('connected', () => { diff --git a/tests/unit/ReplicationMetric.js b/tests/unit/ReplicationMetric.spec.js similarity index 100% rename from tests/unit/ReplicationMetric.js rename to tests/unit/ReplicationMetric.spec.js diff --git a/tests/unit/RoleCredentials.js b/tests/unit/RoleCredentials.spec.js similarity index 51% rename from tests/unit/RoleCredentials.js rename to tests/unit/RoleCredentials.spec.js index 577eab0ac..a7094eccf 100644 --- a/tests/unit/RoleCredentials.js +++ b/tests/unit/RoleCredentials.spec.js @@ -42,8 +42,8 @@ function _assertCredentials(err, roleCredentials, cb) { if (err) { return cb(err); } - const { accessKeyId, secretAccessKey, sessionToken, expired, - expiration } = roleCredentials; + const { accessKeyId, secretAccessKey, sessionToken, expired, expiration } = + roleCredentials; assert.strictEqual(accessKeyId, AccessKeyId); assert.strictEqual(secretAccessKey, SecretAccessKey); assert.strictEqual(sessionToken, SessionToken); @@ -55,100 +55,136 @@ function _assertCredentials(err, roleCredentials, cb) { describe('Credentials Manager', () => { let roleCredentials = null; let vaultServer = null; - before(done => { - const vaultclient = new Client(vaultHost, vaultPort, undefined, - undefined, undefined, undefined, undefined, undefined, undefined, - undefined, proxyPath); + beforeAll(done => { + const vaultclient = new Client( + vaultHost, + vaultPort, + undefined, + undefined, + undefined, + undefined, + undefined, + undefined, + undefined, + undefined, + proxyPath + ); roleCredentials = new RoleCredentials( - vaultclient, role, extension, + vaultclient, + role, + extension, new Logger('test:RoleCredentials').newRequestLogger('requids'), - 1); + 1 + ); vaultServer = server.listen(vaultPort).on('error', done); done(); }); afterEach(() => { simulateServerError = false; }); - after(() => { + afterAll(() => { roleCredentials = null; vaultServer.close(); }); it('should be able to acquire credentials on startup', done => { - roleCredentials.get(err => _assertCredentials(err, - roleCredentials, done)); + roleCredentials.get(err => + _assertCredentials(err, roleCredentials, done) + ); }); - it('should use same credentials if not expired or about to expire', function test(done) { - this.timeout(10000); + it('should use same credentials if not expired or about to expire', done => { roleCredentials.get(err => { if (err) { - return done(err); + done(err); + return; } const currentExpiration = roleCredentials.expiration; // wait for less than the expiration time minus the // anticipation delay to ensure credentials have not // expired - const retryTimeout = (roleCredentials.expiration - Date.now()) - 1500; - return setTimeout(() => roleCredentials.get( - err => _assertCredentials(err, roleCredentials, err => { - assert.ifError(err); - // expiration should not have changed, meaning - // credentials have not been refreshed - assert.strictEqual(currentExpiration, roleCredentials.expiration); - done(); - })), retryTimeout); + const retryTimeout = roleCredentials.expiration - Date.now() - 1500; + setTimeout( + () => + roleCredentials.get(err => + _assertCredentials(err, roleCredentials, err => { + assert.ifError(err); + // expiration should not have changed, meaning + // credentials have not been refreshed + assert.strictEqual( + currentExpiration, + roleCredentials.expiration + ); + done(); + }) + ), + retryTimeout + ); }); - }); + }, 10000); - it('should refresh credentials upon expiration', function test(done) { - this.timeout(10000); + it('should refresh credentials upon expiration', done => { roleCredentials.get(err => { if (err) { - return done(err); + done(err); + return; } const currentExpiration = roleCredentials.expiration; // wait for more than the expiration time to ensure // credentials have expired - const retryTimeout = (roleCredentials.expiration - Date.now()) + 1000; - return setTimeout(() => roleCredentials.get( - err => _assertCredentials(err, roleCredentials, err => { - assert.ifError(err); - // expiration should have changed, meaning - // credentials have been refreshed - assert.notStrictEqual(currentExpiration, roleCredentials.expiration); - done(); - })), retryTimeout); + const retryTimeout = roleCredentials.expiration - Date.now() + 1000; + setTimeout( + () => + roleCredentials.get(err => + _assertCredentials(err, roleCredentials, err => { + assert.ifError(err); + // expiration should have changed, meaning + // credentials have been refreshed + assert.notStrictEqual( + currentExpiration, + roleCredentials.expiration + ); + done(); + }) + ), + retryTimeout + ); }); - }); + }, 10000); - it('should refresh credentials a bit before expiration', function test(done) { - this.timeout(10000); + it('should refresh credentials a bit before expiration', done => { roleCredentials.get(err => { if (err) { - return done(err); + done(err); + return; } const currentExpiration = roleCredentials.expiration; // wait for slightly less than the expiration time but // more than the anticipation delay for renewing // credentials about to expire - const retryTimeout = (roleCredentials.expiration - Date.now()) - 100; - return setTimeout(() => roleCredentials.get( - err => _assertCredentials(err, roleCredentials, err => { - assert.ifError(err); - // expiration should have changed, meaning - // credentials have been refreshed - assert.notStrictEqual(currentExpiration, roleCredentials.expiration); - done(); - })), retryTimeout); + const retryTimeout = roleCredentials.expiration - Date.now() - 100; + setTimeout( + () => + roleCredentials.get(err => + _assertCredentials(err, roleCredentials, err => { + assert.ifError(err); + // expiration should have changed, meaning + // credentials have been refreshed + assert.notStrictEqual( + currentExpiration, + roleCredentials.expiration + ); + done(); + }) + ), + retryTimeout + ); }); - }); + }, 10000); - it('should handle non arsenal errors on refresh', function test(done) { - this.timeout(10000); - const retryTimeout = (roleCredentials.expiration - Date.now()) + - 1000; - return setTimeout(() => { + it('should handle non arsenal errors on refresh', done => { + const retryTimeout = roleCredentials.expiration - Date.now() + 1000; + setTimeout(() => { simulateServerError = true; roleCredentials.get(err => { assert(err); @@ -158,16 +194,28 @@ describe('Credentials Manager', () => { done(); }); }, retryTimeout); - }); + }, 10000); it('RoleCredentials should use a default renewal anticipation delay if not explicit', () => { const vaultclient = new Client( - vaultHost, vaultPort, undefined, - undefined, undefined, undefined, undefined, undefined, undefined, - undefined, proxyPath); + vaultHost, + vaultPort, + undefined, + undefined, + undefined, + undefined, + undefined, + undefined, + undefined, + undefined, + proxyPath + ); const rc = new RoleCredentials( - vaultclient, role, extension, - new Logger('test:RoleCredentials').newRequestLogger('requids')); + vaultclient, + role, + extension, + new Logger('test:RoleCredentials').newRequestLogger('requids') + ); assert(rc._refreshCredsAnticipationSeconds > 0); }); }); diff --git a/tests/unit/api/BackbeatAPI.spec.js b/tests/unit/api/BackbeatAPI.spec.js index 3f163195c..ecb1c15e9 100644 --- a/tests/unit/api/BackbeatAPI.spec.js +++ b/tests/unit/api/BackbeatAPI.spec.js @@ -16,7 +16,7 @@ describe('BackbeatAPI', () => { const coldSite = Object.keys(locationConfig) .filter(site => locationConfig[site].isCold)[0]; - before(() => { + beforeAll(() => { setupIngestionSiteMock(); bbapi = new BackbeatAPI(config, fakeLogger, { timer: true }); }); @@ -210,4 +210,10 @@ describe('BackbeatAPI', () => { assert.deepStrictEqual(sites, params.expected); }); }); + + afterAll(() => { + bbapi._redisPublisher.quit(); + bbapi._redisClient.disconnect(); + bbapi._metrics.disconnect(); + }); }); diff --git a/tests/unit/backbeatConsumer.js b/tests/unit/backbeatConsumer.spec.js similarity index 100% rename from tests/unit/backbeatConsumer.js rename to tests/unit/backbeatConsumer.spec.js diff --git a/tests/unit/backbeatProducer.js b/tests/unit/backbeatProducer.spec.js similarity index 100% rename from tests/unit/backbeatProducer.js rename to tests/unit/backbeatProducer.spec.js diff --git a/tests/unit/conf/Config.js b/tests/unit/conf/Config.spec.js similarity index 100% rename from tests/unit/conf/Config.js rename to tests/unit/conf/Config.spec.js diff --git a/tests/unit/credentials/CredentialsManager.js b/tests/unit/credentials/CredentialsManager.spec.js similarity index 82% rename from tests/unit/credentials/CredentialsManager.js rename to tests/unit/credentials/CredentialsManager.spec.js index 08ed102c4..c61af2a84 100644 --- a/tests/unit/credentials/CredentialsManager.js +++ b/tests/unit/credentials/CredentialsManager.spec.js @@ -48,80 +48,79 @@ describe('CredentialsManager', () => { const log = new Logger('test:CredentialsManager'); describe('::getCredentials', () => { - [ - // [msg, extensionConfig, expected] - [ - 'should return client for assumeRole type', - { - id: id1, accountId: accountId2, - authConfig: assumeRoleAuth, - stsConfig, - }, AWS.ChainableTemporaryCredentials, - ], - ].forEach(([msg, params, expected]) => it(msg, () => { + it('should return client for assumeRole type', () => { const mgr = new CredentialsManager(extension, log); const creds2 = mgr.getCredentials({ id: id2, accountId: accountId2, authConfig: assumeRoleAuth, stsConfig, }); - const creds = mgr.getCredentials(params); - assert(creds instanceof expected); - assert(mgr._accountCredsCache[id1] instanceof expected); + const creds = mgr.getCredentials({ + id: id1, accountId: accountId2, + authConfig: assumeRoleAuth, + stsConfig, + }); + assert(creds instanceof AWS.ChainableTemporaryCredentials); + assert(mgr._accountCredsCache[id1] instanceof AWS.ChainableTemporaryCredentials); assert.notEqual(creds, creds2); assert.notEqual(mgr._accountCredsCache[id1], mgr._accountCredsCache[id2]); - })); + }); - [ - [ - 'should return null when id is missing', - { + it.each([ + { + msg: 'should return null when id is missing', + params: { id: id1, accountId: accountId1, authConfig: assumeRoleAuth, - }, null, - ], - [ - 'should return null when authConfig is missing', - { + }, + expected: null, + }, + { + msg: 'should return null when authConfig is missing', + params: { id: id1, accountId: accountId1, authConfig: null, - }, null, - ], - [ - 'should return null when stsConfig is missing for assumeRole type', - { + }, + expected: null, + }, + { + msg: 'should return null when stsConfig is missing for assumeRole type', + params: { id: id1, accountId: accountId1, authConfig: assumeRoleAuth, stsConfig: null, - }, null, - ], - [ - 'should return null when accountId is missing for assumeRole type', - { + }, + expected: null, + }, + { + msg: 'should return null when accountId is missing for assumeRole type', + params: { id: id1, authConfig: assumeRoleAuth, stsConfig, - }, null, - ], - [ - 'should return null for unsupported auth types', - { + }, + expected: null, + }, + { + msg: 'should return null for unsupported auth types', + params: { id: id1, accountId: accountId1, authConfig: invalidAuth, - }, null, - ], - ].forEach(([msg, params, expected]) => it(msg, () => { + }, + expected: null, + }, + ])('$msg', ({ params, expected }) => { const mgr = new CredentialsManager(extension, log); const creds = mgr.getCredentials(params); assert.strictEqual(creds, expected); - })); + }); }); describe('::removeInactiveCredentials', () => { const stsResponses = sinon.stub(); let stsServer = null; - before(done => { + beforeAll(done => { const server = http.createServer(); server.on('request', (req, res) => { const payload = stsResponseToXML(stsResponses()); @@ -146,7 +145,7 @@ describe('CredentialsManager', () => { stsResponses.reset(); }); - after(() => { + afterAll(() => { stsServer.close(); }); diff --git a/tests/unit/gc/GarbageCollector.spec.js b/tests/unit/gc/GarbageCollector.spec.js index 51b2adf9f..8366a73fc 100644 --- a/tests/unit/gc/GarbageCollector.spec.js +++ b/tests/unit/gc/GarbageCollector.spec.js @@ -11,14 +11,14 @@ const ActionQueueEntry = require('../../../lib/models/ActionQueueEntry'); const ownerId = '79a59df900b949e55d96a1e698fbacedfd6e09d98eacf8f8d5218e7cd47ef2be'; -describe('garbage collector', function garbageCollector() { - this.timeout(10000); +describe('garbage collector', () => { + jest.setTimeout(10000); let gc; let gcTask; let httpServer; let expectBatchDeleteLocations; - before(() => { + beforeAll(() => { gc = new GarbageCollector({ kafkaConfig: {}, s3Config: { @@ -59,7 +59,7 @@ describe('garbage collector', function garbageCollector() { }); httpServer.listen(7777); }); - after(() => { + afterAll(() => { httpServer.close(); }); it('should skip unsupported action type', done => { diff --git a/tests/unit/gc/GarbageCollectorProducer.spec.js b/tests/unit/gc/GarbageCollectorProducer.spec.js index c167fd8fc..0b74bda7a 100644 --- a/tests/unit/gc/GarbageCollectorProducer.spec.js +++ b/tests/unit/gc/GarbageCollectorProducer.spec.js @@ -39,7 +39,7 @@ describe('garbage collector producer', () => { let gcProducer; const kafkaProducerMock = new KafkaProducerMock(); - before(() => { + beforeAll(() => { gcProducer = new GarbageCollectorProducer(); gcProducer._producer = kafkaProducerMock; }); diff --git a/tests/unit/ingestion/IngestionPopulator.js b/tests/unit/ingestion/IngestionPopulator.spec.js similarity index 100% rename from tests/unit/ingestion/IngestionPopulator.js rename to tests/unit/ingestion/IngestionPopulator.spec.js diff --git a/tests/unit/ingestion/IngestionReader.js b/tests/unit/ingestion/IngestionReader.spec.js similarity index 100% rename from tests/unit/ingestion/IngestionReader.js rename to tests/unit/ingestion/IngestionReader.spec.js diff --git a/tests/unit/ingestion/mongoProcessorMetrics.js b/tests/unit/ingestion/mongoProcessorMetrics.spec.js similarity index 100% rename from tests/unit/ingestion/mongoProcessorMetrics.js rename to tests/unit/ingestion/mongoProcessorMetrics.spec.js diff --git a/tests/unit/lib/models/ColdStorageStatusQueueEntry.spec.js b/tests/unit/lib/models/ColdStorageStatusQueueEntry.spec.js index 70efc7207..340c0b4e9 100644 --- a/tests/unit/lib/models/ColdStorageStatusQueueEntry.spec.js +++ b/tests/unit/lib/models/ColdStorageStatusQueueEntry.spec.js @@ -38,7 +38,7 @@ const malformed = Buffer.from(` }`); describe('ColdStorageStatusQueueEntry', () => { - [ + it.each([ { msg: 'should parse non-versioned object status entry', input: nonVersioned, @@ -59,7 +59,7 @@ describe('ColdStorageStatusQueueEntry', () => { input: malformed, error: { message: 'malformed JSON in kafka entry' }, }, - ].forEach(({ msg, input, error }) => it(msg, () => { + ])('$msg', ({ input, error }) => { const res = ColdStorageStatusQueueEntry.createFromKafkaEntry({ value: input }); if (error) { @@ -67,5 +67,5 @@ describe('ColdStorageStatusQueueEntry', () => { } else { assert.ifError(res.error); } - })); + }); }); diff --git a/tests/unit/lib/models/MetricsModel.js b/tests/unit/lib/models/MetricsModel.spec.js similarity index 100% rename from tests/unit/lib/models/MetricsModel.js rename to tests/unit/lib/models/MetricsModel.spec.js diff --git a/tests/unit/lib/queuePopulator/LogReader.spec.js b/tests/unit/lib/queuePopulator/LogReader.spec.js index 2843b2039..6bf0e7938 100644 --- a/tests/unit/lib/queuePopulator/LogReader.spec.js +++ b/tests/unit/lib/queuePopulator/LogReader.spec.js @@ -53,10 +53,10 @@ describe('LogReader', () => { }); }); - // TODO there is currently no initialization of log offset with - // mongodb backend, re-enable when implementing initial log offset - // fetching with mongodb backend. - it.skip('should start from offset 1 on log consumer readRecords error', done => { + // NOTE: there is currently no initialization of log offset with + // mongodb backend (i.e. it i hard-coded to 1): should be updated + // when implementing initial log offset fetching with mongodb backend. + it('should start from offset 1 on log consumer readRecords error', done => { const errorLogReader = new LogReader({ logId: 'test-log-reader', zkClient: zkMock.createClient('localhost:2181'), @@ -72,7 +72,7 @@ describe('LogReader', () => { }); }); - it('Should strip metadata v1 prefixes from object entries', done => { + it('should strip metadata v1 prefixes from object entries', done => { const mockExtension = { filter: sinon.spy(), }; diff --git a/tests/unit/lib/queuePopulator/kafkaLogConsumer/ListRecordStream.js b/tests/unit/lib/queuePopulator/kafkaLogConsumer/ListRecordStream.spec.js similarity index 100% rename from tests/unit/lib/queuePopulator/kafkaLogConsumer/ListRecordStream.js rename to tests/unit/lib/queuePopulator/kafkaLogConsumer/ListRecordStream.spec.js diff --git a/tests/unit/lib/queuePopulator/kafkaLogConsumer/LogConsumer.js b/tests/unit/lib/queuePopulator/kafkaLogConsumer/LogConsumer.spec.js similarity index 99% rename from tests/unit/lib/queuePopulator/kafkaLogConsumer/LogConsumer.js rename to tests/unit/lib/queuePopulator/kafkaLogConsumer/LogConsumer.spec.js index da690c3aa..b9ee570a3 100644 --- a/tests/unit/lib/queuePopulator/kafkaLogConsumer/LogConsumer.js +++ b/tests/unit/lib/queuePopulator/kafkaLogConsumer/LogConsumer.spec.js @@ -61,7 +61,7 @@ describe('LogConsumer', () => { assert.strictEqual(waitAssignementSpy.getCall(1).args.at(0), 2000); return done(); }); - }).timeout(5000); + }, 5000); }); describe('_storeCurrentOffsets', () => { diff --git a/tests/unit/lib/tasks/BackbeatTask.spec.js b/tests/unit/lib/tasks/BackbeatTask.spec.js index b2ac9bd5d..ad945c478 100644 --- a/tests/unit/lib/tasks/BackbeatTask.spec.js +++ b/tests/unit/lib/tasks/BackbeatTask.spec.js @@ -40,5 +40,5 @@ describe('BackbeatTask', () => { // the retry logic. setTimeout(done, 2000); }); - }).timeout(10000); + }, 10000); }); diff --git a/tests/unit/lib/util/LocationStatusManager.spec.js b/tests/unit/lib/util/LocationStatusManager.spec.js index c6284c7fd..10415d221 100644 --- a/tests/unit/lib/util/LocationStatusManager.spec.js +++ b/tests/unit/lib/util/LocationStatusManager.spec.js @@ -208,7 +208,7 @@ describe('LocationStatusManager', () => { }); describe('_setupLocationStatusStore', () => { - [ + it.each([ { case: 'should add missing locations', mongoLocations: [ @@ -246,25 +246,23 @@ describe('LocationStatusManager', () => { }, ], } - ].forEach(params => { - it(params.case, done => { - lsm._mongoClient = { - createCollection: sinon.stub().yields(), - collection: () => ({ - find: sinon.stub().yields(null, { - toArray: sinon.stub().yields(null, params.mongoLocations), - }), - insert: sinon.stub().yields(), - deleteMany: sinon.stub().yields(), + ])('$case', (params, done) => { + lsm._mongoClient = { + createCollection: sinon.stub().yields(), + collection: () => ({ + find: sinon.stub().yields(null, { + toArray: sinon.stub().yields(null, params.mongoLocations), }), - }; - lsm._setupLocationStatusStore(err => { - assert.ifError(err); - const finalLocations = Object.keys(lsm._locationStatusStore); - const validLocations = Object.keys(locationConfig); - assert.deepStrictEqual(finalLocations, validLocations); - return done(); - }); + insert: sinon.stub().yields(), + deleteMany: sinon.stub().yields(), + }), + }; + lsm._setupLocationStatusStore(err => { + assert.ifError(err); + const finalLocations = Object.keys(lsm._locationStatusStore); + const validLocations = Object.keys(locationConfig); + assert.deepStrictEqual(finalLocations, validLocations); + return done(); }); }); }); diff --git a/tests/unit/lib/util/mapLimitWaitPendingIfError.spec.js b/tests/unit/lib/util/mapLimitWaitPendingIfError.spec.js index 662b0b53b..9f3450e32 100644 --- a/tests/unit/lib/util/mapLimitWaitPendingIfError.spec.js +++ b/tests/unit/lib/util/mapLimitWaitPendingIfError.spec.js @@ -1,7 +1,7 @@ const assert = require('assert'); const mapLimitWaitPendingIfError = - require('../../../../lib/util/mapLimitWaitPendingIfError'); + require('../../../../lib/util/mapLimitWaitPendingIfError').default; describe('mapLimitWaitPendingIfError', () => { it('should process an empty array', done => { diff --git a/tests/unit/lib/util/metrics.spec.js b/tests/unit/lib/util/metrics.spec.js index cb9e90404..987e2b6ab 100644 --- a/tests/unit/lib/util/metrics.spec.js +++ b/tests/unit/lib/util/metrics.spec.js @@ -71,7 +71,7 @@ describe('Metrics', () => { { defaultLabel: 'default', label: 'value' }, 15); }); - it('can add default labels to wrapped histogram', () => { + it('can override default label to wrapped histogram', () => { const mockHistogram = sinon.spy(); mockHistogram.observe = sinon.spy(); const observeFn = wrapHistogramObserve(mockHistogram, { label: 'default' }); diff --git a/tests/unit/lib/wrappers/ChangeStream.js b/tests/unit/lib/wrappers/ChangeStream.spec.js similarity index 100% rename from tests/unit/lib/wrappers/ChangeStream.js rename to tests/unit/lib/wrappers/ChangeStream.spec.js diff --git a/tests/unit/lib/wrappers/kafkaConnectWrapper.js b/tests/unit/lib/wrappers/kafkaConnectWrapper.spec.js similarity index 100% rename from tests/unit/lib/wrappers/kafkaConnectWrapper.js rename to tests/unit/lib/wrappers/kafkaConnectWrapper.spec.js diff --git a/tests/unit/lifecycle/LifecycleConductor.spec.js b/tests/unit/lifecycle/LifecycleConductor.spec.js index 8f4b61c0e..1716c1fd2 100644 --- a/tests/unit/lifecycle/LifecycleConductor.spec.js +++ b/tests/unit/lifecycle/LifecycleConductor.spec.js @@ -45,9 +45,7 @@ describe('Lifecycle Conductor', () => { sinon.restore(); }); - describe('processBuckets', function test() { - // timeout set to 4000 to account for 2s for async ops + 1s for bucket queue completion check interval - this.timeout(4000); + describe('processBuckets', () => { // tests that `activeIndexingJobRetrieved` is not reset until the e it('should not reset `activeIndexingJobsRetrieved` while async operations are in progress', done => { const order = []; @@ -98,7 +96,7 @@ describe('Lifecycle Conductor', () => { done(); }); - }); + }, 4000); // timeout set to 4000 to account for 2s for async ops + 1s for bucket queue completion check interval }); describe('_indexesGetOrCreate', () => { @@ -135,40 +133,40 @@ describe('Lifecycle Conductor', () => { }); }); - const tests = [ - [ - 'should return v2', - [ + it.each([ + { + msg: 'should return v2', + input: [ [], // job state indexesForFeature.lifecycle.v2, // getIndex response null, // metadata proxy error true, // flag for status ofin progress job retrieval ], - [ + expected: [ [], // updated job state null, // expected putIndex object lifecycleTaskVersions.v2, // expected version ], - ], - [ - 'should return v1: missing indexes + put indexes', - [ + }, + { + msg: 'should return v1: missing indexes + put indexes', + input: [ [], [], null, true, ], - [ + expected: [ [ { bucket: 'testbucket', indexes: indexesForFeature.lifecycle.v2 }, ], indexesForFeature.lifecycle.v2, lifecycleTaskVersions.v1, ], - ], - [ - 'should return v1: missing indexes + skip put indexes when bucket is in job list', - [ + }, + { + msg: 'should return v1: missing indexes + skip put indexes when bucket is in job list', + input: [ [ { bucket: 'testbucket', indexes: indexesForFeature.lifecycle.v2 }, ], @@ -176,17 +174,17 @@ describe('Lifecycle Conductor', () => { null, true, ], - [ + expected: [ [ { bucket: 'testbucket', indexes: indexesForFeature.lifecycle.v2 }, ], null, lifecycleTaskVersions.v1, ], - ], - [ - 'should return v1: missing indexes + skip put indexes when job list is at limit', - [ + }, + { + msg: 'should return v1: missing indexes + skip put indexes when job list is at limit', + input: [ [ { bucket: 'testbucket2', indexes: indexesForFeature.lifecycle.v2 }, { bucket: 'testbucket3', indexes: indexesForFeature.lifecycle.v2 }, @@ -195,7 +193,7 @@ describe('Lifecycle Conductor', () => { null, true, ], - [ + expected: [ [ { bucket: 'testbucket2', indexes: indexesForFeature.lifecycle.v2 }, { bucket: 'testbucket3', indexes: indexesForFeature.lifecycle.v2 }, @@ -203,56 +201,53 @@ describe('Lifecycle Conductor', () => { null, lifecycleTaskVersions.v1, ], - ], - [ - 'should return v1: missing indexes + skip put indexes when in progress index request fails', - [ + }, + { + msg: 'should return v1: missing indexes + skip put indexes when in progress index request fails', + input: [ [], [], null, false, ], - [ + expected: [ [], null, lifecycleTaskVersions.v1, ], - ], - [ - 'should return v1: index request fails', - [ + }, + { + msg: 'should return v1: index request fails', + input: [ [], [], new Error('test error'), true, ], - [ + expected: [ [], null, lifecycleTaskVersions.v1, ], - ], - ]; - - tests.forEach(([msg, input, expected]) => - it(msg, done => { - const [inJobs, getIndexes, mockError, getInProgressSucceeded] = input; - const [expectedJobs, putIndexes, expectedVersion] = expected; - - const client = new BackbeatMetadataProxyMock(); - conductor.clientManager.getBackbeatMetadataProxy = () => client; - conductor.activeIndexingJobsRetrieved = getInProgressSucceeded; - conductor.activeIndexingJobs = inJobs; - client.indexesObj = getIndexes; - client.error = mockError; + }, + ])('$msg', ({ input, expected }, done) => { + const [inJobs, getIndexes, mockError, getInProgressSucceeded] = input; + const [expectedJobs, putIndexes, expectedVersion] = expected; + + const client = new BackbeatMetadataProxyMock(); + conductor.clientManager.getBackbeatMetadataProxy = () => client; + conductor.activeIndexingJobsRetrieved = getInProgressSucceeded; + conductor.activeIndexingJobs = inJobs; + client.indexesObj = getIndexes; + client.error = mockError; - conductor._indexesGetOrCreate(testTask, log, (err, taskVersion) => { - assert.ifError(err); - assert.deepStrictEqual(client.receivedIdxObj, putIndexes); - assert.deepStrictEqual(conductor.activeIndexingJobs, expectedJobs); - assert.deepStrictEqual(taskVersion, expectedVersion); - done(); - }); - })); + conductor._indexesGetOrCreate(testTask, log, (err, taskVersion) => { + assert.ifError(err); + assert.deepStrictEqual(client.receivedIdxObj, putIndexes); + assert.deepStrictEqual(conductor.activeIndexingJobs, expectedJobs); + assert.deepStrictEqual(taskVersion, expectedVersion); + done(); + }); + }); }); }); diff --git a/tests/unit/lifecycle/LifecycleOplogPopulatorUtils.spec.js b/tests/unit/lifecycle/LifecycleOplogPopulatorUtils.spec.js index 47a643667..a7d39d445 100644 --- a/tests/unit/lifecycle/LifecycleOplogPopulatorUtils.spec.js +++ b/tests/unit/lifecycle/LifecycleOplogPopulatorUtils.spec.js @@ -71,7 +71,7 @@ function buildLifecycleRule(actions, enabled = true) { describe('LifecycleOplogPopulatorUtils', () => { describe('isBucketExtensionEnabled', () => { - [ + it.each([ { it: 'should return false when there are only expiration rules', bucketMd: { @@ -206,11 +206,9 @@ describe('LifecycleOplogPopulatorUtils', () => { }, expectedReturn: true, }, - ].forEach(params => { - it(params.it, () => { - const enabled = LifecycleOplogPopulatorUtils.isBucketExtensionEnabled(params.bucketMd); - assert.strictEqual(enabled, params.expectedReturn); - }); + ])('$it', ({ bucketMd, expectedReturn }) => { + const enabled = LifecycleOplogPopulatorUtils.isBucketExtensionEnabled(bucketMd); + assert.strictEqual(enabled, expectedReturn); }); }); }); diff --git a/tests/unit/lifecycle/LifecycleQueuePopulator.spec.js b/tests/unit/lifecycle/LifecycleQueuePopulator.spec.js index 2394e81d2..86d3f0318 100644 --- a/tests/unit/lifecycle/LifecycleQueuePopulator.spec.js +++ b/tests/unit/lifecycle/LifecycleQueuePopulator.spec.js @@ -180,7 +180,7 @@ describe('LifecycleQueuePopulator', () => { lcqp = new LifecycleQueuePopulator(params); lcqp.locationConfigs = Object.assign({}, coldLocationConfigs, locationConfigs); }); - it('it should call _handleDeleteOp on delete message', () => { + it('should call _handleDeleteOp on delete message', () => { const handleDeleteStub = sinon.stub(lcqp, '_handleDeleteOp').returns(); lcqp.filter({ type: 'delete', @@ -230,7 +230,7 @@ describe('LifecycleQueuePopulator', () => { afterEach(() => { kafkaSendStub.reset(); }); - [ + it.each([ { it: 'should skip non dmf archived/restored objects', type: 'delete', @@ -316,31 +316,29 @@ describe('LifecycleQueuePopulator', () => { }, called: true, }, - ].forEach(params => { - it(params.it, () => { - const entry = { - type: params.type, - bucket: 'lc-queue-populator-test-bucket', - key: params.key, - value: JSON.stringify(params.md), - }; - lcqp._handleDeleteOp(entry); - assert.strictEqual(kafkaSendStub.calledOnce, params.called); - if (!params.called) { - return; - } - const message = JSON.parse(kafkaSendStub.args[0][0][0].message); - const expectedMessage = { - bucketName: 'lc-queue-populator-test-bucket', - objectKey: params.md.key, - archiveInfo: params.md.archive.archiveInfo, - requestId: message.requestId, - }; - if (params.md.versionId) { - expectedMessage.objectVersion = encode(params.md.versionId); - } - assert.deepStrictEqual(message, expectedMessage); - }); + ])('$it', params => { + const entry = { + type: params.type, + bucket: 'lc-queue-populator-test-bucket', + key: params.key, + value: JSON.stringify(params.md), + }; + lcqp._handleDeleteOp(entry); + assert.strictEqual(kafkaSendStub.calledOnce, params.called); + if (!params.called) { + return; + } + const message = JSON.parse(kafkaSendStub.args[0][0][0].message); + const expectedMessage = { + bucketName: 'lc-queue-populator-test-bucket', + objectKey: params.md.key, + archiveInfo: params.md.archive.archiveInfo, + requestId: message.requestId, + }; + if (params.md.versionId) { + expectedMessage.objectVersion = encode(params.md.versionId); + } + assert.deepStrictEqual(message, expectedMessage); }); }); }); diff --git a/tests/unit/lifecycle/LifecycleTask.spec.js b/tests/unit/lifecycle/LifecycleTask.spec.js index 2ca3ac5e0..d265860fc 100644 --- a/tests/unit/lifecycle/LifecycleTask.spec.js +++ b/tests/unit/lifecycle/LifecycleTask.spec.js @@ -96,7 +96,7 @@ const lp = { describe('lifecycle task helper methods', () => { let lct; - before(() => { + beforeAll(() => { lct = new LifecycleTask(lp); lct.setSupportedRules([ 'expiration', @@ -495,7 +495,7 @@ describe('lifecycle task helper methods', () => { }, ]; - before(() => { + beforeAll(() => { // overwrite _sendObjectAction to read entry sent class LifecycleTaskMock extends LifecycleTask { _sendObjectAction(entry, cb) { @@ -1517,7 +1517,7 @@ describe('lifecycle task helper methods', () => { }, ]; - before(() => { + beforeAll(() => { class LifecycleTaskMock extends LifecycleTask { _sendObjectAction(entry, cb) { this.latestEntry = entry; @@ -1665,7 +1665,7 @@ describe('lifecycle task helper methods', () => { }, ]; - before(() => { + beforeAll(() => { class LifecycleTaskMock extends LifecycleTask { _sendObjectAction(entry, cb) { this.latestEntry = entry; @@ -1710,7 +1710,7 @@ describe('lifecycle task helper methods', () => { describe('_ncvHeapObjectsClear', () => { let lct2; - before(() => { + beforeAll(() => { class LifecycleTaskMock extends LifecycleTask { _sendObjectAction(entry, cb) { this.latestEntry = entry; @@ -1794,7 +1794,7 @@ describe('lifecycle task helper methods', () => { describe('_ncvHeapBucketClear', () => { let lct2; - before(() => { + beforeAll(() => { class LifecycleTaskMock extends LifecycleTask { _sendObjectAction(entry, cb) { this.latestEntry = entry; @@ -1879,7 +1879,7 @@ describe('lifecycle task helper methods', () => { details: {}, }; - before(() => { + beforeAll(() => { class LifecycleTaskMock extends LifecycleTask { constructor(lp) { super(lp); diff --git a/tests/unit/lifecycle/LifecycleUpdateExpirationTask.spec.js b/tests/unit/lifecycle/LifecycleUpdateExpirationTask.spec.js index 25f4a2a5b..e1559bc80 100644 --- a/tests/unit/lifecycle/LifecycleUpdateExpirationTask.spec.js +++ b/tests/unit/lifecycle/LifecycleUpdateExpirationTask.spec.js @@ -124,4 +124,28 @@ describe('LifecycleUpdateExpirationTask', () => { done(); }); }); + + it('should throw an error when location is not specified', done => { + const mdObj = new ObjectMD(); + backbeatMetadataProxyClient.setMdObj(mdObj); + const invalidActionEntry = ActionQueueEntry.create('deleteObject') + .setAttribute('target', { + owner: 'eab6642741045d0ae7cb3333962ad56f847ce0d9bb73de98eb4959428fc28108', + bucket: 'somebucket', + key: 'somekey', + accountId: '871467171849', + }) + .setAttribute('details', { + lastModified: '2023-06-02T12:50:57.016Z', + }) + .addContext('details', { + origin: 'lifecycle', + ruleType: 'expiration', + reqId: '8b902aef7346801d99fc', + }); + task.processActionEntry(invalidActionEntry, err => { + assert(err.is.MissingParameter); + done(); + }); + }); }); diff --git a/tests/unit/management/patchConfiguration.js b/tests/unit/management/patchConfiguration.spec.js similarity index 99% rename from tests/unit/management/patchConfiguration.js rename to tests/unit/management/patchConfiguration.spec.js index b396ddf6c..1d05e89dd 100644 --- a/tests/unit/management/patchConfiguration.js +++ b/tests/unit/management/patchConfiguration.spec.js @@ -85,7 +85,7 @@ describe('patchConfiguration', () => { const bucket3 = createBucketMDObject('bucket-3', 'location-2', { status: 'enabled' }); - before(done => { + beforeAll(done => { async.waterfall([ next => { this.md = new Metadata('mongodb', { mongodb: mongoConfig }, @@ -113,7 +113,7 @@ describe('patchConfiguration', () => { Config.setIngestionBuckets({}, []); }); - after(() => { + afterAll(() => { const client = this.md.client; client.db.dropDatabase(); }); diff --git a/tests/unit/notification/NotificationConfigManager.js b/tests/unit/notification/NotificationConfigManager.spec.js similarity index 100% rename from tests/unit/notification/NotificationConfigManager.js rename to tests/unit/notification/NotificationConfigManager.spec.js diff --git a/tests/unit/notification/NotificationQueuePopulator.js b/tests/unit/notification/NotificationQueuePopulator.spec.js similarity index 100% rename from tests/unit/notification/NotificationQueuePopulator.js rename to tests/unit/notification/NotificationQueuePopulator.spec.js diff --git a/tests/unit/notification/NotificationQueueProcessor.js b/tests/unit/notification/NotificationQueueProcessor.spec.js similarity index 98% rename from tests/unit/notification/NotificationQueueProcessor.js rename to tests/unit/notification/NotificationQueueProcessor.spec.js index 4645280bd..a41f3e3cb 100644 --- a/tests/unit/notification/NotificationQueueProcessor.js +++ b/tests/unit/notification/NotificationQueueProcessor.spec.js @@ -76,7 +76,7 @@ const sentEntry = JSON.stringify({ ], }); -describe('NotificationQueueProcessor:: ', () => { +describe('NotificationQueueProcessor::', () => { let notificationQueueProcessor; beforeEach(() => { diff --git a/tests/unit/notification/QueueProcessor.spec.js b/tests/unit/notification/QueueProcessor.spec.js index 7f11bb534..c1ae61363 100644 --- a/tests/unit/notification/QueueProcessor.spec.js +++ b/tests/unit/notification/QueueProcessor.spec.js @@ -8,7 +8,7 @@ const QueueProcessor = require( describe('notification QueueProcessor', () => { let qp; - before(done => { + beforeAll(done => { qp = new QueueProcessor(null, null, { destinations: [ { @@ -72,10 +72,7 @@ describe('notification QueueProcessor', () => { // also test the resilience of the producer, but at the time this // test was written Bucket Notification tests are not yet running // in Integration (refs: BB-419, INTGR-895) - it('should process 10K notifications with 1K concurrency to send to external destination', - function test10KNotif(done) { - this.timeout(5000); - + it('should process 10K notifications with 1K concurrency to send to external destination', done => { const origLogger = qp.logger; qp.logger = { debug: () => {}, @@ -112,7 +109,7 @@ describe('notification QueueProcessor', () => { clearInterval(flushDeliveryReportsInterval); done(); }); - }); + }, 5000); it('should send notification to external destination with delivery error', done => { const send = sinon.stub().callsFake((messages, cb) => cb(new Error('delivery error'))); diff --git a/tests/unit/notification/utils/config.js b/tests/unit/notification/utils/config.spec.js similarity index 99% rename from tests/unit/notification/utils/config.js rename to tests/unit/notification/utils/config.spec.js index b97d9ddcb..12ca51928 100644 --- a/tests/unit/notification/utils/config.js +++ b/tests/unit/notification/utils/config.spec.js @@ -305,7 +305,7 @@ describe('Notification configuration util', () => { }; } - before(() => { + beforeAll(() => { configMap = notifConfUtil.configArrayToMap(testConfigs); }); diff --git a/tests/unit/notification/utils/message.js b/tests/unit/notification/utils/message.spec.js similarity index 100% rename from tests/unit/notification/utils/message.js rename to tests/unit/notification/utils/message.spec.js diff --git a/tests/unit/oplogPopulator/Allocator.js b/tests/unit/oplogPopulator/Allocator.spec.js similarity index 100% rename from tests/unit/oplogPopulator/Allocator.js rename to tests/unit/oplogPopulator/Allocator.spec.js diff --git a/tests/unit/oplogPopulator/Connector.js b/tests/unit/oplogPopulator/Connector.spec.js similarity index 100% rename from tests/unit/oplogPopulator/Connector.js rename to tests/unit/oplogPopulator/Connector.spec.js diff --git a/tests/unit/oplogPopulator/ConnectorsManager.js b/tests/unit/oplogPopulator/ConnectorsManager.spec.js similarity index 100% rename from tests/unit/oplogPopulator/ConnectorsManager.js rename to tests/unit/oplogPopulator/ConnectorsManager.spec.js diff --git a/tests/unit/oplogPopulator/allocationStrategy/LeastFullConnector.js b/tests/unit/oplogPopulator/allocationStrategy/LeastFullConnector.spec.js similarity index 100% rename from tests/unit/oplogPopulator/allocationStrategy/LeastFullConnector.js rename to tests/unit/oplogPopulator/allocationStrategy/LeastFullConnector.spec.js diff --git a/tests/unit/oplogPopulator/oplogPopulator.js b/tests/unit/oplogPopulator/oplogPopulator.spec.js similarity index 100% rename from tests/unit/oplogPopulator/oplogPopulator.js rename to tests/unit/oplogPopulator/oplogPopulator.spec.js diff --git a/tests/unit/replication/MultipleBackendTask.js b/tests/unit/replication/MultipleBackendTask.spec.js similarity index 98% rename from tests/unit/replication/MultipleBackendTask.js rename to tests/unit/replication/MultipleBackendTask.spec.js index 09b199ad7..17e172c9f 100644 --- a/tests/unit/replication/MultipleBackendTask.js +++ b/tests/unit/replication/MultipleBackendTask.spec.js @@ -16,8 +16,7 @@ const MAX_AWS_PART_SIZE = (1024 * 1024 * 1024) * 5; // 5GB const MAX_AWS_OBJECT_SIZE = (1024 * 1024 * 1024 * 1024) * 5; // 5TB const retryConfig = { scality: { timeoutS: 300 } }; -describe('MultipleBackendTask', function test() { - this.timeout(5000); +describe('MultipleBackendTask', () => { let task; function checkPartLength(contentLength, expectedPartSize) { @@ -81,7 +80,7 @@ describe('MultipleBackendTask', function test() { }); describe('::initiateMultipartUpload', () => { - it('should use exponential backoff if retryable error ', done => { + it('should use exponential backoff if retryable error', done => { const doneOnce = jsutil.once(done); setTimeout(() => { // inhibits further retries @@ -91,7 +90,7 @@ describe('MultipleBackendTask', function test() { requestInitiateMPU({ retryable: true }, doneOnce); }); - it('should not use exponential backoff if non-retryable error ', done => + it('should not use exponential backoff if non-retryable error', done => requestInitiateMPU({ retryable: false }, err => { assert(err); done(); @@ -225,8 +224,7 @@ describe('MultipleBackendTask', function test() { }); it('should ensure all parts of the original object are intact', - function test() { - this.timeout(10000); + () => { const minMPUObjectSize = MIN_AWS_PART_SIZE + 1; const contentLengths = [MAX_AWS_OBJECT_SIZE]; Array.from(Array(1024).keys()).forEach(n => { @@ -253,7 +251,7 @@ describe('MultipleBackendTask', function test() { } assert(sum === contentLength); }); - }); + }, 10000); it('should get single part count for GCP', () => { const contentLength = (1024 * 1024) * 5; @@ -438,4 +436,4 @@ describe('MultipleBackendTask', function test() { }); }); }); -}); +}, 5000); diff --git a/tests/unit/replication/QueueProcessor.spec.js b/tests/unit/replication/QueueProcessor.spec.js index 865ed03a4..453ebdf45 100644 --- a/tests/unit/replication/QueueProcessor.spec.js +++ b/tests/unit/replication/QueueProcessor.spec.js @@ -48,6 +48,10 @@ describe('Queue Processor', () => { ); }); + afterEach(() => { + qp._redis.quit(); + }); + describe('handle liveness', () => { let mockLog; beforeEach(() => { diff --git a/tests/unit/replication/ReplicationAPI.spec.js b/tests/unit/replication/ReplicationAPI.spec.js index c93b3e21a..daff21e3f 100644 --- a/tests/unit/replication/ReplicationAPI.spec.js +++ b/tests/unit/replication/ReplicationAPI.spec.js @@ -33,7 +33,7 @@ describe('ReplicationAPI', () => { messages = []; }); - describe('::sendDataMoverAction ', () => { + describe('::sendDataMoverAction', () => { it('should publish to archive topic', done => { const action = ActionQueueEntry.create('copyLocation'); action diff --git a/tests/unit/replication/ReplicationMetrics.js b/tests/unit/replication/ReplicationMetrics.spec.js similarity index 100% rename from tests/unit/replication/ReplicationMetrics.js rename to tests/unit/replication/ReplicationMetrics.spec.js diff --git a/tests/unit/replication/ReplicationStatusProcessor.js b/tests/unit/replication/ReplicationStatusProcessor.spec.js similarity index 100% rename from tests/unit/replication/ReplicationStatusProcessor.js rename to tests/unit/replication/ReplicationStatusProcessor.spec.js diff --git a/tests/unit/replication/UpdateReplicationStatus.js b/tests/unit/replication/UpdateReplicationStatus.spec.js similarity index 100% rename from tests/unit/replication/UpdateReplicationStatus.js rename to tests/unit/replication/UpdateReplicationStatus.spec.js diff --git a/tests/unit/replication/getLocationsFromStorageClass.js b/tests/unit/replication/getLocationsFromStorageClass.spec.js similarity index 100% rename from tests/unit/replication/getLocationsFromStorageClass.js rename to tests/unit/replication/getLocationsFromStorageClass.spec.js diff --git a/tests/unit/sortedSetHelper.js b/tests/unit/sortedSetHelper.spec.js similarity index 100% rename from tests/unit/sortedSetHelper.js rename to tests/unit/sortedSetHelper.spec.js diff --git a/tests/unit/utils/MongoUtils.js b/tests/unit/utils/MongoUtils.spec.js similarity index 100% rename from tests/unit/utils/MongoUtils.js rename to tests/unit/utils/MongoUtils.spec.js diff --git a/.github/dockerfiles/syntheticbucketd/Dockerfile b/tests/utils/syntheticbucketd/Dockerfile similarity index 89% rename from .github/dockerfiles/syntheticbucketd/Dockerfile rename to tests/utils/syntheticbucketd/Dockerfile index 0621daf9d..0893a1be4 100644 --- a/.github/dockerfiles/syntheticbucketd/Dockerfile +++ b/tests/utils/syntheticbucketd/Dockerfile @@ -5,6 +5,6 @@ COPY package.json /app COPY yarn.lock /app RUN cd /app && yarn install --network-concurrency 1 -COPY tests/utils /app +COPY * /app CMD ["node", "/app/syntheticbucketd.js", "200000", "1000"] diff --git a/tests/utils/syntheticbucketd/package.json b/tests/utils/syntheticbucketd/package.json new file mode 100644 index 000000000..59fb38e9e --- /dev/null +++ b/tests/utils/syntheticbucketd/package.json @@ -0,0 +1,9 @@ +{ + "name": "syntheticbucketd", + "version": "1.0.0", + "main": "/app/syntheticbucketd.js", + "license": "MIT", + "dependencies": { + "werelogs": "scality/werelogs#8.1.0" + } +} diff --git a/tests/utils/syntheticbucketd.js b/tests/utils/syntheticbucketd/syntheticbucketd.js similarity index 100% rename from tests/utils/syntheticbucketd.js rename to tests/utils/syntheticbucketd/syntheticbucketd.js diff --git a/tests/utils/syntheticbucketd/yarn.lock b/tests/utils/syntheticbucketd/yarn.lock new file mode 100644 index 000000000..c90a72e8a --- /dev/null +++ b/tests/utils/syntheticbucketd/yarn.lock @@ -0,0 +1,14 @@ +# THIS IS AN AUTOGENERATED FILE. DO NOT EDIT THIS FILE DIRECTLY. +# yarn lockfile v1 + + +safe-json-stringify@1.0.3: + version "1.0.3" + resolved "https://registry.yarnpkg.com/safe-json-stringify/-/safe-json-stringify-1.0.3.tgz#3cb6717660a086d07cb5bd9b7a6875bcf67bd05e" + integrity sha512-FspOTGvddR3ZE2k+NqHsywbCrnD7xow+r156P04MEMURbRF65Pdccpl2f4XU3KjXKsK5zdF03ScdKoL7Ti4Gpg== + +werelogs@scality/werelogs#8.1.0: + version "8.1.0" + resolved "https://codeload.github.com/scality/werelogs/tar.gz/e8f828725642c54c511cdbe580b18f43d3589313" + dependencies: + safe-json-stringify "1.0.3" diff --git a/tsconfig.json b/tsconfig.json new file mode 100644 index 000000000..9e0e37890 --- /dev/null +++ b/tsconfig.json @@ -0,0 +1,37 @@ +{ + "compilerOptions": { + /* Visit https://aka.ms/tsconfig to read more about this file */ + "incremental": true, /* Save .tsbuildinfo files to allow for incremental compilation of projects. */ + "module": "commonjs", /* Specify what module code is generated. */ + "rootDir": "./", /* Specify the root folder within your source files. */ + "moduleResolution": "node", /* Specify how TypeScript looks up a file from a given module specifier. */ + "declaration": true, /* Generate .d.ts files from TypeScript and JavaScript files in your project. */ + "sourceMap": true, /* Create source map files for emitted JavaScript files. */ + "outDir": "./dist", /* Specify an output folder for all emitted files. */ + "esModuleInterop": true, /* Emit additional JavaScript to ease support for importing CommonJS modules. This enables 'allowSyntheticDefaultImports' for type compatibility. */ + "forceConsistentCasingInFileNames": true, /* Ensure that casing is correct in imports. */ + "strict": true, /* Enable all strict type-checking options. */ + "noImplicitAny": false, /* Enable error reporting for expressions and declarations with an implied 'any' type. */ + "skipLibCheck": true, /* Skip type checking all .d.ts files. */ + "lib": ["es2021"], /* JS APIs and definitions to include */ + "target": "es2021", /* Specify JS target revision */ + "resolveJsonModule": true, /* Allows importing modules with a .json extension */ + "allowJs": true, /* Allow JavaScript files to be imported inside your project */ + "checkJs": false, /* Enable error reporting for JavaScript files */ + "noEmitOnError": false, /* Do not emit compiler output files like JavaScript source code, source-maps or declarations if any errors were reported. */ + }, + "include": [ + "./bin", + "./conf", + "./extensions", + "./index.js", + "./lib", + "./policies", + "./tests/**/*.json" + ], + "exclude": [ + "node_modules/*" + ], + "compileOnSave": true, + "extends": "@tsconfig/node16/tsconfig.json" +} diff --git a/yarn.lock b/yarn.lock index 53881acab..77d6ae14b 100644 --- a/yarn.lock +++ b/yarn.lock @@ -2,6 +2,14 @@ # yarn lockfile v1 +"@ampproject/remapping@^2.2.0": + version "2.2.1" + resolved "https://registry.yarnpkg.com/@ampproject/remapping/-/remapping-2.2.1.tgz#99e8e11851128b8702cd57c33684f1d0f260b630" + integrity sha512-lFMjJTrFL3j7L9yBxwYfCq2k6qqwHyzuUl/XBnif78PWTJYyL/dfowQHWE3sp6U6ZzqWiiIZnpTMO96zhkjwtg== + dependencies: + "@jridgewell/gen-mapping" "^0.3.0" + "@jridgewell/trace-mapping" "^0.3.9" + "@azure/abort-controller@^1.0.0": version "1.1.0" resolved "https://registry.yarnpkg.com/@azure/abort-controller/-/abort-controller-1.1.0.tgz#788ee78457a55af8a1ad342acb182383d2119249" @@ -170,11 +178,326 @@ events "^3.0.0" tslib "^2.2.0" +"@babel/code-frame@^7.0.0": + version "7.18.6" + resolved "https://registry.yarnpkg.com/@babel/code-frame/-/code-frame-7.18.6.tgz#3b25d38c89600baa2dcc219edfa88a74eb2c427a" + integrity sha512-TDCmlK5eOvH+eH7cdAFlNXeVJqWIQ7gW9tY1GJIpUtFb6CmjVyq2VM3u71bOyR8CRihcCgMUYoDNyLXao3+70Q== + dependencies: + "@babel/highlight" "^7.18.6" + +"@babel/code-frame@^7.12.13", "@babel/code-frame@^7.22.13": + version "7.22.13" + resolved "https://registry.yarnpkg.com/@babel/code-frame/-/code-frame-7.22.13.tgz#e3c1c099402598483b7a8c46a721d1038803755e" + integrity sha512-XktuhWlJ5g+3TJXc5upd9Ks1HutSArik6jf2eAjYFyIOf4ej3RN+184cZbzDvbPnuTJIUhPKKJE3cIsYTiAT3w== + dependencies: + "@babel/highlight" "^7.22.13" + chalk "^2.4.2" + +"@babel/compat-data@^7.22.9": + version "7.23.2" + resolved "https://registry.yarnpkg.com/@babel/compat-data/-/compat-data-7.23.2.tgz#6a12ced93455827037bfb5ed8492820d60fc32cc" + integrity sha512-0S9TQMmDHlqAZ2ITT95irXKfxN9bncq8ZCoJhun3nHL/lLUxd2NKBJYoNGWH7S0hz6fRQwWlAWn/ILM0C70KZQ== + +"@babel/core@^7.11.6", "@babel/core@^7.12.3": + version "7.23.2" + resolved "https://registry.yarnpkg.com/@babel/core/-/core-7.23.2.tgz#ed10df0d580fff67c5f3ee70fd22e2e4c90a9f94" + integrity sha512-n7s51eWdaWZ3vGT2tD4T7J6eJs3QoBXydv7vkUM06Bf1cbVD2Kc2UrkzhiQwobfV7NwOnQXYL7UBJ5VPU+RGoQ== + dependencies: + "@ampproject/remapping" "^2.2.0" + "@babel/code-frame" "^7.22.13" + "@babel/generator" "^7.23.0" + "@babel/helper-compilation-targets" "^7.22.15" + "@babel/helper-module-transforms" "^7.23.0" + "@babel/helpers" "^7.23.2" + "@babel/parser" "^7.23.0" + "@babel/template" "^7.22.15" + "@babel/traverse" "^7.23.2" + "@babel/types" "^7.23.0" + convert-source-map "^2.0.0" + debug "^4.1.0" + gensync "^1.0.0-beta.2" + json5 "^2.2.3" + semver "^6.3.1" + +"@babel/generator@^7.23.0", "@babel/generator@^7.7.2": + version "7.23.0" + resolved "https://registry.yarnpkg.com/@babel/generator/-/generator-7.23.0.tgz#df5c386e2218be505b34837acbcb874d7a983420" + integrity sha512-lN85QRR+5IbYrMWM6Y4pE/noaQtg4pNiqeNGX60eqOfo6gtEj6uw/JagelB8vVztSd7R6M5n1+PQkDbHbBRU4g== + dependencies: + "@babel/types" "^7.23.0" + "@jridgewell/gen-mapping" "^0.3.2" + "@jridgewell/trace-mapping" "^0.3.17" + jsesc "^2.5.1" + +"@babel/helper-compilation-targets@^7.22.15": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/helper-compilation-targets/-/helper-compilation-targets-7.22.15.tgz#0698fc44551a26cf29f18d4662d5bf545a6cfc52" + integrity sha512-y6EEzULok0Qvz8yyLkCvVX+02ic+By2UdOhylwUOvOn9dvYc9mKICJuuU1n1XBI02YWsNsnrY1kc6DVbjcXbtw== + dependencies: + "@babel/compat-data" "^7.22.9" + "@babel/helper-validator-option" "^7.22.15" + browserslist "^4.21.9" + lru-cache "^5.1.1" + semver "^6.3.1" + +"@babel/helper-environment-visitor@^7.22.20": + version "7.22.20" + resolved "https://registry.yarnpkg.com/@babel/helper-environment-visitor/-/helper-environment-visitor-7.22.20.tgz#96159db61d34a29dba454c959f5ae4a649ba9167" + integrity sha512-zfedSIzFhat/gFhWfHtgWvlec0nqB9YEIVrpuwjruLlXfUSnA8cJB0miHKwqDnQ7d32aKo2xt88/xZptwxbfhA== + +"@babel/helper-function-name@^7.23.0": + version "7.23.0" + resolved "https://registry.yarnpkg.com/@babel/helper-function-name/-/helper-function-name-7.23.0.tgz#1f9a3cdbd5b2698a670c30d2735f9af95ed52759" + integrity sha512-OErEqsrxjZTJciZ4Oo+eoZqeW9UIiOcuYKRJA4ZAgV9myA+pOXhhmpfNCKjEH/auVfEYVFJ6y1Tc4r0eIApqiw== + dependencies: + "@babel/template" "^7.22.15" + "@babel/types" "^7.23.0" + +"@babel/helper-hoist-variables@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/helper-hoist-variables/-/helper-hoist-variables-7.22.5.tgz#c01a007dac05c085914e8fb652b339db50d823bb" + integrity sha512-wGjk9QZVzvknA6yKIUURb8zY3grXCcOZt+/7Wcy8O2uctxhplmUPkOdlgoNhmdVee2c92JXbf1xpMtVNbfoxRw== + dependencies: + "@babel/types" "^7.22.5" + +"@babel/helper-module-imports@^7.22.15": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/helper-module-imports/-/helper-module-imports-7.22.15.tgz#16146307acdc40cc00c3b2c647713076464bdbf0" + integrity sha512-0pYVBnDKZO2fnSPCrgM/6WMc7eS20Fbok+0r88fp+YtWVLZrp4CkafFGIp+W0VKw4a22sgebPT99y+FDNMdP4w== + dependencies: + "@babel/types" "^7.22.15" + +"@babel/helper-module-transforms@^7.23.0": + version "7.23.0" + resolved "https://registry.yarnpkg.com/@babel/helper-module-transforms/-/helper-module-transforms-7.23.0.tgz#3ec246457f6c842c0aee62a01f60739906f7047e" + integrity sha512-WhDWw1tdrlT0gMgUJSlX0IQvoO1eN279zrAUbVB+KpV2c3Tylz8+GnKOLllCS6Z/iZQEyVYxhZVUdPTqs2YYPw== + dependencies: + "@babel/helper-environment-visitor" "^7.22.20" + "@babel/helper-module-imports" "^7.22.15" + "@babel/helper-simple-access" "^7.22.5" + "@babel/helper-split-export-declaration" "^7.22.6" + "@babel/helper-validator-identifier" "^7.22.20" + +"@babel/helper-plugin-utils@^7.0.0", "@babel/helper-plugin-utils@^7.10.4", "@babel/helper-plugin-utils@^7.12.13", "@babel/helper-plugin-utils@^7.14.5", "@babel/helper-plugin-utils@^7.22.5", "@babel/helper-plugin-utils@^7.8.0": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/helper-plugin-utils/-/helper-plugin-utils-7.22.5.tgz#dd7ee3735e8a313b9f7b05a773d892e88e6d7295" + integrity sha512-uLls06UVKgFG9QD4OeFYLEGteMIAa5kpTPcFL28yuCIIzsf6ZyKZMllKVOCZFhiZ5ptnwX4mtKdWCBE/uT4amg== + +"@babel/helper-simple-access@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/helper-simple-access/-/helper-simple-access-7.22.5.tgz#4938357dc7d782b80ed6dbb03a0fba3d22b1d5de" + integrity sha512-n0H99E/K+Bika3++WNL17POvo4rKWZ7lZEp1Q+fStVbUi8nxPQEBOlTmCOxW/0JsS56SKKQ+ojAe2pHKJHN35w== + dependencies: + "@babel/types" "^7.22.5" + +"@babel/helper-split-export-declaration@^7.22.6": + version "7.22.6" + resolved "https://registry.yarnpkg.com/@babel/helper-split-export-declaration/-/helper-split-export-declaration-7.22.6.tgz#322c61b7310c0997fe4c323955667f18fcefb91c" + integrity sha512-AsUnxuLhRYsisFiaJwvp1QF+I3KjD5FOxut14q/GzovUe6orHLesW2C7d754kRm53h5gqrz6sFl6sxc4BVtE/g== + dependencies: + "@babel/types" "^7.22.5" + +"@babel/helper-string-parser@^7.22.5": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/helper-string-parser/-/helper-string-parser-7.22.5.tgz#533f36457a25814cf1df6488523ad547d784a99f" + integrity sha512-mM4COjgZox8U+JcXQwPijIZLElkgEpO5rsERVDJTc2qfCDfERyob6k5WegS14SX18IIjv+XD+GrqNumY5JRCDw== + +"@babel/helper-validator-identifier@^7.18.6": + version "7.18.6" + resolved "https://registry.yarnpkg.com/@babel/helper-validator-identifier/-/helper-validator-identifier-7.18.6.tgz#9c97e30d31b2b8c72a1d08984f2ca9b574d7a076" + integrity sha512-MmetCkz9ej86nJQV+sFCxoGGrUbU3q02kgLciwkrt9QqEB7cP39oKEY0PakknEO0Gu20SskMRi+AYZ3b1TpN9g== + +"@babel/helper-validator-identifier@^7.22.20": + version "7.22.20" + resolved "https://registry.yarnpkg.com/@babel/helper-validator-identifier/-/helper-validator-identifier-7.22.20.tgz#c4ae002c61d2879e724581d96665583dbc1dc0e0" + integrity sha512-Y4OZ+ytlatR8AI+8KZfKuL5urKp7qey08ha31L8b3BwewJAoJamTzyvxPR/5D+KkdJCGPq/+8TukHBlY10FX9A== + +"@babel/helper-validator-option@^7.22.15": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/helper-validator-option/-/helper-validator-option-7.22.15.tgz#694c30dfa1d09a6534cdfcafbe56789d36aba040" + integrity sha512-bMn7RmyFjY/mdECUbgn9eoSY4vqvacUnS9i9vGAGttgFWesO6B4CYWA7XlpbWgBt71iv/hfbPlynohStqnu5hA== + +"@babel/helpers@^7.23.2": + version "7.23.2" + resolved "https://registry.yarnpkg.com/@babel/helpers/-/helpers-7.23.2.tgz#2832549a6e37d484286e15ba36a5330483cac767" + integrity sha512-lzchcp8SjTSVe/fPmLwtWVBFC7+Tbn8LGHDVfDp9JGxpAY5opSaEFgt8UQvrnECWOTdji2mOWMz1rOhkHscmGQ== + dependencies: + "@babel/template" "^7.22.15" + "@babel/traverse" "^7.23.2" + "@babel/types" "^7.23.0" + +"@babel/highlight@^7.18.6": + version "7.18.6" + resolved "https://registry.yarnpkg.com/@babel/highlight/-/highlight-7.18.6.tgz#81158601e93e2563795adcbfbdf5d64be3f2ecdf" + integrity sha512-u7stbOuYjaPezCuLj29hNW1v64M2Md2qupEKP1fHc7WdOA3DgLh37suiSrZYY7haUB7iBeQZ9P1uiRF359do3g== + dependencies: + "@babel/helper-validator-identifier" "^7.18.6" + chalk "^2.0.0" + js-tokens "^4.0.0" + +"@babel/highlight@^7.22.13": + version "7.22.20" + resolved "https://registry.yarnpkg.com/@babel/highlight/-/highlight-7.22.20.tgz#4ca92b71d80554b01427815e06f2df965b9c1f54" + integrity sha512-dkdMCN3py0+ksCgYmGG8jKeGA/8Tk+gJwSYYlFGxG5lmhfKNoAy004YpLxpS1W2J8m/EK2Ew+yOs9pVRwO89mg== + dependencies: + "@babel/helper-validator-identifier" "^7.22.20" + chalk "^2.4.2" + js-tokens "^4.0.0" + +"@babel/parser@^7.1.0", "@babel/parser@^7.14.7", "@babel/parser@^7.20.7", "@babel/parser@^7.22.15", "@babel/parser@^7.23.0": + version "7.23.0" + resolved "https://registry.yarnpkg.com/@babel/parser/-/parser-7.23.0.tgz#da950e622420bf96ca0d0f2909cdddac3acd8719" + integrity sha512-vvPKKdMemU85V9WE/l5wZEmImpCtLqbnTvqDS2U1fJ96KrxoW7KrXhNsNCblQlg8Ck4b85yxdTyelsMUgFUXiw== + +"@babel/plugin-syntax-async-generators@^7.8.4": + version "7.8.4" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-async-generators/-/plugin-syntax-async-generators-7.8.4.tgz#a983fb1aeb2ec3f6ed042a210f640e90e786fe0d" + integrity sha512-tycmZxkGfZaxhMRbXlPXuVFpdWlXpir2W4AMhSJgRKzk/eDlIXOhb2LHWoLpDF7TEHylV5zNhykX6KAgHJmTNw== + dependencies: + "@babel/helper-plugin-utils" "^7.8.0" + +"@babel/plugin-syntax-bigint@^7.8.3": + version "7.8.3" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-bigint/-/plugin-syntax-bigint-7.8.3.tgz#4c9a6f669f5d0cdf1b90a1671e9a146be5300cea" + integrity sha512-wnTnFlG+YxQm3vDxpGE57Pj0srRU4sHE/mDkt1qv2YJJSeUAec2ma4WLUnUPeKjyrfntVwe/N6dCXpU+zL3Npg== + dependencies: + "@babel/helper-plugin-utils" "^7.8.0" + +"@babel/plugin-syntax-class-properties@^7.8.3": + version "7.12.13" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-class-properties/-/plugin-syntax-class-properties-7.12.13.tgz#b5c987274c4a3a82b89714796931a6b53544ae10" + integrity sha512-fm4idjKla0YahUNgFNLCB0qySdsoPiZP3iQE3rky0mBUtMZ23yDJ9SJdg6dXTSDnulOVqiF3Hgr9nbXvXTQZYA== + dependencies: + "@babel/helper-plugin-utils" "^7.12.13" + +"@babel/plugin-syntax-import-meta@^7.8.3": + version "7.10.4" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-import-meta/-/plugin-syntax-import-meta-7.10.4.tgz#ee601348c370fa334d2207be158777496521fd51" + integrity sha512-Yqfm+XDx0+Prh3VSeEQCPU81yC+JWZ2pDPFSS4ZdpfZhp4MkFMaDC1UqseovEKwSUpnIL7+vK+Clp7bfh0iD7g== + dependencies: + "@babel/helper-plugin-utils" "^7.10.4" + +"@babel/plugin-syntax-json-strings@^7.8.3": + version "7.8.3" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-json-strings/-/plugin-syntax-json-strings-7.8.3.tgz#01ca21b668cd8218c9e640cb6dd88c5412b2c96a" + integrity sha512-lY6kdGpWHvjoe2vk4WrAapEuBR69EMxZl+RoGRhrFGNYVK8mOPAW8VfbT/ZgrFbXlDNiiaxQnAtgVCZ6jv30EA== + dependencies: + "@babel/helper-plugin-utils" "^7.8.0" + +"@babel/plugin-syntax-jsx@^7.7.2": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-jsx/-/plugin-syntax-jsx-7.22.5.tgz#a6b68e84fb76e759fc3b93e901876ffabbe1d918" + integrity sha512-gvyP4hZrgrs/wWMaocvxZ44Hw0b3W8Pe+cMxc8V1ULQ07oh8VNbIRaoD1LRZVTvD+0nieDKjfgKg89sD7rrKrg== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/plugin-syntax-logical-assignment-operators@^7.8.3": + version "7.10.4" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-logical-assignment-operators/-/plugin-syntax-logical-assignment-operators-7.10.4.tgz#ca91ef46303530448b906652bac2e9fe9941f699" + integrity sha512-d8waShlpFDinQ5MtvGU9xDAOzKH47+FFoney2baFIoMr952hKOLp1HR7VszoZvOsV/4+RRszNY7D17ba0te0ig== + dependencies: + "@babel/helper-plugin-utils" "^7.10.4" + +"@babel/plugin-syntax-nullish-coalescing-operator@^7.8.3": + version "7.8.3" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-nullish-coalescing-operator/-/plugin-syntax-nullish-coalescing-operator-7.8.3.tgz#167ed70368886081f74b5c36c65a88c03b66d1a9" + integrity sha512-aSff4zPII1u2QD7y+F8oDsz19ew4IGEJg9SVW+bqwpwtfFleiQDMdzA/R+UlWDzfnHFCxxleFT0PMIrR36XLNQ== + dependencies: + "@babel/helper-plugin-utils" "^7.8.0" + +"@babel/plugin-syntax-numeric-separator@^7.8.3": + version "7.10.4" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-numeric-separator/-/plugin-syntax-numeric-separator-7.10.4.tgz#b9b070b3e33570cd9fd07ba7fa91c0dd37b9af97" + integrity sha512-9H6YdfkcK/uOnY/K7/aA2xpzaAgkQn37yzWUMRK7OaPOqOpGS1+n0H5hxT9AUw9EsSjPW8SVyMJwYRtWs3X3ug== + dependencies: + "@babel/helper-plugin-utils" "^7.10.4" + +"@babel/plugin-syntax-object-rest-spread@^7.8.3": + version "7.8.3" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-object-rest-spread/-/plugin-syntax-object-rest-spread-7.8.3.tgz#60e225edcbd98a640332a2e72dd3e66f1af55871" + integrity sha512-XoqMijGZb9y3y2XskN+P1wUGiVwWZ5JmoDRwx5+3GmEplNyVM2s2Dg8ILFQm8rWM48orGy5YpI5Bl8U1y7ydlA== + dependencies: + "@babel/helper-plugin-utils" "^7.8.0" + +"@babel/plugin-syntax-optional-catch-binding@^7.8.3": + version "7.8.3" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-optional-catch-binding/-/plugin-syntax-optional-catch-binding-7.8.3.tgz#6111a265bcfb020eb9efd0fdfd7d26402b9ed6c1" + integrity sha512-6VPD0Pc1lpTqw0aKoeRTMiB+kWhAoT24PA+ksWSBrFtl5SIRVpZlwN3NNPQjehA2E/91FV3RjLWoVTglWcSV3Q== + dependencies: + "@babel/helper-plugin-utils" "^7.8.0" + +"@babel/plugin-syntax-optional-chaining@^7.8.3": + version "7.8.3" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-optional-chaining/-/plugin-syntax-optional-chaining-7.8.3.tgz#4f69c2ab95167e0180cd5336613f8c5788f7d48a" + integrity sha512-KoK9ErH1MBlCPxV0VANkXW2/dw4vlbGDrFgz8bmUsBGYkFRcbRwMh6cIJubdPrkxRwuGdtCk0v/wPTKbQgBjkg== + dependencies: + "@babel/helper-plugin-utils" "^7.8.0" + +"@babel/plugin-syntax-top-level-await@^7.8.3": + version "7.14.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-top-level-await/-/plugin-syntax-top-level-await-7.14.5.tgz#c1cfdadc35a646240001f06138247b741c34d94c" + integrity sha512-hx++upLv5U1rgYfwe1xBQUhRmU41NEvpUvrp8jkrSCdvGSnM5/qdRMtylJ6PG5OFkBaHkbTAKTnd3/YyESRHFw== + dependencies: + "@babel/helper-plugin-utils" "^7.14.5" + +"@babel/plugin-syntax-typescript@^7.7.2": + version "7.22.5" + resolved "https://registry.yarnpkg.com/@babel/plugin-syntax-typescript/-/plugin-syntax-typescript-7.22.5.tgz#aac8d383b062c5072c647a31ef990c1d0af90272" + integrity sha512-1mS2o03i7t1c6VzH6fdQ3OA8tcEIxwG18zIPRp+UY1Ihv6W+XZzBCVxExF9upussPXJ0xE9XRHwMoNs1ep/nRQ== + dependencies: + "@babel/helper-plugin-utils" "^7.22.5" + +"@babel/template@^7.22.15", "@babel/template@^7.3.3": + version "7.22.15" + resolved "https://registry.yarnpkg.com/@babel/template/-/template-7.22.15.tgz#09576efc3830f0430f4548ef971dde1350ef2f38" + integrity sha512-QPErUVm4uyJa60rkI73qneDacvdvzxshT3kksGqlGWYdOTIUOwJ7RDUL8sGqslY1uXWSL6xMFKEXDS3ox2uF0w== + dependencies: + "@babel/code-frame" "^7.22.13" + "@babel/parser" "^7.22.15" + "@babel/types" "^7.22.15" + +"@babel/traverse@^7.23.2", "@babel/traverse@^7.7.2": + version "7.23.2" + resolved "https://registry.yarnpkg.com/@babel/traverse/-/traverse-7.23.2.tgz#329c7a06735e144a506bdb2cad0268b7f46f4ad8" + integrity sha512-azpe59SQ48qG6nu2CzcMLbxUudtN+dOM9kDbUqGq3HXUJRlo7i8fvPoxQUzYgLZ4cMVmuZgm8vvBpNeRhd6XSw== + dependencies: + "@babel/code-frame" "^7.22.13" + "@babel/generator" "^7.23.0" + "@babel/helper-environment-visitor" "^7.22.20" + "@babel/helper-function-name" "^7.23.0" + "@babel/helper-hoist-variables" "^7.22.5" + "@babel/helper-split-export-declaration" "^7.22.6" + "@babel/parser" "^7.23.0" + "@babel/types" "^7.23.0" + debug "^4.1.0" + globals "^11.1.0" + +"@babel/types@^7.0.0", "@babel/types@^7.20.7", "@babel/types@^7.22.15", "@babel/types@^7.22.5", "@babel/types@^7.23.0", "@babel/types@^7.3.3": + version "7.23.0" + resolved "https://registry.yarnpkg.com/@babel/types/-/types-7.23.0.tgz#8c1f020c9df0e737e4e247c0619f58c68458aaeb" + integrity sha512-0oIyUfKoI3mSqMvsxBdclDwxXKXAUA8v/apZbc+iSyARYou1o8ZGDxbUYyLFoW2arqS2jDGqJuZvv1d/io1axg== + dependencies: + "@babel/helper-string-parser" "^7.22.5" + "@babel/helper-validator-identifier" "^7.22.20" + to-fast-properties "^2.0.0" + +"@bcoe/v8-coverage@^0.2.3": + version "0.2.3" + resolved "https://registry.yarnpkg.com/@bcoe/v8-coverage/-/v8-coverage-0.2.3.tgz#75a2e8b51cb758a7553d6804a5932d7aace75c39" + integrity sha512-0hYQ8SB4Db5zvZB4axdMHGwEaQjkZzFjQiN9LVYvIFB2nSUHW9tYpxWriPrWDASIxiaXax83REcLxuSdnGPZtw== + "@colors/colors@1.5.0": version "1.5.0" resolved "https://registry.yarnpkg.com/@colors/colors/-/colors-1.5.0.tgz#bb504579c1cae923e6576a4f5da43d25f97bdbd9" integrity sha512-ooWCrlZP11i8GImSjTHYHLkvFDP48nS4+204nGb1RiX/WXYHmJA2III9/e2DWVabCESdW7hBAEzHRqUn9OUVvQ== +"@cspotcode/source-map-support@^0.8.0": + version "0.8.1" + resolved "https://registry.yarnpkg.com/@cspotcode/source-map-support/-/source-map-support-0.8.1.tgz#00629c35a688e05a88b1cda684fb9d5e73f000a1" + integrity sha512-IchNf6dN4tHoMFIn/7OE8LWZ19Y6q/67Bmf6vnGREv8RSbBVb9LPJxEcnwrcwX6ixSvaiGoomAUvu4YSxXrVgw== + dependencies: + "@jridgewell/trace-mapping" "0.3.9" + "@dabh/diagnostics@^2.0.2": version "2.0.3" resolved "https://registry.yarnpkg.com/@dabh/diagnostics/-/diagnostics-2.0.3.tgz#7f7e97ee9a725dffc7808d93668cc984e1dc477a" @@ -184,26 +507,21 @@ enabled "2.0.x" kuler "^2.0.0" -"@eslint/eslintrc@^2.0.0": - version "2.0.0" - resolved "https://registry.yarnpkg.com/@eslint/eslintrc/-/eslintrc-2.0.0.tgz#943309d8697c52fc82c076e90c1c74fbbe69dbff" - integrity sha512-fluIaaV+GyV24CCu/ggiHdV+j4RNh85yQnAYS/G2mZODZgGmmlrgCydjUcV3YvxCm9x8nMAfThsqTni4KiXT4A== +"@eslint/eslintrc@^1.3.3": + version "1.3.3" + resolved "https://registry.yarnpkg.com/@eslint/eslintrc/-/eslintrc-1.3.3.tgz#2b044ab39fdfa75b4688184f9e573ce3c5b0ff95" + integrity sha512-uj3pT6Mg+3t39fvLrj8iuCIJ38zKO9FpGtJ4BBJebJhEwjoT+KLVNCcHT5QC9NGRIEi7fZ0ZR8YRb884auB4Lg== dependencies: ajv "^6.12.4" debug "^4.3.2" espree "^9.4.0" - globals "^13.19.0" + globals "^13.15.0" ignore "^5.2.0" import-fresh "^3.2.1" js-yaml "^4.1.0" minimatch "^3.1.2" strip-json-comments "^3.1.1" -"@eslint/js@8.35.0": - version "8.35.0" - resolved "https://registry.yarnpkg.com/@eslint/js/-/js-8.35.0.tgz#b7569632b0b788a0ca0e438235154e45d42813a7" - integrity sha512-JXdzbRiWclLVoD8sNUjR443VVlYqiYmDVT6rGUEIEHU5YJW0gaVZwV2xgM7D4arkvASqD0IlLUVjHiFuxaftRw== - "@gar/promisify@^1.0.1": version "1.1.3" resolved "https://registry.yarnpkg.com/@gar/promisify/-/promisify-1.1.3.tgz#555193ab2e3bb3b6adc3d551c9c030d9e860daf6" @@ -281,14 +599,14 @@ dependencies: "@hapi/hoek" "^9.0.0" -"@humanwhocodes/config-array@^0.11.8": - version "0.11.8" - resolved "https://registry.yarnpkg.com/@humanwhocodes/config-array/-/config-array-0.11.8.tgz#03595ac2075a4dc0f191cc2131de14fbd7d410b9" - integrity sha512-UybHIJzJnR5Qc/MsD9Kr+RpO2h+/P1GhOwdiLPXK5TWk5sgTdu88bTD9UP+CKbPPh5Rni1u0GjAdYQLemG8g+g== +"@humanwhocodes/config-array@^0.10.5": + version "0.10.7" + resolved "https://registry.yarnpkg.com/@humanwhocodes/config-array/-/config-array-0.10.7.tgz#6d53769fd0c222767e6452e8ebda825c22e9f0dc" + integrity sha512-MDl6D6sBsaV452/QSdX+4CXIjZhIcI0PELsxUjk4U828yd58vk3bTIvk/6w5FY+4hIy9sLW0sfrV7K7Kc++j/w== dependencies: "@humanwhocodes/object-schema" "^1.2.1" debug "^4.1.1" - minimatch "^3.0.5" + minimatch "^3.0.4" "@humanwhocodes/module-importer@^1.0.1": version "1.0.1" @@ -300,6 +618,280 @@ resolved "https://registry.yarnpkg.com/@humanwhocodes/object-schema/-/object-schema-1.2.1.tgz#b520529ec21d8e5945a1851dfd1c32e94e39ff45" integrity sha512-ZnQMnLV4e7hDlUvw8H+U8ASL02SS2Gn6+9Ac3wGGLIe7+je2AeAOxPY+izIPJDfFDb7eDjev0Us8MO1iFRN8hA== +"@istanbuljs/load-nyc-config@^1.0.0": + version "1.1.0" + resolved "https://registry.yarnpkg.com/@istanbuljs/load-nyc-config/-/load-nyc-config-1.1.0.tgz#fd3db1d59ecf7cf121e80650bb86712f9b55eced" + integrity sha512-VjeHSlIzpv/NyD3N0YuHfXOPDIixcA1q2ZV98wsMqcYlPmv2n3Yb2lYP9XMElnaFVXg5A7YLTeLu6V84uQDjmQ== + dependencies: + camelcase "^5.3.1" + find-up "^4.1.0" + get-package-type "^0.1.0" + js-yaml "^3.13.1" + resolve-from "^5.0.0" + +"@istanbuljs/schema@^0.1.2": + version "0.1.3" + resolved "https://registry.yarnpkg.com/@istanbuljs/schema/-/schema-0.1.3.tgz#e45e384e4b8ec16bce2fd903af78450f6bf7ec98" + integrity sha512-ZXRY4jNvVgSVQ8DL3LTcakaAtXwTVUxE81hslsyD2AtoXW/wVob10HkOJ1X/pAlcI7D+2YoZKg5do8G/w6RYgA== + +"@jest/console@^29.1.2", "@jest/console@^29.7.0": + version "29.7.0" + resolved "https://registry.yarnpkg.com/@jest/console/-/console-29.7.0.tgz#cd4822dbdb84529265c5a2bdb529a3c9cc950ffc" + integrity sha512-5Ni4CU7XHQi32IJ398EEP4RrB8eV09sXP2ROqD4bksHrnTree52PsxvX8tpL8LvTZ3pFzXyPbNQReSN41CAhOg== + dependencies: + "@jest/types" "^29.6.3" + "@types/node" "*" + chalk "^4.0.0" + jest-message-util "^29.7.0" + jest-util "^29.7.0" + slash "^3.0.0" + +"@jest/core@^29.1.2": + version "29.7.0" + resolved "https://registry.yarnpkg.com/@jest/core/-/core-29.7.0.tgz#b6cccc239f30ff36609658c5a5e2291757ce448f" + integrity sha512-n7aeXWKMnGtDA48y8TLWJPJmLmmZ642Ceo78cYWEpiD7FzDgmNDV/GCVRorPABdXLJZ/9wzzgZAlHjXjxDHGsg== + dependencies: + "@jest/console" "^29.7.0" + "@jest/reporters" "^29.7.0" + "@jest/test-result" "^29.7.0" + "@jest/transform" "^29.7.0" + "@jest/types" "^29.6.3" + "@types/node" "*" + ansi-escapes "^4.2.1" + chalk "^4.0.0" + ci-info "^3.2.0" + exit "^0.1.2" + graceful-fs "^4.2.9" + jest-changed-files "^29.7.0" + jest-config "^29.7.0" + jest-haste-map "^29.7.0" + jest-message-util "^29.7.0" + jest-regex-util "^29.6.3" + jest-resolve "^29.7.0" + jest-resolve-dependencies "^29.7.0" + jest-runner "^29.7.0" + jest-runtime "^29.7.0" + jest-snapshot "^29.7.0" + jest-util "^29.7.0" + jest-validate "^29.7.0" + jest-watcher "^29.7.0" + micromatch "^4.0.4" + pretty-format "^29.7.0" + slash "^3.0.0" + strip-ansi "^6.0.0" + +"@jest/environment@^29.1.2", "@jest/environment@^29.7.0": + version "29.7.0" + resolved "https://registry.yarnpkg.com/@jest/environment/-/environment-29.7.0.tgz#24d61f54ff1f786f3cd4073b4b94416383baf2a7" + integrity sha512-aQIfHDq33ExsN4jP1NWGXhxgQ/wixs60gDiKO+XVMd8Mn0NWPWgc34ZQDTb2jKaUWQ7MuwoitXAsN2XVXNMpAw== + dependencies: + "@jest/fake-timers" "^29.7.0" + "@jest/types" "^29.6.3" + "@types/node" "*" + jest-mock "^29.7.0" + +"@jest/expect-utils@^29.1.2", "@jest/expect-utils@^29.7.0": + version "29.7.0" + resolved "https://registry.yarnpkg.com/@jest/expect-utils/-/expect-utils-29.7.0.tgz#023efe5d26a8a70f21677d0a1afc0f0a44e3a1c6" + integrity sha512-GlsNBWiFQFCVi9QVSx7f5AgMeLxe9YCCs5PuP2O2LdjDAA8Jh9eX7lA1Jq/xdXw3Wb3hyvlFNfZIfcRetSzYcA== + dependencies: + jest-get-type "^29.6.3" + +"@jest/expect@^29.1.2", "@jest/expect@^29.7.0": + version "29.7.0" + resolved "https://registry.yarnpkg.com/@jest/expect/-/expect-29.7.0.tgz#76a3edb0cb753b70dfbfe23283510d3d45432bf2" + integrity sha512-8uMeAMycttpva3P1lBHB8VciS9V0XAr3GymPpipdyQXbBcuhkLQOSe8E/p92RyAdToS6ZD1tFkX+CkhoECE0dQ== + dependencies: + expect "^29.7.0" + jest-snapshot "^29.7.0" + +"@jest/fake-timers@^29.1.2", "@jest/fake-timers@^29.7.0": + version "29.7.0" + resolved "https://registry.yarnpkg.com/@jest/fake-timers/-/fake-timers-29.7.0.tgz#fd91bf1fffb16d7d0d24a426ab1a47a49881a565" + integrity sha512-q4DH1Ha4TTFPdxLsqDXK1d3+ioSL7yL5oCMJZgDYm6i+6CygW5E5xVr/D1HdsGxjt1ZWSfUAs9OxSB/BNelWrQ== + dependencies: + "@jest/types" "^29.6.3" + "@sinonjs/fake-timers" "^10.0.2" + "@types/node" "*" + jest-message-util "^29.7.0" + jest-mock "^29.7.0" + jest-util "^29.7.0" + +"@jest/globals@^29.1.2", "@jest/globals@^29.7.0": + version "29.7.0" + resolved "https://registry.yarnpkg.com/@jest/globals/-/globals-29.7.0.tgz#8d9290f9ec47ff772607fa864ca1d5a2efae1d4d" + integrity sha512-mpiz3dutLbkW2MNFubUGUEVLkTGiqW6yLVTA+JbP6fI6J5iL9Y0Nlg8k95pcF8ctKwCS7WVxteBs29hhfAotzQ== + dependencies: + "@jest/environment" "^29.7.0" + "@jest/expect" "^29.7.0" + "@jest/types" "^29.6.3" + jest-mock "^29.7.0" + +"@jest/reporters@^29.7.0": + version "29.7.0" + resolved "https://registry.yarnpkg.com/@jest/reporters/-/reporters-29.7.0.tgz#04b262ecb3b8faa83b0b3d321623972393e8f4c7" + integrity sha512-DApq0KJbJOEzAFYjHADNNxAE3KbhxQB1y5Kplb5Waqw6zVbuWatSnMjE5gs8FUgEPmNsnZA3NCWl9NG0ia04Pg== + dependencies: + "@bcoe/v8-coverage" "^0.2.3" + "@jest/console" "^29.7.0" + "@jest/test-result" "^29.7.0" + "@jest/transform" "^29.7.0" + "@jest/types" "^29.6.3" + "@jridgewell/trace-mapping" "^0.3.18" + "@types/node" "*" + chalk "^4.0.0" + collect-v8-coverage "^1.0.0" + exit "^0.1.2" + glob "^7.1.3" + graceful-fs "^4.2.9" + istanbul-lib-coverage "^3.0.0" + istanbul-lib-instrument "^6.0.0" + istanbul-lib-report "^3.0.0" + istanbul-lib-source-maps "^4.0.0" + istanbul-reports "^3.1.3" + jest-message-util "^29.7.0" + jest-util "^29.7.0" + jest-worker "^29.7.0" + slash "^3.0.0" + string-length "^4.0.1" + strip-ansi "^6.0.0" + v8-to-istanbul "^9.0.1" + +"@jest/schemas@^29.0.0", "@jest/schemas@^29.6.3": + version "29.6.3" + resolved "https://registry.yarnpkg.com/@jest/schemas/-/schemas-29.6.3.tgz#430b5ce8a4e0044a7e3819663305a7b3091c8e03" + integrity sha512-mo5j5X+jIZmJQveBKeS/clAueipV7KgiX1vMgCxam1RNYiqE1w62n0/tJJnHtjW8ZHcQco5gY85jA3mi0L+nSA== + dependencies: + "@sinclair/typebox" "^0.27.8" + +"@jest/source-map@^29.0.0", "@jest/source-map@^29.6.3": + version "29.6.3" + resolved "https://registry.yarnpkg.com/@jest/source-map/-/source-map-29.6.3.tgz#d90ba772095cf37a34a5eb9413f1b562a08554c4" + integrity sha512-MHjT95QuipcPrpLM+8JMSzFx6eHp5Bm+4XeFDJlwsvVBjmKNiIAvasGK2fxz2WbGRlnvqehFbh07MMa7n3YJnw== + dependencies: + "@jridgewell/trace-mapping" "^0.3.18" + callsites "^3.0.0" + graceful-fs "^4.2.9" + +"@jest/test-result@^29.1.2", "@jest/test-result@^29.7.0": + version "29.7.0" + resolved "https://registry.yarnpkg.com/@jest/test-result/-/test-result-29.7.0.tgz#8db9a80aa1a097bb2262572686734baed9b1657c" + integrity sha512-Fdx+tv6x1zlkJPcWXmMDAG2HBnaR9XPSd5aDWQVsfrZmLVT3lU1cwyxLgRmXR9yrq4NBoEm9BMsfgFzTQAbJYA== + dependencies: + "@jest/console" "^29.7.0" + "@jest/types" "^29.6.3" + "@types/istanbul-lib-coverage" "^2.0.0" + collect-v8-coverage "^1.0.0" + +"@jest/test-sequencer@^29.1.2", "@jest/test-sequencer@^29.7.0": + version "29.7.0" + resolved "https://registry.yarnpkg.com/@jest/test-sequencer/-/test-sequencer-29.7.0.tgz#6cef977ce1d39834a3aea887a1726628a6f072ce" + integrity sha512-GQwJ5WZVrKnOJuiYiAF52UNUJXgTZx1NHjFSEB0qEMmSZKAkdMoIzw/Cj6x6NF4AvV23AUqDpFzQkN/eYCYTxw== + dependencies: + "@jest/test-result" "^29.7.0" + graceful-fs "^4.2.9" + jest-haste-map "^29.7.0" + slash "^3.0.0" + +"@jest/transform@^29.1.2", "@jest/transform@^29.7.0": + version "29.7.0" + resolved "https://registry.yarnpkg.com/@jest/transform/-/transform-29.7.0.tgz#df2dd9c346c7d7768b8a06639994640c642e284c" + integrity sha512-ok/BTPFzFKVMwO5eOHRrvnBVHdRy9IrsrW1GpMaQ9MCnilNLXQKmAX8s1YXDFaai9xJpac2ySzV0YeRRECr2Vw== + dependencies: + "@babel/core" "^7.11.6" + "@jest/types" "^29.6.3" + "@jridgewell/trace-mapping" "^0.3.18" + babel-plugin-istanbul "^6.1.1" + chalk "^4.0.0" + convert-source-map "^2.0.0" + fast-json-stable-stringify "^2.1.0" + graceful-fs "^4.2.9" + jest-haste-map "^29.7.0" + jest-regex-util "^29.6.3" + jest-util "^29.7.0" + micromatch "^4.0.4" + pirates "^4.0.4" + slash "^3.0.0" + write-file-atomic "^4.0.2" + +"@jest/types@^29.1.2", "@jest/types@^29.6.3": + version "29.6.3" + resolved "https://registry.yarnpkg.com/@jest/types/-/types-29.6.3.tgz#1131f8cf634e7e84c5e77bab12f052af585fba59" + integrity sha512-u3UPsIilWKOM3F9CXtrG8LEJmNxwoCQC/XVj4IKYXvvpx7QIi/Kg1LI5uDmDpKlac62NUtX7eLjRh+jVZcLOzw== + dependencies: + "@jest/schemas" "^29.6.3" + "@types/istanbul-lib-coverage" "^2.0.0" + "@types/istanbul-reports" "^3.0.0" + "@types/node" "*" + "@types/yargs" "^17.0.8" + chalk "^4.0.0" + +"@jridgewell/gen-mapping@^0.3.0", "@jridgewell/gen-mapping@^0.3.2": + version "0.3.3" + resolved "https://registry.yarnpkg.com/@jridgewell/gen-mapping/-/gen-mapping-0.3.3.tgz#7e02e6eb5df901aaedb08514203b096614024098" + integrity sha512-HLhSWOLRi875zjjMG/r+Nv0oCW8umGb0BgEhyX3dDX3egwZtB8PqLnjz3yedt8R5StBrzcg4aBpnh8UA9D1BoQ== + dependencies: + "@jridgewell/set-array" "^1.0.1" + "@jridgewell/sourcemap-codec" "^1.4.10" + "@jridgewell/trace-mapping" "^0.3.9" + +"@jridgewell/resolve-uri@3.1.0", "@jridgewell/resolve-uri@^3.0.3": + version "3.1.0" + resolved "https://registry.yarnpkg.com/@jridgewell/resolve-uri/-/resolve-uri-3.1.0.tgz#2203b118c157721addfe69d47b70465463066d78" + integrity sha512-F2msla3tad+Mfht5cJq7LSXcdudKTWCVYUgw6pLFOOHSTtZlj6SWNYAp+AhuqLmWdBO2X5hPrLcu8cVP8fy28w== + +"@jridgewell/resolve-uri@^3.1.0": + version "3.1.1" + resolved "https://registry.yarnpkg.com/@jridgewell/resolve-uri/-/resolve-uri-3.1.1.tgz#c08679063f279615a3326583ba3a90d1d82cc721" + integrity sha512-dSYZh7HhCDtCKm4QakX0xFpsRDqjjtZf/kjI/v3T3Nwt5r8/qz/M19F9ySyOqU94SXBmeG9ttTul+YnR4LOxFA== + +"@jridgewell/set-array@^1.0.1": + version "1.1.2" + resolved "https://registry.yarnpkg.com/@jridgewell/set-array/-/set-array-1.1.2.tgz#7c6cf998d6d20b914c0a55a91ae928ff25965e72" + integrity sha512-xnkseuNADM0gt2bs+BvhO0p78Mk762YnZdsuzFV018NoG1Sj1SCQvpSqa7XUaTam5vAGasABV9qXASMKnFMwMw== + +"@jridgewell/sourcemap-codec@1.4.14": + version "1.4.14" + resolved "https://registry.yarnpkg.com/@jridgewell/sourcemap-codec/-/sourcemap-codec-1.4.14.tgz#add4c98d341472a289190b424efbdb096991bb24" + integrity sha512-XPSJHWmi394fuUuzDnGz1wiKqWfo1yXecHQMRf2l6hztTO+nPru658AyDngaBe7isIxEkRsPR3FZh+s7iVa4Uw== + +"@jridgewell/sourcemap-codec@^1.4.10", "@jridgewell/sourcemap-codec@^1.4.14": + version "1.4.15" + resolved "https://registry.yarnpkg.com/@jridgewell/sourcemap-codec/-/sourcemap-codec-1.4.15.tgz#d7c6e6755c78567a951e04ab52ef0fd26de59f32" + integrity sha512-eF2rxCRulEKXHTRiDrDy6erMYWqNw4LPdQ8UQA4huuxaQsVeRPFl2oM8oDGxMFhJUWZf9McpLtJasDDZb/Bpeg== + +"@jridgewell/trace-mapping@0.3.9": + version "0.3.9" + resolved "https://registry.yarnpkg.com/@jridgewell/trace-mapping/-/trace-mapping-0.3.9.tgz#6534fd5933a53ba7cbf3a17615e273a0d1273ff9" + integrity sha512-3Belt6tdc8bPgAtbcmdtNJlirVoTmEb5e2gC94PnkwEW9jI6CAHUeoG85tjWP5WquqfavoMtMwiG4P926ZKKuQ== + dependencies: + "@jridgewell/resolve-uri" "^3.0.3" + "@jridgewell/sourcemap-codec" "^1.4.10" + +"@jridgewell/trace-mapping@^0.3.12": + version "0.3.16" + resolved "https://registry.yarnpkg.com/@jridgewell/trace-mapping/-/trace-mapping-0.3.16.tgz#a7982f16c18cae02be36274365433e5b49d7b23f" + integrity sha512-LCQ+NeThyJ4k1W2d+vIKdxuSt9R3pQSZ4P92m7EakaYuXcVWbHuT5bjNcqLd4Rdgi6xYWYDvBJZJLZSLanjDcA== + dependencies: + "@jridgewell/resolve-uri" "3.1.0" + "@jridgewell/sourcemap-codec" "1.4.14" + +"@jridgewell/trace-mapping@^0.3.17", "@jridgewell/trace-mapping@^0.3.18": + version "0.3.20" + resolved "https://registry.yarnpkg.com/@jridgewell/trace-mapping/-/trace-mapping-0.3.20.tgz#72e45707cf240fa6b081d0366f8265b0cd10197f" + integrity sha512-R8LcPeWZol2zR8mmH3JeKQ6QRCFb7XgUhV9ZlGhHLGyg4wpPiPZNQOOWhFZhxKw8u//yTbNGI42Bx/3paXEQ+Q== + dependencies: + "@jridgewell/resolve-uri" "^3.1.0" + "@jridgewell/sourcemap-codec" "^1.4.14" + +"@jridgewell/trace-mapping@^0.3.9": + version "0.3.14" + resolved "https://registry.yarnpkg.com/@jridgewell/trace-mapping/-/trace-mapping-0.3.14.tgz#b231a081d8f66796e475ad588a1ef473112701ed" + integrity sha512-bJWEfQ9lPTvm3SneWwRFVLzrh6nhjwqw7TUFFBEMzwvg7t7PCDenf2lDwqo4NQXzdpgBXyFgDWnQA+2vkruksQ== + dependencies: + "@jridgewell/resolve-uri" "^3.0.3" + "@jridgewell/sourcemap-codec" "^1.4.10" + "@log4js-node/log4js-api@^1.0.0": version "1.0.2" resolved "https://registry.yarnpkg.com/@log4js-node/log4js-api/-/log4js-api-1.0.2.tgz#7a8143fb33f077df3e579dca7f18fea74a02ec8b" @@ -313,12 +905,12 @@ "@nodelib/fs.stat" "2.0.5" run-parallel "^1.1.9" -"@nodelib/fs.stat@2.0.5": +"@nodelib/fs.stat@2.0.5", "@nodelib/fs.stat@^2.0.2": version "2.0.5" resolved "https://registry.yarnpkg.com/@nodelib/fs.stat/-/fs.stat-2.0.5.tgz#5bd262af94e9d25bd1e71b05deed44876a222e8b" integrity sha512-RkhPPp2zrqDAQA/2jNhnztcPAlv64XdhIp7a7454A5ovI7Bukxgt7MX7udwAu3zg1DcpPU0rz3VV1SeaqvY4+A== -"@nodelib/fs.walk@^1.2.8": +"@nodelib/fs.walk@^1.2.3": version "1.2.8" resolved "https://registry.yarnpkg.com/@nodelib/fs.walk/-/fs.walk-1.2.8.tgz#e95737e8bb6746ddedf69c556953494f196fe69a" integrity sha512-oGB+UxlgWcgQkgwo8GcEGwemoTFt3FIO9ababBmaGwXIoBKZ+GTy0pP185beGg7Llih/NSHSV2XAs1lnznocSg== @@ -371,6 +963,11 @@ resolved "https://registry.yarnpkg.com/@sideway/pinpoint/-/pinpoint-2.0.0.tgz#cff8ffadc372ad29fd3f78277aeb29e632cc70df" integrity sha512-RNiOoTPkptFtSVzQevY/yWtZwf/RxyVnPy/OcA9HBM3MlGDnBEYL5B41H0MTn0Uec8Hi+2qUtTfG2WWZBmMejQ== +"@sinclair/typebox@^0.27.8": + version "0.27.8" + resolved "https://registry.yarnpkg.com/@sinclair/typebox/-/typebox-0.27.8.tgz#6667fac16c436b5434a387a34dedb013198f6e6e" + integrity sha512-+Fj43pSMwJs4KRrH/938Uf+uAELIgVBmQzg/q1YG10djyfA3TnrU8N8XzqCh/okZdszqBQTZf96idMfE5lnwTA== + "@sinonjs/commons@^1.6.0", "@sinonjs/commons@^1.7.0", "@sinonjs/commons@^1.8.1", "@sinonjs/commons@^1.8.3": version "1.8.3" resolved "https://registry.yarnpkg.com/@sinonjs/commons/-/commons-1.8.3.tgz#3802ddd21a50a949b6721ddd72da36e67e7f1b2d" @@ -378,6 +975,13 @@ dependencies: type-detect "4.0.8" +"@sinonjs/commons@^3.0.0": + version "3.0.0" + resolved "https://registry.yarnpkg.com/@sinonjs/commons/-/commons-3.0.0.tgz#beb434fe875d965265e04722ccfc21df7f755d72" + integrity sha512-jXBtWAF4vmdNmZgD5FoKsVLv3rPgDnLgPbU84LIJ3otV44vJlDRokVng5v8NFJdCf/da9legHcKaRuZs4L7faA== + dependencies: + type-detect "4.0.8" + "@sinonjs/fake-timers@>=5": version "9.1.2" resolved "https://registry.yarnpkg.com/@sinonjs/fake-timers/-/fake-timers-9.1.2.tgz#4eaab737fab77332ab132d396a3c0d364bd0ea8c" @@ -385,6 +989,13 @@ dependencies: "@sinonjs/commons" "^1.7.0" +"@sinonjs/fake-timers@^10.0.2": + version "10.3.0" + resolved "https://registry.yarnpkg.com/@sinonjs/fake-timers/-/fake-timers-10.3.0.tgz#55fdff1ecab9f354019129daf4df0dd4d923ea66" + integrity sha512-V4BG07kuYSUkTCSBHG8G8TNhM+F19jXFWnQtzj+we8DrkpSBCee9Z3Ms8yiGer/dlmhe35/Xdgyo3/0rQKg7YA== + dependencies: + "@sinonjs/commons" "^3.0.0" + "@sinonjs/fake-timers@^7.0.4": version "7.1.2" resolved "https://registry.yarnpkg.com/@sinonjs/fake-timers/-/fake-timers-7.1.2.tgz#2524eae70c4910edccf99b2f4e6efc5894aff7b5" @@ -421,27 +1032,87 @@ resolved "https://registry.yarnpkg.com/@tootallnate/once/-/once-2.0.0.tgz#f544a148d3ab35801c1f633a7441fd87c2e484bf" integrity sha512-XCuKFP5PS55gnMVu3dty8KPatLqUoy/ZYzDzAGCQ8JNFCkLXzmI7vNHCR+XpbZaMWQK/vQubr7PkYq8g470J/A== +"@tsconfig/node10@^1.0.7": + version "1.0.9" + resolved "https://registry.yarnpkg.com/@tsconfig/node10/-/node10-1.0.9.tgz#df4907fc07a886922637b15e02d4cebc4c0021b2" + integrity sha512-jNsYVVxU8v5g43Erja32laIDHXeoNvFEpX33OK4d6hljo3jDhCBDhx5dhCCTMWUojscpAagGiRkBKxpdl9fxqA== + +"@tsconfig/node12@^1.0.7": + version "1.0.11" + resolved "https://registry.yarnpkg.com/@tsconfig/node12/-/node12-1.0.11.tgz#ee3def1f27d9ed66dac6e46a295cffb0152e058d" + integrity sha512-cqefuRsh12pWyGsIoBKJA9luFu3mRxCA+ORZvA4ktLSzIuCUtWVxGIuXigEwO5/ywWFMZ2QEGKWvkZG1zDMTag== + +"@tsconfig/node14@^1.0.0": + version "1.0.3" + resolved "https://registry.yarnpkg.com/@tsconfig/node14/-/node14-1.0.3.tgz#e4386316284f00b98435bf40f72f75a09dabf6c1" + integrity sha512-ysT8mhdixWK6Hw3i1V2AeRqZ5WfXg1G43mqoYlM2nc6388Fq5jcXyr5mRsqViLx/GJYdoL0bfXD8nmF+Zn/Iow== + +"@tsconfig/node16@^1.0.2", "@tsconfig/node16@^1.0.3": + version "1.0.3" + resolved "https://registry.yarnpkg.com/@tsconfig/node16/-/node16-1.0.3.tgz#472eaab5f15c1ffdd7f8628bd4c4f753995ec79e" + integrity sha512-yOlFc+7UtL/89t2ZhjPvvB/DeAr3r+Dq58IgzsFkOAvVC6NMJXmCGjbptdXdR9qsX7pKcTL+s87FtYREi2dEEQ== + "@types/async@^3.2.12": version "3.2.15" resolved "https://registry.yarnpkg.com/@types/async/-/async-3.2.15.tgz#26d4768fdda0e466f18d6c9918ca28cc89a4e1fe" integrity sha512-PAmPfzvFA31mRoqZyTVsgJMsvbynR429UTTxhmfsUCrWGh3/fxOrzqBtaTPJsn4UtzTv4Vb0+/O7CARWb69N4g== +"@types/babel__core@^7.1.14": + version "7.20.3" + resolved "https://registry.yarnpkg.com/@types/babel__core/-/babel__core-7.20.3.tgz#d5625a50b6f18244425a1359a858c73d70340778" + integrity sha512-54fjTSeSHwfan8AyHWrKbfBWiEUrNTZsUwPTDSNaaP1QDQIZbeNUg3a59E9D+375MzUw/x1vx2/0F5LBz+AeYA== + dependencies: + "@babel/parser" "^7.20.7" + "@babel/types" "^7.20.7" + "@types/babel__generator" "*" + "@types/babel__template" "*" + "@types/babel__traverse" "*" + +"@types/babel__generator@*": + version "7.6.6" + resolved "https://registry.yarnpkg.com/@types/babel__generator/-/babel__generator-7.6.6.tgz#676f89f67dc8ddaae923f70ebc5f1fa800c031a8" + integrity sha512-66BXMKb/sUWbMdBNdMvajU7i/44RkrA3z/Yt1c7R5xejt8qh84iU54yUWCtm0QwGJlDcf/gg4zd/x4mpLAlb/w== + dependencies: + "@babel/types" "^7.0.0" + +"@types/babel__template@*": + version "7.4.3" + resolved "https://registry.yarnpkg.com/@types/babel__template/-/babel__template-7.4.3.tgz#db9ac539a2fe05cfe9e168b24f360701bde41f5f" + integrity sha512-ciwyCLeuRfxboZ4isgdNZi/tkt06m8Tw6uGbBSBgWrnnZGNXiEyM27xc/PjXGQLqlZ6ylbgHMnm7ccF9tCkOeQ== + dependencies: + "@babel/parser" "^7.1.0" + "@babel/types" "^7.0.0" + +"@types/babel__traverse@*", "@types/babel__traverse@^7.0.6": + version "7.20.3" + resolved "https://registry.yarnpkg.com/@types/babel__traverse/-/babel__traverse-7.20.3.tgz#a971aa47441b28ef17884ff945d0551265a2d058" + integrity sha512-Lsh766rGEFbaxMIDH7Qa+Yha8cMVI3qAK6CHt3OR0YfxOIn5Z54iHiyDRycHrBqeIiqGa20Kpsv1cavfBKkRSw== + dependencies: + "@babel/types" "^7.20.7" + "@types/cookie@^0.4.1": version "0.4.1" resolved "https://registry.yarnpkg.com/@types/cookie/-/cookie-0.4.1.tgz#bfd02c1f2224567676c1545199f87c3a861d878d" integrity sha512-XW/Aa8APYr6jSVVA1y/DEIZX0/GMKLEVekNG727R8cs56ahETkRAy/3DR7+fJyh7oUgGwNQaRfXCun0+KbWY7Q== "@types/cors@^2.8.12": - version "2.8.13" - resolved "https://registry.yarnpkg.com/@types/cors/-/cors-2.8.13.tgz#b8ade22ba455a1b8cb3b5d3f35910fd204f84f94" - integrity sha512-RG8AStHlUiV5ysZQKq97copd2UmVYw3/pRMLefISZ3S1hK104Cwm7iLQ3fTKx+lsUH2CE8FlLaYeEA2LSeqYUA== + version "2.8.15" + resolved "https://registry.yarnpkg.com/@types/cors/-/cors-2.8.15.tgz#eb143aa2f8807ddd78e83cbff141bbedd91b60ee" + integrity sha512-n91JxbNLD8eQIuXDIChAN1tCKNWCEgpceU9b7ZMbFA+P+Q4yIeh80jizFLEvolRPc1ES0VdwFlGv+kJTSirogw== + dependencies: + "@types/node" "*" + +"@types/graceful-fs@^4.1.3": + version "4.1.8" + resolved "https://registry.yarnpkg.com/@types/graceful-fs/-/graceful-fs-4.1.8.tgz#417e461e4dc79d957dc3107f45fe4973b09c2915" + integrity sha512-NhRH7YzWq8WiNKVavKPBmtLYZHxNY19Hh+az28O/phfp68CF45pMFud+ZzJ8ewnxnC5smIdF3dqFeiSUQ5I+pw== dependencies: "@types/node" "*" "@types/hapi__joi@*": - version "17.1.10" - resolved "https://registry.yarnpkg.com/@types/hapi__joi/-/hapi__joi-17.1.10.tgz#d45dcd8fdf01ac72d259c460e14ea97036d56655" - integrity sha512-+We0kOukfhPcQv9yKvgJn3SFbCUvRir4Uvu1WuuFpysQlXGAH9lp675GHs9l8MxuhapYIq0dKLNVf/nEbe+G9A== + version "17.1.12" + resolved "https://registry.yarnpkg.com/@types/hapi__joi/-/hapi__joi-17.1.12.tgz#fc9911e568b5219ccf5ff95f4186dd1bc54c4c32" + integrity sha512-a7ao2UWX6q9DMbQgP7RlQGkE6fUBbh0Qv9Cs8DAQiGyOeFtGiJ4GtgKzsscUb1Syw4zSV3zC9GDfNOMOKxW5jg== "@types/hapi__joi@~15": version "15.0.4" @@ -450,6 +1121,30 @@ dependencies: "@types/hapi__joi" "*" +"@types/istanbul-lib-coverage@*", "@types/istanbul-lib-coverage@^2.0.0", "@types/istanbul-lib-coverage@^2.0.1": + version "2.0.5" + resolved "https://registry.yarnpkg.com/@types/istanbul-lib-coverage/-/istanbul-lib-coverage-2.0.5.tgz#fdfdd69fa16d530047d9963635bd77c71a08c068" + integrity sha512-zONci81DZYCZjiLe0r6equvZut0b+dBRPBN5kBDjsONnutYNtJMoWQ9uR2RkL1gLG9NMTzvf+29e5RFfPbeKhQ== + +"@types/istanbul-lib-report@*": + version "3.0.2" + resolved "https://registry.yarnpkg.com/@types/istanbul-lib-report/-/istanbul-lib-report-3.0.2.tgz#394798d5f727402eb5ec99eb9618ffcd2b7645a1" + integrity sha512-8toY6FgdltSdONav1XtUHl4LN1yTmLza+EuDazb/fEmRNCwjyqNVIQWs2IfC74IqjHkREs/nQ2FWq5kZU9IC0w== + dependencies: + "@types/istanbul-lib-coverage" "*" + +"@types/istanbul-reports@^3.0.0": + version "3.0.3" + resolved "https://registry.yarnpkg.com/@types/istanbul-reports/-/istanbul-reports-3.0.3.tgz#0313e2608e6d6955d195f55361ddeebd4b74c6e7" + integrity sha512-1nESsePMBlf0RPRffLZi5ujYh7IH1BWL4y9pr+Bn3cJBdxz+RTP8bUFljLz9HvzhhOSWKdyBZ4DIivdL6rvgZg== + dependencies: + "@types/istanbul-lib-report" "*" + +"@types/json-schema@^7.0.9": + version "7.0.11" + resolved "https://registry.yarnpkg.com/@types/json-schema/-/json-schema-7.0.11.tgz#d421b6c527a3037f7c84433fd2c4229e016863d3" + integrity sha512-wOuvG1SN4Us4rez+tylwwwCV1psiNVOkJeM3AUWUNWg/jDQY2+HE/444y5gc+jBmRqASOm2Oeh5c1axHobwRKQ== + "@types/json5@^0.0.29": version "0.0.29" resolved "https://registry.yarnpkg.com/@types/json5/-/json5-0.0.29.tgz#ee28707ae94e11d2b827bcbe5270bcea7f3e71ee" @@ -464,9 +1159,9 @@ form-data "^3.0.0" "@types/node@*": - version "18.11.17" - resolved "https://registry.yarnpkg.com/@types/node/-/node-18.11.17.tgz#5c009e1d9c38f4a2a9d45c0b0c493fe6cdb4bcb5" - integrity sha512-HJSUJmni4BeDHhfzn6nF0sVmd1SMezP7/4F0Lq+aXzmp2xm9O7WXrUtHW/CHlYVtZUbByEvWidHqRtcJXGF2Ng== + version "18.8.5" + resolved "https://registry.yarnpkg.com/@types/node/-/node-18.8.5.tgz#6a31f820c1077c3f8ce44f9e203e68a176e8f59e" + integrity sha512-Bq7G3AErwe5A/Zki5fdD3O6+0zDChhg671NfPjtIcbtzDNZTv4NPKMRFr7gtYPG7y+B8uTiNK4Ngd9T0FTar6Q== "@types/node@>=10.0.0": version "20.2.5" @@ -478,6 +1173,21 @@ resolved "https://registry.yarnpkg.com/@types/node/-/node-18.17.9.tgz#b219fa0da3c574399f4a44455ea4b388029afd56" integrity sha512-fxaKquqYcPOGwE7tC1anJaPJ0GHyOVzfA2oUoXECjBjrtsIz4YJvtNYsq8LUcjEUehEF+jGpx8Z+lFrtT6z0tg== +"@types/node@^18.11.9": + version "18.11.9" + resolved "https://registry.yarnpkg.com/@types/node/-/node-18.11.9.tgz#02d013de7058cea16d36168ef2fc653464cfbad4" + integrity sha512-CRpX21/kGdzjOpFsZSkcrXMGIBWMGNIHXXBVFSH+ggkftxg+XYP20TESbh+zFvFj3EQOl5byk0HTRn1IL6hbqg== + +"@types/prettier@^2.1.5": + version "2.7.1" + resolved "https://registry.yarnpkg.com/@types/prettier/-/prettier-2.7.1.tgz#dfd20e2dc35f027cdd6c1908e80a5ddc7499670e" + integrity sha512-ri0UmynRRvZiiUJdiz38MmIblKK+oH30MztdBVR95dv/Ubw6neWSb8u1XpRb72L4qsZOhz+L+z9JD40SJmfWow== + +"@types/stack-utils@^2.0.0": + version "2.0.1" + resolved "https://registry.yarnpkg.com/@types/stack-utils/-/stack-utils-2.0.1.tgz#20f18294f797f2209b5f65c8e3b5c8e8261d127c" + integrity sha512-Hl219/BT5fLAaz6NDkSuhzasy49dwQS/DSdu4MdggFB8zcXv7vflBI3xp7FEmkmdDkBUI2bPUNeMttp2knYdxw== + "@types/tunnel@^0.0.3": version "0.0.3" resolved "https://registry.yarnpkg.com/@types/tunnel/-/tunnel-0.0.3.tgz#f109e730b072b3136347561fc558c9358bb8c6e9" @@ -491,9 +1201,9 @@ integrity sha512-1EkWuw7rT3BMz2HpmcEOr/HL61mWNA6Ulr/KdbXR9AI0A55wD4Qfv8hizd8Q1DnknSIzzDvQmvvY/guvX7jjZA== "@types/webidl-conversions@*": - version "7.0.0" - resolved "https://registry.yarnpkg.com/@types/webidl-conversions/-/webidl-conversions-7.0.0.tgz#2b8e60e33906459219aa587e9d1a612ae994cfe7" - integrity sha512-xTE1E+YF4aWPJJeUzaZI5DRntlkY3+BCVJi0axFptnjGmAoWxkyREIh/XMrfxVLejwQxMCfDXdICo0VLxThrog== + version "7.0.2" + resolved "https://registry.yarnpkg.com/@types/webidl-conversions/-/webidl-conversions-7.0.2.tgz#d703e2bf61d8b77a7669adcd8fdf98108155d594" + integrity sha512-uNv6b/uGRLlCVmelat2rA8bcVd3k/42mV2EmjhPh6JLkd35T5bgwR/t6xy7a9MWhd9sixIeBUzhBenvk3NO+DQ== "@types/whatwg-url@^8.2.1": version "8.2.2" @@ -503,6 +1213,65 @@ "@types/node" "*" "@types/webidl-conversions" "*" +"@types/yargs-parser@*": + version "21.0.2" + resolved "https://registry.yarnpkg.com/@types/yargs-parser/-/yargs-parser-21.0.2.tgz#7bd04c5da378496ef1695a1008bf8f71847a8b8b" + integrity sha512-5qcvofLPbfjmBfKaLfj/+f+Sbd6pN4zl7w7VSVI5uz7m9QZTuB2aZAa2uo1wHFBNN2x6g/SoTkXmd8mQnQF2Cw== + +"@types/yargs@^17.0.8": + version "17.0.29" + resolved "https://registry.yarnpkg.com/@types/yargs/-/yargs-17.0.29.tgz#06aabc72497b798c643c812a8b561537fea760cf" + integrity sha512-nacjqA3ee9zRF/++a3FUY1suHTFKZeHba2n8WeDw9cCVdmzmHpIxyzOJBcpHvvEmS8E9KqWlSnWHUkOrkhWcvA== + dependencies: + "@types/yargs-parser" "*" + +"@typescript-eslint/scope-manager@5.40.0": + version "5.40.0" + resolved "https://registry.yarnpkg.com/@typescript-eslint/scope-manager/-/scope-manager-5.40.0.tgz#d6ea782c8e3a2371ba3ea31458dcbdc934668fc4" + integrity sha512-d3nPmjUeZtEWRvyReMI4I1MwPGC63E8pDoHy0BnrYjnJgilBD3hv7XOiETKLY/zTwI7kCnBDf2vWTRUVpYw0Uw== + dependencies: + "@typescript-eslint/types" "5.40.0" + "@typescript-eslint/visitor-keys" "5.40.0" + +"@typescript-eslint/types@5.40.0": + version "5.40.0" + resolved "https://registry.yarnpkg.com/@typescript-eslint/types/-/types-5.40.0.tgz#8de07e118a10b8f63c99e174a3860f75608c822e" + integrity sha512-V1KdQRTXsYpf1Y1fXCeZ+uhjW48Niiw0VGt4V8yzuaDTU8Z1Xl7yQDyQNqyAFcVhpYXIVCEuxSIWTsLDpHgTbw== + +"@typescript-eslint/typescript-estree@5.40.0": + version "5.40.0" + resolved "https://registry.yarnpkg.com/@typescript-eslint/typescript-estree/-/typescript-estree-5.40.0.tgz#e305e6a5d65226efa5471ee0f12e0ffaab6d3075" + integrity sha512-b0GYlDj8TLTOqwX7EGbw2gL5EXS2CPEWhF9nGJiGmEcmlpNBjyHsTwbqpyIEPVpl6br4UcBOYlcI2FJVtJkYhg== + dependencies: + "@typescript-eslint/types" "5.40.0" + "@typescript-eslint/visitor-keys" "5.40.0" + debug "^4.3.4" + globby "^11.1.0" + is-glob "^4.0.3" + semver "^7.3.7" + tsutils "^3.21.0" + +"@typescript-eslint/utils@^5.10.0": + version "5.40.0" + resolved "https://registry.yarnpkg.com/@typescript-eslint/utils/-/utils-5.40.0.tgz#647f56a875fd09d33c6abd70913c3dd50759b772" + integrity sha512-MO0y3T5BQ5+tkkuYZJBjePewsY+cQnfkYeRqS6tPh28niiIwPnQ1t59CSRcs1ZwJJNOdWw7rv9pF8aP58IMihA== + dependencies: + "@types/json-schema" "^7.0.9" + "@typescript-eslint/scope-manager" "5.40.0" + "@typescript-eslint/types" "5.40.0" + "@typescript-eslint/typescript-estree" "5.40.0" + eslint-scope "^5.1.1" + eslint-utils "^3.0.0" + semver "^7.3.7" + +"@typescript-eslint/visitor-keys@5.40.0": + version "5.40.0" + resolved "https://registry.yarnpkg.com/@typescript-eslint/visitor-keys/-/visitor-keys-5.40.0.tgz#dd2d38097f68e0d2e1e06cb9f73c0173aca54b68" + integrity sha512-ijJ+6yig+x9XplEpG2K6FUdJeQGGj/15U3S56W9IqXKJqleuD7zJ2AX/miLezwxpd7ZxDAqO87zWufKg+RPZyQ== + dependencies: + "@typescript-eslint/types" "5.40.0" + eslint-visitor-keys "^3.3.0" + "@zenko/cloudserver@git+https://github.com/scality/cloudserver#8.6.8": version "8.6.8" resolved "git+https://github.com/scality/cloudserver#7fe0e2891bc0afcd09848ba639ba3c202dc8e46a" @@ -621,10 +1390,20 @@ acorn-jsx@^5.3.2: resolved "https://registry.yarnpkg.com/acorn-jsx/-/acorn-jsx-5.3.2.tgz#7ed5bb55908b3b2f1bc55c6af1653bada7f07937" integrity sha512-rq9s+JNhf0IChjtDXxllJ7g41oZk5SlXtp0LHwyA5cejwn7vKmKp4pPri6YEePv2PU65sAsegbXtIinmDFDXgQ== +acorn-walk@^8.1.1: + version "8.2.0" + resolved "https://registry.yarnpkg.com/acorn-walk/-/acorn-walk-8.2.0.tgz#741210f2e2426454508853a2f44d0ab83b7f69c1" + integrity sha512-k+iyHEuPgSw6SbuDpGQM+06HQUa04DZ3o+F6CSzXMvvI5KMvnaEqXe+YVe555R9nn6GPt404fos4wcgpw12SDA== + +acorn@^8.4.1: + version "8.8.1" + resolved "https://registry.yarnpkg.com/acorn/-/acorn-8.8.1.tgz#0a3f9cbecc4ec3bea6f0a80b66ae8dd2da250b73" + integrity sha512-7zFpHzhnqYKrkYdUjF1HI1bzd0VygEGX8lFk4k5zVMqHEoES+P+7TKI+EvLO9WVMJ8eekdO0aDEK044xTXwPPA== + acorn@^8.8.0: - version "8.8.2" - resolved "https://registry.yarnpkg.com/acorn/-/acorn-8.8.2.tgz#1b2f25db02af965399b9776b0c2c391276d37c4a" - integrity sha512-xjIYgE8HBrkpd/sJqOGNspf8uHG+NOHGOw6a/Urj8taM2EXfdNAH2oFcPeIFfsv3+kz/mJrS5VuMqbNLjCa2vw== + version "8.8.0" + resolved "https://registry.yarnpkg.com/acorn/-/acorn-8.8.0.tgz#88c0187620435c7f6015803f5539dae05a9dbea8" + integrity sha512-QOxyigPVrpZ2GXT+PFyZTl6TtOFc5egxHIP9IlQ+RbupQuX4RkT/Bee4/kQuC02Xkzg84JcT7oLYtDIQxp+v7w== after@0.8.2: version "0.8.2" @@ -682,6 +1461,13 @@ ajv@^6.10.0, ajv@^6.12.3, ajv@^6.12.4: json-schema-traverse "^0.4.1" uri-js "^4.2.2" +ansi-escapes@^4.2.1: + version "4.3.2" + resolved "https://registry.yarnpkg.com/ansi-escapes/-/ansi-escapes-4.3.2.tgz#6b2291d1db7d98b6521d5f1efa42d0f3a9feb65e" + integrity sha512-gKXj5ALrKWQLsYG9jlTRmR/xKluxHV+Z9QEwNIgCfM1/uwPMCuzVVnh5mwTd+OuBZcwSIMbqssNWRm1lE51QaQ== + dependencies: + type-fest "^0.21.3" + ansi-regex@^5.0.1: version "5.0.1" resolved "https://registry.yarnpkg.com/ansi-regex/-/ansi-regex-5.0.1.tgz#082cb2c89c9fe8659a311a53bd6a4dc5301db304" @@ -694,13 +1480,26 @@ ansi-styles@^3.2.1: dependencies: color-convert "^1.9.0" -ansi-styles@^4.1.0: +ansi-styles@^4.0.0, ansi-styles@^4.1.0: version "4.3.0" resolved "https://registry.yarnpkg.com/ansi-styles/-/ansi-styles-4.3.0.tgz#edd803628ae71c04c85ae7a0906edad34b648937" integrity sha512-zbB9rCJAT1rbjiVDb2hqKFHNYLxgtk8NURxZ3IZwD3F6NtxbXZQCnnSi1Lkx+IDohdPlFp222wVALIheZJQSEg== dependencies: color-convert "^2.0.1" +ansi-styles@^5.0.0: + version "5.2.0" + resolved "https://registry.yarnpkg.com/ansi-styles/-/ansi-styles-5.2.0.tgz#07449690ad45777d1924ac2abb2fc8895dba836b" + integrity sha512-Cxwpt2SfTzTtXcfOlzGEee8O+c+MmUgGrNiBcXnuWxuFJHe6a5Hz7qwhwe5OgaSYI0IJvkLqWX1ASG+cJOkEiA== + +anymatch@^3.0.3: + version "3.1.3" + resolved "https://registry.yarnpkg.com/anymatch/-/anymatch-3.1.3.tgz#790c58b19ba1720a84205b57c618d5ad8524973e" + integrity sha512-KMReFUr0B4t+D+OBkjR3KYqvocp2XaSzO55UcB6mgQMd3KbcE+mWTyvVV7D/zsdEbNnV6acZUutkiHQXvTr1Rw== + dependencies: + normalize-path "^3.0.0" + picomatch "^2.0.4" + "aproba@^1.0.3 || ^2.0.0": version "2.0.0" resolved "https://registry.yarnpkg.com/aproba/-/aproba-2.0.0.tgz#52520b8ae5b569215b354efc0caa3fe1e45a8adc" @@ -714,6 +1513,11 @@ are-we-there-yet@^3.0.0: delegates "^1.0.0" readable-stream "^3.6.0" +arg@^4.1.0: + version "4.1.3" + resolved "https://registry.yarnpkg.com/arg/-/arg-4.1.3.tgz#269fc7ad5b8e42cb63c896d5666017261c144089" + integrity sha512-58S9QDqG0Xx27YwPSt9fJxivjYl432YCwfDMfZ+71RAqUrZef7LrKQZ3LHLOwCS4FLNBplP533Zx895SeOCHvA== + argparse@^1.0.7: version "1.0.10" resolved "https://registry.yarnpkg.com/argparse/-/argparse-1.0.10.tgz#bcd6791ea5ae09725e17e5ad988134cd40b3d911" @@ -742,6 +1546,11 @@ array-includes@^3.1.4: get-intrinsic "^1.1.1" is-string "^1.0.7" +array-union@^2.1.0: + version "2.1.0" + resolved "https://registry.yarnpkg.com/array-union/-/array-union-2.1.0.tgz#b798420adbeb1de828d84acd8a2e23d3efe85e8d" + integrity sha512-HGyxoOTYUyCM6stUe6EJgnd4EoewAI7zMdfqO+kGjnlZmBDz/cR5pf8r/cR4Wq60sL/p0IkcjUEEPwS3GFrIyw== + array.prototype.flat@^1.2.5: version "1.3.0" resolved "https://registry.yarnpkg.com/array.prototype.flat/-/array.prototype.flat-1.3.0.tgz#0b0c1567bf57b38b56b4c97b8aa72ab45e4adc7b" @@ -1088,6 +1897,97 @@ azure-storage@~2.10.7: xml2js "~0.2.8" xmlbuilder "^9.0.7" +babel-jest@^29.1.2: + version "29.1.2" + resolved "https://registry.yarnpkg.com/babel-jest/-/babel-jest-29.1.2.tgz#540d3241925c55240fb0c742e3ffc5f33a501978" + integrity sha512-IuG+F3HTHryJb7gacC7SQ59A9kO56BctUsT67uJHp1mMCHUOMXpDwOHWGifWqdWVknN2WNkCVQELPjXx0aLJ9Q== + dependencies: + "@jest/transform" "^29.1.2" + "@types/babel__core" "^7.1.14" + babel-plugin-istanbul "^6.1.1" + babel-preset-jest "^29.0.2" + chalk "^4.0.0" + graceful-fs "^4.2.9" + slash "^3.0.0" + +babel-jest@^29.7.0: + version "29.7.0" + resolved "https://registry.yarnpkg.com/babel-jest/-/babel-jest-29.7.0.tgz#f4369919225b684c56085998ac63dbd05be020d5" + integrity sha512-BrvGY3xZSwEcCzKvKsCi2GgHqDqsYkOP4/by5xCgIwGXQxIEh+8ew3gmrE1y7XRR6LHZIj6yLYnUi/mm2KXKBg== + dependencies: + "@jest/transform" "^29.7.0" + "@types/babel__core" "^7.1.14" + babel-plugin-istanbul "^6.1.1" + babel-preset-jest "^29.6.3" + chalk "^4.0.0" + graceful-fs "^4.2.9" + slash "^3.0.0" + +babel-plugin-istanbul@^6.1.1: + version "6.1.1" + resolved "https://registry.yarnpkg.com/babel-plugin-istanbul/-/babel-plugin-istanbul-6.1.1.tgz#fa88ec59232fd9b4e36dbbc540a8ec9a9b47da73" + integrity sha512-Y1IQok9821cC9onCx5otgFfRm7Lm+I+wwxOx738M/WLPZ9Q42m4IG5W0FNX8WLL2gYMZo3JkuXIH2DOpWM+qwA== + dependencies: + "@babel/helper-plugin-utils" "^7.0.0" + "@istanbuljs/load-nyc-config" "^1.0.0" + "@istanbuljs/schema" "^0.1.2" + istanbul-lib-instrument "^5.0.4" + test-exclude "^6.0.0" + +babel-plugin-jest-hoist@^29.0.2: + version "29.0.2" + resolved "https://registry.yarnpkg.com/babel-plugin-jest-hoist/-/babel-plugin-jest-hoist-29.0.2.tgz#ae61483a829a021b146c016c6ad39b8bcc37c2c8" + integrity sha512-eBr2ynAEFjcebVvu8Ktx580BD1QKCrBG1XwEUTXJe285p9HA/4hOhfWCFRQhTKSyBV0VzjhG7H91Eifz9s29hg== + dependencies: + "@babel/template" "^7.3.3" + "@babel/types" "^7.3.3" + "@types/babel__core" "^7.1.14" + "@types/babel__traverse" "^7.0.6" + +babel-plugin-jest-hoist@^29.6.3: + version "29.6.3" + resolved "https://registry.yarnpkg.com/babel-plugin-jest-hoist/-/babel-plugin-jest-hoist-29.6.3.tgz#aadbe943464182a8922c3c927c3067ff40d24626" + integrity sha512-ESAc/RJvGTFEzRwOTT4+lNDk/GNHMkKbNzsvT0qKRfDyyYTskxB5rnU2njIDYVxXCBHHEI1c0YwHob3WaYujOg== + dependencies: + "@babel/template" "^7.3.3" + "@babel/types" "^7.3.3" + "@types/babel__core" "^7.1.14" + "@types/babel__traverse" "^7.0.6" + +babel-preset-current-node-syntax@^1.0.0: + version "1.0.1" + resolved "https://registry.yarnpkg.com/babel-preset-current-node-syntax/-/babel-preset-current-node-syntax-1.0.1.tgz#b4399239b89b2a011f9ddbe3e4f401fc40cff73b" + integrity sha512-M7LQ0bxarkxQoN+vz5aJPsLBn77n8QgTFmo8WK0/44auK2xlCXrYcUxHFxgU7qW5Yzw/CjmLRK2uJzaCd7LvqQ== + dependencies: + "@babel/plugin-syntax-async-generators" "^7.8.4" + "@babel/plugin-syntax-bigint" "^7.8.3" + "@babel/plugin-syntax-class-properties" "^7.8.3" + "@babel/plugin-syntax-import-meta" "^7.8.3" + "@babel/plugin-syntax-json-strings" "^7.8.3" + "@babel/plugin-syntax-logical-assignment-operators" "^7.8.3" + "@babel/plugin-syntax-nullish-coalescing-operator" "^7.8.3" + "@babel/plugin-syntax-numeric-separator" "^7.8.3" + "@babel/plugin-syntax-object-rest-spread" "^7.8.3" + "@babel/plugin-syntax-optional-catch-binding" "^7.8.3" + "@babel/plugin-syntax-optional-chaining" "^7.8.3" + "@babel/plugin-syntax-top-level-await" "^7.8.3" + +babel-preset-jest@^29.0.2: + version "29.0.2" + resolved "https://registry.yarnpkg.com/babel-preset-jest/-/babel-preset-jest-29.0.2.tgz#e14a7124e22b161551818d89e5bdcfb3b2b0eac7" + integrity sha512-BeVXp7rH5TK96ofyEnHjznjLMQ2nAeDJ+QzxKnHAAMs0RgrQsCywjAN8m4mOm5Di0pxU//3AoEeJJrerMH5UeA== + dependencies: + babel-plugin-jest-hoist "^29.0.2" + babel-preset-current-node-syntax "^1.0.0" + +babel-preset-jest@^29.6.3: + version "29.6.3" + resolved "https://registry.yarnpkg.com/babel-preset-jest/-/babel-preset-jest-29.6.3.tgz#fa05fa510e7d493896d7b0dd2033601c840f171c" + integrity sha512-0B3bhxR6snWXJZtR/RliHTDPRgn1sNHOR0yVtq/IiQFyuOVjFS+wuio/R4gSNkyYmKmJB4wGZv2NZanmKmTnNA== + dependencies: + babel-plugin-jest-hoist "^29.6.3" + babel-preset-current-node-syntax "^1.0.0" + backo2@1.0.2: version "1.0.2" resolved "https://registry.yarnpkg.com/backo2/-/backo2-1.0.2.tgz#31ab1ac8b129363463e35b3ebb69f4dfcfba7947" @@ -1195,6 +2095,13 @@ brace-expansion@^1.1.7: balanced-match "^1.0.0" concat-map "0.0.1" +braces@^3.0.2: + version "3.0.2" + resolved "https://registry.yarnpkg.com/braces/-/braces-3.0.2.tgz#3454e1a462ee8d599e236df336cd9ea4f8afe107" + integrity sha512-b8um+L1RzM3WDSzvhm6gIz1yfTbBt6YTlcEKAvsmqCZZFw46z626lVj9j1yEPW33H5H+lBQpZMP1k8l+78Ha0A== + dependencies: + fill-range "^7.0.1" + breakbeat@scality/breakbeat#v1.0.1: version "1.0.1" resolved "https://codeload.github.com/scality/breakbeat/tar.gz/c85885488597cb1f30ece57c8272525fd2c7ae8b" @@ -1216,6 +2123,30 @@ browserify-mime@^1.2.9: resolved "https://registry.yarnpkg.com/browserify-mime/-/browserify-mime-1.2.9.tgz#aeb1af28de6c0d7a6a2ce40adb68ff18422af31f" integrity sha512-uz+ItyJXBLb6wgon1ELEiVowJBEsy03PUWGRQU7cxxx9S+DW2hujPp+DaMYEOClRPzsn7NB99NtJ6pGnt8y+CQ== +browserslist@^4.21.9: + version "4.22.1" + resolved "https://registry.yarnpkg.com/browserslist/-/browserslist-4.22.1.tgz#ba91958d1a59b87dab6fed8dfbcb3da5e2e9c619" + integrity sha512-FEVc202+2iuClEhZhrWy6ZiAcRLvNMyYcxZ8raemul1DYVOVdFsbqckWLdsixQZCpJlwe77Z3UTalE7jsjnKfQ== + dependencies: + caniuse-lite "^1.0.30001541" + electron-to-chromium "^1.4.535" + node-releases "^2.0.13" + update-browserslist-db "^1.0.13" + +bs-logger@0.x: + version "0.2.6" + resolved "https://registry.yarnpkg.com/bs-logger/-/bs-logger-0.2.6.tgz#eb7d365307a72cf974cc6cda76b68354ad336bd8" + integrity sha512-pd8DCoxmbgc7hyPKOvxtqNcjYoOsABPQdcCUjGp3d42VR2CX1ORhk2A87oqqu5R1kk+76nsxZupkmyd+MVtCog== + dependencies: + fast-json-stable-stringify "2.x" + +bser@2.1.1: + version "2.1.1" + resolved "https://registry.yarnpkg.com/bser/-/bser-2.1.1.tgz#e6787da20ece9d07998533cfd9de6f5c38f4bc05" + integrity sha512-gQxTNE/GAfIIrmHLUE3oJyp5FO6HRBfhjnw4/wMmA63ZGDJnWBmgY/lyQBpnDUkGmAhbSe39tx2d/iTOAfglwQ== + dependencies: + node-int64 "^0.4.0" + bson@4.0.0: version "4.0.0" resolved "https://registry.yarnpkg.com/bson/-/bson-4.0.0.tgz#7684b512e9ae59fe3518ff2b35ec0c9741bf9f57" @@ -1260,6 +2191,11 @@ buffer-equal-constant-time@1.0.1: resolved "https://registry.yarnpkg.com/buffer-equal-constant-time/-/buffer-equal-constant-time-1.0.1.tgz#f8e71132f7ffe6e01a5c9697a4c6f3e48d5cc819" integrity sha512-zRpUiDwd/xk6ADqPMATG8vc9VPrkck7T07OIx0gnjmJAnHnTVXNQG3vfvWNuiZIkwu9KrKdA1iJKfsfTVxE6NA== +buffer-from@^1.0.0: + version "1.1.2" + resolved "https://registry.yarnpkg.com/buffer-from/-/buffer-from-1.1.2.tgz#2b146a6fd72e80b4f55d255f35ed59a3a9a41bd5" + integrity sha512-E+XQCRwSbaaiChtv6k6Dwgc+bx+Bs6vuKJHHl5kox/BaKbhiXzqQOwK4cO22yElGp2OCmjwVhT3HmxgyPGnJfQ== + buffer-shims@~1.0.0: version "1.0.0" resolved "https://registry.yarnpkg.com/buffer-shims/-/buffer-shims-1.0.0.tgz#9978ce317388c649ad8793028c3477ef044a8b51" @@ -1351,12 +2287,27 @@ callsites@^3.0.0: resolved "https://registry.yarnpkg.com/callsites/-/callsites-3.1.0.tgz#b3630abd8943432f54b3f0519238e33cd7df2f73" integrity sha512-P8BjAsXvZS+VIDUI11hHCQEv74YT67YUi5JJFNWIqL235sBmjX4+qx9Muvls5ivyNENctx46xQLQ3aTuE7ssaQ== +camelcase@^5.3.1: + version "5.3.1" + resolved "https://registry.yarnpkg.com/camelcase/-/camelcase-5.3.1.tgz#e3c9b31569e106811df242f715725a1f4c494320" + integrity sha512-L28STB170nwWS63UjtlEOE3dldQApaJXZkOI1uMFfzf3rRuPegHaHesyee+YxQ+W6SvRDQV6UrdOdRiR153wJg== + +camelcase@^6.2.0: + version "6.3.0" + resolved "https://registry.yarnpkg.com/camelcase/-/camelcase-6.3.0.tgz#5685b95eb209ac9c0c177467778c9c84df58ba9a" + integrity sha512-Gmy6FhYlCY7uOElZUSbxo2UCDH8owEk996gkbrpsgGtrJLM3J7jGxl9Ic7Qwwj4ivOE5AWZWRMecDdF7hqGjFA== + +caniuse-lite@^1.0.30001541: + version "1.0.30001561" + resolved "https://registry.yarnpkg.com/caniuse-lite/-/caniuse-lite-1.0.30001561.tgz#752f21f56f96f1b1a52e97aae98c57c562d5d9da" + integrity sha512-NTt0DNoKe958Q0BE0j0c1V9jbUzhBxHIEJy7asmGrpE0yG63KTV7PLHPnK2E1O9RsQrQ081I3NLuXGS6zht3cw== + caseless@~0.12.0: version "0.12.0" resolved "https://registry.yarnpkg.com/caseless/-/caseless-0.12.0.tgz#1b681c21ff84033c826543090689420d187151dc" integrity sha512-4tYFyifaFfGacoiObjJegolkwSU4xQNGbVgUiNYVUxbQ2x2lUsFvY4hVgVzGiIe6WLOPqycWXA40l+PWsxthUw== -chalk@^2.4.1: +chalk@^2.0.0, chalk@^2.4.1, chalk@^2.4.2: version "2.4.2" resolved "https://registry.yarnpkg.com/chalk/-/chalk-2.4.2.tgz#cd42541677a54333cf541a49108c1432b44c9424" integrity sha512-Mti+f9lpJNcwF4tWV8/OrTTtF1gZi+f8FqlyAdouralcFWFQWF2+NgCHShjkCb+IFBLq9buZwE1xckQU4peSuQ== @@ -1373,6 +2324,11 @@ chalk@^4.0.0: ansi-styles "^4.1.0" supports-color "^7.1.0" +char-regex@^1.0.2: + version "1.0.2" + resolved "https://registry.yarnpkg.com/char-regex/-/char-regex-1.0.2.tgz#d744358226217f981ed58f479b1d6bcc29545dcf" + integrity sha512-kWWXztvZ5SBQV+eRgKFeh8q5sLuZY2+8WUIzlxWVTg+oGwY14qylx1KbKzHd8P6ZYkAg0xyIDU9JMHhyJMZ1jw== + charenc@~0.0.1: version "0.0.2" resolved "https://registry.yarnpkg.com/charenc/-/charenc-0.0.2.tgz#c0a1d2f3a7092e03774bfa83f14c0fc5790a8667" @@ -1383,11 +2339,30 @@ chownr@^2.0.0: resolved "https://registry.yarnpkg.com/chownr/-/chownr-2.0.0.tgz#15bfbe53d2eab4cf70f18a8cd68ebe5b3cb1dece" integrity sha512-bIomtDF5KGpdogkLd9VspvFzk9KfpyyGlS8YFVZl7TGPBHL5snIOnxeshwVgPteQ9b4Eydl+pVbIyE1DcvCWgQ== +ci-info@^3.2.0: + version "3.5.0" + resolved "https://registry.yarnpkg.com/ci-info/-/ci-info-3.5.0.tgz#bfac2a29263de4c829d806b1ab478e35091e171f" + integrity sha512-yH4RezKOGlOhxkmhbeNuC4eYZKAUsEaGtBuBzDDP1eFUKiccDWzBABxBfOx31IDwDIXMTxWuwAxUGModvkbuVw== + +cjs-module-lexer@^1.0.0: + version "1.2.2" + resolved "https://registry.yarnpkg.com/cjs-module-lexer/-/cjs-module-lexer-1.2.2.tgz#9f84ba3244a512f3a54e5277e8eef4c489864e40" + integrity sha512-cOU9usZw8/dXIXKtwa8pM0OTJQuJkxMN6w30csNRUerHfeQ5R6U3kkU/FtJeIf3M202OHfY2U8ccInBG7/xogA== + clean-stack@^2.0.0: version "2.2.0" resolved "https://registry.yarnpkg.com/clean-stack/-/clean-stack-2.2.0.tgz#ee8472dbb129e727b31e8a10a427dee9dfe4008b" integrity sha512-4diC9HaTE+KRAMWhDhrGOECgWZxoevMc5TlkObMqNSsVU62PYzXZ/SMTjzyGAFF1YusgxGcSWTEXBhp0CPwQ1A== +cliui@^8.0.1: + version "8.0.1" + resolved "https://registry.yarnpkg.com/cliui/-/cliui-8.0.1.tgz#0c04b075db02cbfe60dc8e6cf2f5486b1a3608aa" + integrity sha512-BSeNnyus75C4//NQ9gQt1/csTXyo/8Sb+afLAkzAptFuMsod9HFokGNudZpi/oQV73hnVK+sR+5PVRMd+Dr7YQ== + dependencies: + string-width "^4.2.0" + strip-ansi "^6.0.1" + wrap-ansi "^7.0.0" + clone@^2.1.2: version "2.1.2" resolved "https://registry.yarnpkg.com/clone/-/clone-2.1.2.tgz#1b7f4b9f591f1e8f83670401600345a02887435f" @@ -1398,6 +2373,16 @@ cluster-key-slot@^1.1.0: resolved "https://registry.yarnpkg.com/cluster-key-slot/-/cluster-key-slot-1.1.0.tgz#30474b2a981fb12172695833052bc0d01336d10d" integrity sha512-2Nii8p3RwAPiFwsnZvukotvow2rIHM+yQ6ZcBXGHdniadkYGZYiGmkHJIbZPIV9nfv7m/U1IPMVVcAhoWFeklw== +co@^4.6.0: + version "4.6.0" + resolved "https://registry.yarnpkg.com/co/-/co-4.6.0.tgz#6ea6bdf3d853ae54ccb8e47bfa0bf3f9031fb184" + integrity sha512-QVb0dM5HvG+uaxitm8wONl7jltx8dqhfU33DcqtOZcLSVIKSDDLDi7+0LbAKiyI8hD9u42m2YxXSkMGWThaecQ== + +collect-v8-coverage@^1.0.0: + version "1.0.1" + resolved "https://registry.yarnpkg.com/collect-v8-coverage/-/collect-v8-coverage-1.0.1.tgz#cc2c8e94fc18bbdffe64d6534570c8a673b27f59" + integrity sha512-iBPtljfCNcTKNAto0KEtDfZ3qzjJvqE3aTGZsbhjSBlorqpXJlaWWtPO35D+ZImoC3KWejX64o+yPGxhWSTzfg== + color-convert@^1.9.0, color-convert@^1.9.3: version "1.9.3" resolved "https://registry.yarnpkg.com/color-convert/-/color-convert-1.9.3.tgz#bb71850690e1f136567de629d2d5471deda4c1e8" @@ -1532,6 +2517,16 @@ content-type@~1.0.4: resolved "https://registry.yarnpkg.com/content-type/-/content-type-1.0.4.tgz#e138cc75e040c727b1966fe5e5f8c9aee256fe3b" integrity sha512-hIP3EEPs8tB9AT1L+NUqtwOAps4mk2Zob89MWXMHjHWg9milF/j4osnnQLXBCBFBk/tvIG/tUc9mOUJiPBhPXA== +convert-source-map@^1.6.0: + version "1.9.0" + resolved "https://registry.yarnpkg.com/convert-source-map/-/convert-source-map-1.9.0.tgz#7faae62353fb4213366d0ca98358d22e8368b05f" + integrity sha512-ASFBup0Mz1uyiIjANan1jzLQami9z1PoYSZCiiYW2FczPbenXc45FZdBZLzOT+r6+iciuEModtmCti+hjaAk0A== + +convert-source-map@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/convert-source-map/-/convert-source-map-2.0.0.tgz#4b560f649fc4e918dd0ab75cf4961e8bc882d82a" + integrity sha512-Kvp459HrV2FEJ1CAsi1Ku+MY3kasH19TFykTz2xWmMeq6bk2NU3XXvfJ+Q61m0xktWwt+1HSYf3JZsTms3aRJg== + cookie-signature@1.0.6: version "1.0.6" resolved "https://registry.yarnpkg.com/cookie-signature/-/cookie-signature-1.0.6.tgz#e303a882b342cc3ee8ca513a79999734dab3ae2c" @@ -1565,6 +2560,11 @@ cors@~2.8.5: object-assign "^4" vary "^1" +create-require@^1.1.0: + version "1.1.1" + resolved "https://registry.yarnpkg.com/create-require/-/create-require-1.1.1.tgz#c1d7e8f1e5f6cfc9ff65f9cd352d37348756c333" + integrity sha512-dcKFX3jn0MpIaXjisoRvexIJVEKzaq7z2rZKxf+MSr9TkdmHmsU4m2lcLojrj/FHl8mk5VxMmYA+ftRkP/3oKQ== + cron-parser@^2.11.0, cron-parser@^2.15.0, cron-parser@^2.18.0: version "2.18.0" resolved "https://registry.yarnpkg.com/cron-parser/-/cron-parser-2.18.0.tgz#de1bb0ad528c815548371993f81a54e5a089edcf" @@ -1584,7 +2584,7 @@ cross-spawn@^6.0.5: shebang-command "^1.2.0" which "^1.2.9" -cross-spawn@^7.0.2: +cross-spawn@^7.0.2, cross-spawn@^7.0.3: version "7.0.3" resolved "https://registry.yarnpkg.com/cross-spawn/-/cross-spawn-7.0.3.tgz#f73a85b9d5d41d045551c177e2882d4ac85728a6" integrity sha512-iRDPJKUPVEND7dHPO8rkbOnPpyDygcDFtWjpeWNCgy8WP2rXcxXL8TskReQl6OrB2G7+UJrags1q15Fudc7G6w== @@ -1637,7 +2637,7 @@ debug@2.6.9, debug@^2.6.9: dependencies: ms "2.0.0" -debug@4, debug@^4.1.0, debug@^4.1.1, debug@^4.3.1, debug@^4.3.2, debug@^4.3.3, debug@~4.3.1, debug@~4.3.2: +debug@4, debug@^4.1.0, debug@^4.1.1, debug@^4.3.1, debug@^4.3.2, debug@^4.3.3, debug@^4.3.4, debug@~4.3.1, debug@~4.3.2: version "4.3.4" resolved "https://registry.yarnpkg.com/debug/-/debug-4.3.4.tgz#1319f6579357f2338d3337d2cdd4914bb5dcc865" integrity sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ== @@ -1665,11 +2665,26 @@ debug@~4.1.0: dependencies: ms "^2.1.1" +dedent@^0.7.0: + version "0.7.0" + resolved "https://registry.yarnpkg.com/dedent/-/dedent-0.7.0.tgz#2495ddbaf6eb874abb0e1be9df22d2e5a544326c" + integrity sha512-Q6fKUPqnAHAyhiUgFU7BUzLiv0kd8saH9al7tnu5Q/okj6dnupxyTgFIBjVzJATdfIAm9NAsvXNzjaKa+bxVyA== + +dedent@^1.0.0: + version "1.5.1" + resolved "https://registry.yarnpkg.com/dedent/-/dedent-1.5.1.tgz#4f3fc94c8b711e9bb2800d185cd6ad20f2a90aff" + integrity sha512-+LxW+KLWxu3HW3M2w2ympwtqPrqYRzU8fqi6Fhd18fBALe15blJPI/I4+UHveMVG6lJqB4JNd4UG0S5cnVHwIg== + deep-is@^0.1.3: version "0.1.4" resolved "https://registry.yarnpkg.com/deep-is/-/deep-is-0.1.4.tgz#a6f2dce612fadd2ef1f519b73551f17e85199831" integrity sha512-oIPzksmTg4/MriiaYGO+okXDT7ztn/w3Eptv/+gSIdMdKsJo0u4CfYNFJPy+4SKMuCqGw2wxnA+URMg3t8a/bQ== +deepmerge@^4.2.2: + version "4.2.2" + resolved "https://registry.yarnpkg.com/deepmerge/-/deepmerge-4.2.2.tgz#44d2ea3679b8f4d4ffba33f03d865fc1e7bf4955" + integrity sha512-FJ3UgI4gIl+PHZm53knsuSFpE+nESMr7M4v9QcgB7S63Kj/6WqMiFQJpBBYz1Pt+66bZpP3Q7Lye0Oo9MPKEdg== + deferred-leveldown@~0.2.0: version "0.2.0" resolved "https://registry.yarnpkg.com/deferred-leveldown/-/deferred-leveldown-0.2.0.tgz#2cef1f111e1c57870d8bbb8af2650e587cd2f5b4" @@ -1728,16 +2743,38 @@ destroy@1.2.0: resolved "https://registry.yarnpkg.com/destroy/-/destroy-1.2.0.tgz#4803735509ad8be552934c67df614f94e66fa015" integrity sha512-2sJGJTaXIIaR1w4iJSNoN0hnMY7Gpc/n8D4qSCJw8QqFWXf7cuAgnEHxBpweaVcPevC2l3KpjYCx3NypQQgaJg== +detect-newline@^3.0.0: + version "3.1.0" + resolved "https://registry.yarnpkg.com/detect-newline/-/detect-newline-3.1.0.tgz#576f5dfc63ae1a192ff192d8ad3af6308991b651" + integrity sha512-TLz+x/vEXm/Y7P7wn1EJFNLxYpUD4TgMosxY6fAVJUnJMbupHBOncxyWUG9OpTaH9EBD7uFI5LfEgmMOc54DsA== + +diff-sequences@^29.0.0: + version "29.0.0" + resolved "https://registry.yarnpkg.com/diff-sequences/-/diff-sequences-29.0.0.tgz#bae49972ef3933556bcb0800b72e8579d19d9e4f" + integrity sha512-7Qe/zd1wxSDL4D/X/FPjOMB+ZMDt71W94KYaq05I2l0oQqgXgs7s4ftYYmV38gBSrPz2vcygxfs1xn0FT+rKNA== + +diff-sequences@^29.6.3: + version "29.6.3" + resolved "https://registry.yarnpkg.com/diff-sequences/-/diff-sequences-29.6.3.tgz#4deaf894d11407c51efc8418012f9e70b84ea921" + integrity sha512-EjePK1srD3P08o2j4f0ExnylqRs5B9tJjcp9t1krH2qRi8CCdsYfwe9JgSLurFBWwq4uOlipzfk5fHNvwFKr8Q== + diff@3.2.0: version "3.2.0" resolved "https://registry.yarnpkg.com/diff/-/diff-3.2.0.tgz#c9ce393a4b7cbd0b058a725c93df299027868ff9" integrity sha512-597ykPFhtJYaXqPq6fF7Vl1fXTKgPdLOntyxpmdzUOKiYGqK7zcnbplj5088+8qJnWdzXhyeau5iVr8HVo9dgg== -diff@^4.0.2: +diff@^4.0.1, diff@^4.0.2: version "4.0.2" resolved "https://registry.yarnpkg.com/diff/-/diff-4.0.2.tgz#60f3aecb89d5fae520c11aa19efc2bb982aade7d" integrity sha512-58lmxKSA4BNyLz+HHMUzlOEpg09FV+ev6ZMe3vJihgdxzgcwZ8VoEEPmALCZG9LmqfVoNMMKpttIYTVG6uDY7A== +dir-glob@^3.0.1: + version "3.0.1" + resolved "https://registry.yarnpkg.com/dir-glob/-/dir-glob-3.0.1.tgz#56dbf73d992a4a93ba1584f4534063fd2e41717f" + integrity sha512-WkrWp9GR4KXfKGYzOLmTuGVi1UWFfws377n9cc55/tb6DuqyF6pcQ5AbiHEshaDpY9v6oaSr2XCDidGmMwdzIA== + dependencies: + path-type "^4.0.0" + diskusage@1.1.3, diskusage@^1.1.1, diskusage@^1.1.3: version "1.1.3" resolved "https://registry.yarnpkg.com/diskusage/-/diskusage-1.1.3.tgz#680d7dbf1b679168a195c9240eb3552cbd2c067b" @@ -1780,6 +2817,21 @@ ee-first@1.1.1: resolved "https://registry.yarnpkg.com/ee-first/-/ee-first-1.1.1.tgz#590c61156b0ae2f4f0255732a158b266bc56b21d" integrity sha512-WMwm9LhRUo+WUaRN+vRuETqG89IgZphVSNkdFgeb6sS/E4OrDIN7t48CAewSHXc6C8lefD8KKfr5vY61brQlow== +electron-to-chromium@^1.4.535: + version "1.4.576" + resolved "https://registry.yarnpkg.com/electron-to-chromium/-/electron-to-chromium-1.4.576.tgz#0c6940fdc0d60f7e34bd742b29d8fa847c9294d1" + integrity sha512-yXsZyXJfAqzWk1WKryr0Wl0MN2D47xodPvEEwlVePBnhU5E7raevLQR+E6b9JAD3GfL/7MbAL9ZtWQQPcLx7wA== + +emittery@^0.10.2: + version "0.10.2" + resolved "https://registry.yarnpkg.com/emittery/-/emittery-0.10.2.tgz#902eec8aedb8c41938c46e9385e9db7e03182933" + integrity sha512-aITqOwnLanpHLNXZJENbOgjUBeHocD+xsSJmNrjovKBW5HbSpW3d1pEls7GFQPUWXiwG9+0P4GtHfEqC/4M0Iw== + +emittery@^0.13.1: + version "0.13.1" + resolved "https://registry.yarnpkg.com/emittery/-/emittery-0.13.1.tgz#c04b8c3457490e0847ae51fced3af52d338e3dad" + integrity sha512-DeWwawk6r5yR9jFgnDKYt4sLS0LmHJJi3ZOnb5/JdbYwj3nW+FxQnHIjhBKz8YLC7oRNPVM9NQ47I3CVx34eqQ== + emoji-regex@^8.0.0: version "8.0.0" resolved "https://registry.yarnpkg.com/emoji-regex/-/emoji-regex-8.0.0.tgz#e818fd69ce5ccfcb404594f842963bf53164cc37" @@ -2001,6 +3053,11 @@ es6-symbol@^3.1.0, es6-symbol@^3.1.1, es6-symbol@^3.1.3: d "^1.0.1" ext "^1.1.2" +escalade@^3.1.1: + version "3.1.1" + resolved "https://registry.yarnpkg.com/escalade/-/escalade-3.1.1.tgz#d8cfdc7000965c5a0174b4a82eaa5c0552742e40" + integrity sha512-k0er2gUkLf8O0zKJiAhmkTnJlTvINGv7ygDNPbeIsX/TJjGJZHuh9B2UxbsaEkmlEo9MfhrSzmhIlhRlI2GXnw== + escape-html@~1.0.3: version "1.0.3" resolved "https://registry.yarnpkg.com/escape-html/-/escape-html-1.0.3.tgz#0258eae4d3d0c0974de1c169188ef0051d1d1988" @@ -2011,6 +3068,11 @@ escape-string-regexp@1.0.5, escape-string-regexp@^1.0.5: resolved "https://registry.yarnpkg.com/escape-string-regexp/-/escape-string-regexp-1.0.5.tgz#1b61c0562190a8dff6ae3bb2cf0200ca130b86d4" integrity sha512-vbRorB5FUQWvla16U8R/qgaFIya2qGzwDrNmCZuYKrbdSUMG6I1ZCGQRefkRVhuOkIGVne7BQ35DSfo1qvJqFg== +escape-string-regexp@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/escape-string-regexp/-/escape-string-regexp-2.0.0.tgz#a30304e99daa32e23b2fd20f51babd07cffca344" + integrity sha512-UpzcLCXolUWcNu5HtVMHYdXJjArjsF9C0aNnquZYY4uW/Vu0miy5YoWvbV345HauVvcAUnpRuhMMcqTcGOY2+w== + escape-string-regexp@^4.0.0: version "4.0.0" resolved "https://registry.yarnpkg.com/escape-string-regexp/-/escape-string-regexp-4.0.0.tgz#14ba83a5d373e3d311e5afca29cf5bfad965bf34" @@ -2067,6 +3129,29 @@ eslint-plugin-import@^2.14.0: resolve "^1.22.0" tsconfig-paths "^3.14.1" +eslint-plugin-jest@^27.1.1: + version "27.1.1" + resolved "https://registry.yarnpkg.com/eslint-plugin-jest/-/eslint-plugin-jest-27.1.1.tgz#60f68dee15d4ffd9cdff65158d7fa46a65dbaaf5" + integrity sha512-vuSuXGKHHi/UAffIM46QKm4g0tQP+6n52nRxUpMq6x6x9rhnv5WM7ktSu3h9cTnXE4b0Y0ODQTgRlCm9rdRLvg== + dependencies: + "@typescript-eslint/utils" "^5.10.0" + +eslint-plugin-mocha@^10.2.0: + version "10.2.0" + resolved "https://registry.yarnpkg.com/eslint-plugin-mocha/-/eslint-plugin-mocha-10.2.0.tgz#15b05ce5be4b332bb0d76826ec1c5ebf67102ad6" + integrity sha512-ZhdxzSZnd1P9LqDPF0DBcFLpRIGdh1zkF2JHnQklKQOvrQtT73kdP5K9V2mzvbLR+cCAO9OI48NXK/Ax9/ciCQ== + dependencies: + eslint-utils "^3.0.0" + rambda "^7.4.0" + +eslint-scope@^5.1.1: + version "5.1.1" + resolved "https://registry.yarnpkg.com/eslint-scope/-/eslint-scope-5.1.1.tgz#e786e59a66cb92b3f6c1fb0d508aab174848f48c" + integrity sha512-2NxwbF/hZ0KpepYN0cNbo+FN6XoK7GaHlQhgx/hIZl6Va0bF45RQOOwhLIy8lQDbuCiadSLCBnH2CFYquit5bw== + dependencies: + esrecurse "^4.3.0" + estraverse "^4.1.1" + eslint-scope@^7.1.1: version "7.1.1" resolved "https://registry.yarnpkg.com/eslint-scope/-/eslint-scope-7.1.1.tgz#fff34894c2f65e5226d3041ac480b4513a163642" @@ -2092,16 +3177,14 @@ eslint-visitor-keys@^3.3.0: resolved "https://registry.yarnpkg.com/eslint-visitor-keys/-/eslint-visitor-keys-3.3.0.tgz#f6480fa6b1f30efe2d1968aa8ac745b862469826" integrity sha512-mQ+suqKJVyeuwGYHAdjMFqjCyfl8+Ldnxuyp3ldiMBFKkvytrXUZWaiPCEav8qDHKty44bD+qV1IP4T+w+xXRA== -eslint@^8.14.0: - version "8.35.0" - resolved "https://registry.yarnpkg.com/eslint/-/eslint-8.35.0.tgz#fffad7c7e326bae606f0e8f436a6158566d42323" - integrity sha512-BxAf1fVL7w+JLRQhWl2pzGeSiGqbWumV4WNvc9Rhp6tiCtm4oHnyPBSEtMGZwrQgudFQ+otqzWoPB7x+hxoWsw== +eslint@^8.25.0: + version "8.25.0" + resolved "https://registry.yarnpkg.com/eslint/-/eslint-8.25.0.tgz#00eb962f50962165d0c4ee3327708315eaa8058b" + integrity sha512-DVlJOZ4Pn50zcKW5bYH7GQK/9MsoQG2d5eDH0ebEkE8PbgzTTmtt/VTH9GGJ4BfeZCpBLqFfvsjX35UacUL83A== dependencies: - "@eslint/eslintrc" "^2.0.0" - "@eslint/js" "8.35.0" - "@humanwhocodes/config-array" "^0.11.8" + "@eslint/eslintrc" "^1.3.3" + "@humanwhocodes/config-array" "^0.10.5" "@humanwhocodes/module-importer" "^1.0.1" - "@nodelib/fs.walk" "^1.2.8" ajv "^6.10.0" chalk "^4.0.0" cross-spawn "^7.0.2" @@ -2112,19 +3195,19 @@ eslint@^8.14.0: eslint-utils "^3.0.0" eslint-visitor-keys "^3.3.0" espree "^9.4.0" - esquery "^1.4.2" + esquery "^1.4.0" esutils "^2.0.2" fast-deep-equal "^3.1.3" file-entry-cache "^6.0.1" find-up "^5.0.0" - glob-parent "^6.0.2" - globals "^13.19.0" + glob-parent "^6.0.1" + globals "^13.15.0" + globby "^11.1.0" grapheme-splitter "^1.0.4" ignore "^5.2.0" import-fresh "^3.0.0" imurmurhash "^0.1.4" is-glob "^4.0.0" - is-path-inside "^3.0.3" js-sdsl "^4.1.4" js-yaml "^4.1.0" json-stable-stringify-without-jsonify "^1.0.1" @@ -2139,9 +3222,9 @@ eslint@^8.14.0: text-table "^0.2.0" espree@^9.4.0: - version "9.4.1" - resolved "https://registry.yarnpkg.com/espree/-/espree-9.4.1.tgz#51d6092615567a2c2cff7833445e37c28c0065bd" - integrity sha512-XwctdmTO6SIvCzd9810yyNzIrOrqNYV9Koizx4C/mRhf9uq0o4yHoCEU/670pOxOL/MSraektvSAji79kX90Vg== + version "9.4.0" + resolved "https://registry.yarnpkg.com/espree/-/espree-9.4.0.tgz#cd4bc3d6e9336c433265fc0aa016fc1aaf182f8a" + integrity sha512-DQmnRpLj7f6TgN/NYb0MTzJXL+vJF9h3pHy4JhCIs3zwcgez8xmGg3sXHcEO97BrmO2OSvCwMdfdlyl+E9KjOw== dependencies: acorn "^8.8.0" acorn-jsx "^5.3.2" @@ -2152,10 +3235,10 @@ esprima@^4.0.0: resolved "https://registry.yarnpkg.com/esprima/-/esprima-4.0.1.tgz#13b04cdb3e6c5d19df91ab6987a8695619b0aa71" integrity sha512-eGuFFw7Upda+g4p+QHvnW0RyTX/SVeJBDM/gCtMARO0cLuT2HcEKnTPvhjV6aGeqrCB/sbNop0Kszm0jsaWU4A== -esquery@^1.4.2: - version "1.5.0" - resolved "https://registry.yarnpkg.com/esquery/-/esquery-1.5.0.tgz#6ce17738de8577694edd7361c57182ac8cb0db0b" - integrity sha512-YQLXUplAwJgCydQ78IMJywZCceoqk1oH01OERdSAJc/7U2AylwjhSCLDEtqwg811idIS/9fIU5GjG73IgjKMVg== +esquery@^1.4.0: + version "1.4.0" + resolved "https://registry.yarnpkg.com/esquery/-/esquery-1.4.0.tgz#2148ffc38b82e8c7057dfed48425b3e61f0f24a5" + integrity sha512-cCDispWt5vHHtwMY2YrAQ4ibFkAL8RbH5YGBnZBc90MolvvfkkQcJro/aZiAQUlQ3qgrYS6D6v8Gc5G5CQsc9w== dependencies: estraverse "^5.1.0" @@ -2166,6 +3249,11 @@ esrecurse@^4.3.0: dependencies: estraverse "^5.2.0" +estraverse@^4.1.1: + version "4.3.0" + resolved "https://registry.yarnpkg.com/estraverse/-/estraverse-4.3.0.tgz#398ad3f3c5a24948be7725e83d11a7de28cdbd1d" + integrity sha512-39nnKffWz8xN1BU/2c79n9nB9HDzo0niYUqx6xyqUnyoAnQyyWpOTdZEeiCch8BBu515t4wp9ZmgVfVhn9EBpw== + estraverse@^5.1.0, estraverse@^5.2.0: version "5.3.0" resolved "https://registry.yarnpkg.com/estraverse/-/estraverse-5.3.0.tgz#2eea5290702f26ab8fe5370370ff86c965d21123" @@ -2201,6 +3289,48 @@ events@^3.0.0: resolved "https://registry.yarnpkg.com/events/-/events-3.3.0.tgz#31a95ad0a924e2d2c419a813aeb2c4e878ea7400" integrity sha512-mQw+2fkQbALzQ7V0MY0IqdnXNOeTtP4r0lN9z7AAawCXgqea7bDii20AYrIBrFd/Hx0M2Ocz6S111CaFkUcb0Q== +execa@^5.0.0: + version "5.1.1" + resolved "https://registry.yarnpkg.com/execa/-/execa-5.1.1.tgz#f80ad9cbf4298f7bd1d4c9555c21e93741c411dd" + integrity sha512-8uSpZZocAZRBAPIEINJj3Lo9HyGitllczc27Eh5YYojjMFMn8yHMDMaUHE2Jqfq05D/wucwI4JGURyXt1vchyg== + dependencies: + cross-spawn "^7.0.3" + get-stream "^6.0.0" + human-signals "^2.1.0" + is-stream "^2.0.0" + merge-stream "^2.0.0" + npm-run-path "^4.0.1" + onetime "^5.1.2" + signal-exit "^3.0.3" + strip-final-newline "^2.0.0" + +exit@^0.1.2: + version "0.1.2" + resolved "https://registry.yarnpkg.com/exit/-/exit-0.1.2.tgz#0632638f8d877cc82107d30a0fff1a17cba1cd0c" + integrity sha512-Zk/eNKV2zbjpKzrsQ+n1G6poVbErQxJ0LBOJXaKZ1EViLzH+hrLu9cdXI4zw9dBQJslwBEpbQ2P1oS7nDxs6jQ== + +expect@^29.1.2: + version "29.1.2" + resolved "https://registry.yarnpkg.com/expect/-/expect-29.1.2.tgz#82f8f28d7d408c7c68da3a386a490ee683e1eced" + integrity sha512-AuAGn1uxva5YBbBlXb+2JPxJRuemZsmlGcapPXWNSBNsQtAULfjioREGBWuI0EOvYUKjDnrCy8PW5Zlr1md5mw== + dependencies: + "@jest/expect-utils" "^29.1.2" + jest-get-type "^29.0.0" + jest-matcher-utils "^29.1.2" + jest-message-util "^29.1.2" + jest-util "^29.1.2" + +expect@^29.7.0: + version "29.7.0" + resolved "https://registry.yarnpkg.com/expect/-/expect-29.7.0.tgz#578874590dcb3214514084c08115d8aee61e11bc" + integrity sha512-2Zks0hf1VLFYI1kbh0I5jP3KHHyCHpkfyHBzsSXRFgl/Bg9mWYfMW8oD+PdMPlEwy5HNsR9JutYy6pMeOh61nw== + dependencies: + "@jest/expect-utils" "^29.7.0" + jest-get-type "^29.6.3" + jest-matcher-utils "^29.7.0" + jest-message-util "^29.7.0" + jest-util "^29.7.0" + express@^4.17.1: version "4.18.1" resolved "https://registry.yarnpkg.com/express/-/express-4.18.1.tgz#7797de8b9c72c857b9cd0e14a5eea80666267caf" @@ -2265,7 +3395,18 @@ fast-deep-equal@^3.1.1, fast-deep-equal@^3.1.3: resolved "https://registry.yarnpkg.com/fast-deep-equal/-/fast-deep-equal-3.1.3.tgz#3a7d56b559d6cbc3eb512325244e619a65c6c525" integrity sha512-f3qQ9oQy9j2AhBe/H9VC91wLmKBCCU/gDOnKNAYG5hswO7BLKj09Hc5HYNz9cGI++xlpDCIgDaitVs03ATR84Q== -fast-json-stable-stringify@^2.0.0: +fast-glob@^3.2.9: + version "3.2.12" + resolved "https://registry.yarnpkg.com/fast-glob/-/fast-glob-3.2.12.tgz#7f39ec99c2e6ab030337142da9e0c18f37afae80" + integrity sha512-DVj4CQIYYow0BlaelwK1pHl5n5cRSJfM60UA0zK891sVInoPri2Ekj7+e1CT3/3qxXenpI+nBBmQAcJPJgaj4w== + dependencies: + "@nodelib/fs.stat" "^2.0.2" + "@nodelib/fs.walk" "^1.2.3" + glob-parent "^5.1.2" + merge2 "^1.3.0" + micromatch "^4.0.4" + +fast-json-stable-stringify@2.x, fast-json-stable-stringify@^2.0.0, fast-json-stable-stringify@^2.1.0: version "2.1.0" resolved "https://registry.yarnpkg.com/fast-json-stable-stringify/-/fast-json-stable-stringify-2.1.0.tgz#874bf69c6f404c2b5d99c481341399fd55892633" integrity sha512-lhd/wF+Lk98HZoTCtlVraHtfh5XYijIjalXck7saUtuanSDyLMxnHhSXEDJqHxD7msR8D0uCmqlkwjCV8xvwHw== @@ -2281,12 +3422,19 @@ fast-safe-stringify@^2.1.1: integrity sha512-W+KJc2dmILlPplD/H4K9l9LcAHAfPtP6BY84uVLXQ6Evcz9Lcg33Y2z1IVblT6xdY54PXYVHEv+0Wpq8Io6zkA== fastq@^1.6.0: - version "1.15.0" - resolved "https://registry.yarnpkg.com/fastq/-/fastq-1.15.0.tgz#d04d07c6a2a68fe4599fea8d2e103a937fae6b3a" - integrity sha512-wBrocU2LCXXa+lWBt8RoIRD89Fi8OdABODa/kEnyeyjS5aZO5/GNvI5sEINADqP/h8M29UHTHUb53sUu5Ihqdw== + version "1.13.0" + resolved "https://registry.yarnpkg.com/fastq/-/fastq-1.13.0.tgz#616760f88a7526bdfc596b7cab8c18938c36b98c" + integrity sha512-YpkpUnK8od0o1hmeSc7UUs/eB/vIPWJYjKck2QKIzAf71Vm1AAQ3EbuZB3g2JIy+pg+ERD0vqI79KyZiB2e2Nw== dependencies: reusify "^1.0.4" +fb-watchman@^2.0.0: + version "2.0.2" + resolved "https://registry.yarnpkg.com/fb-watchman/-/fb-watchman-2.0.2.tgz#e9524ee6b5c77e9e5001af0f85f3adbb8623255c" + integrity sha512-p5161BqbuCaSnB8jIbzQHOlpgsPmK5rJVDfDKO91Axs5NC1uu3HRQm6wt9cd9/+GtQQIO53JdGXXoyDpTAsgYA== + dependencies: + bser "2.1.1" + "fcntl@github:scality/node-fcntl#0.2.0": version "0.2.0" resolved "https://codeload.github.com/scality/node-fcntl/tar.gz/ae7493da46fb353eff0c4cb79e2cce838197c066" @@ -2320,6 +3468,13 @@ file-uri-to-path@1.0.0: resolved "https://registry.yarnpkg.com/file-uri-to-path/-/file-uri-to-path-1.0.0.tgz#553a7b8446ff6f684359c445f1e37a05dacc33dd" integrity sha512-0Zt+s3L7Vf1biwWZ29aARiVYLx7iMGnEUl9x33fbB/j3jR81u/O2LbqK+Bm1CDSNDKVtJ/YjwY7TUd5SkeLQLw== +fill-range@^7.0.1: + version "7.0.1" + resolved "https://registry.yarnpkg.com/fill-range/-/fill-range-7.0.1.tgz#1919a6a7c75fe38b2c7c77e5198535da9acdda40" + integrity sha512-qOo9F+dMUmC2Lcb4BbVvnKJxTPjCm+RRpe4gDuGrzkL7mEVl/djYSu2OdQ2Pa302N4oqkSg9ir6jaLWJ2USVpQ== + dependencies: + to-regex-range "^5.0.1" + finalhandler@1.2.0: version "1.2.0" resolved "https://registry.yarnpkg.com/finalhandler/-/finalhandler-1.2.0.tgz#7d23fe5731b207b4640e4fcd00aec1f9207a7b32" @@ -2340,6 +3495,14 @@ find-up@^2.1.0: dependencies: locate-path "^2.0.0" +find-up@^4.0.0, find-up@^4.1.0: + version "4.1.0" + resolved "https://registry.yarnpkg.com/find-up/-/find-up-4.1.0.tgz#97afe7d6cdc0bc5928584b7c8d7b16e8a9aa5d19" + integrity sha512-PpOwAdQ/YlXQ2vj8a3h8IipDuYRi3wceVQQGYWxNINccq40Anw7BlsEXCMbt1Zt+OLA6Fq9suIpIWD0OsnISlw== + dependencies: + locate-path "^5.0.0" + path-exists "^4.0.0" + find-up@^5.0.0: version "5.0.0" resolved "https://registry.yarnpkg.com/find-up/-/find-up-5.0.0.tgz#4c92819ecb7083561e4f4a240a86be5198f536fc" @@ -2444,6 +3607,11 @@ fs.realpath@^1.0.0: resolved "https://registry.yarnpkg.com/fs.realpath/-/fs.realpath-1.0.0.tgz#1504ad2523158caa40db4a2787cb01411994ea4f" integrity sha512-OO0pH2lK6a0hZnAdau5ItzHPI6pUlvI7jMVnxUQRtw4owF2wk8lOSabtGDCTP4Ggrg2MbGnWO9X8K1t4+fGMDw== +fsevents@^2.3.2: + version "2.3.2" + resolved "https://registry.yarnpkg.com/fsevents/-/fsevents-2.3.2.tgz#8a526f78b8fdf4623b709e0b975c52c24c02fd1a" + integrity sha512-xiqMQR4xAeHTuB9uWm+fFRcIOgKBMiOBP+eXiyT7jsgVCq1bkVygt00oASowB7EdtpOHaaPgKt812P9ab+DDKA== + function-bind@^1.1.1: version "1.1.1" resolved "https://registry.yarnpkg.com/function-bind/-/function-bind-1.1.1.tgz#a56899d3ea3c9bab874bb9773b7c5ede92f4895d" @@ -2507,6 +3675,16 @@ gcp-metadata@^0.6.1, gcp-metadata@^0.6.3: extend "^3.0.1" retry-axios "0.3.2" +gensync@^1.0.0-beta.2: + version "1.0.0-beta.2" + resolved "https://registry.yarnpkg.com/gensync/-/gensync-1.0.0-beta.2.tgz#32a6ee76c3d7f52d46b2b1ae5d93fea8580a25e0" + integrity sha512-3hN7NaskYvMDLQY55gnW3NQ+mesEAepTqlg+VEbj7zzqEMBVNhzcGYYeqFo/TlYz6eQiFcp1HcsCZO+nGgS8zg== + +get-caller-file@^2.0.5: + version "2.0.5" + resolved "https://registry.yarnpkg.com/get-caller-file/-/get-caller-file-2.0.5.tgz#4f94412a82db32f36e3b0b9741f8a97feb031f7e" + integrity sha512-DyFP3BM/3YHTQOCUL/w0OZHR0lpKeGrxotcHWcqNEdnltqFwXVfhEBQ94eIo34AfQpo0rGki4cyIiftY06h2Fg== + get-folder-size@^2.0.1: version "2.0.1" resolved "https://registry.yarnpkg.com/get-folder-size/-/get-folder-size-2.0.1.tgz#3fe0524dd3bad05257ef1311331417bcd020a497" @@ -2524,6 +3702,16 @@ get-intrinsic@^1.0.2, get-intrinsic@^1.1.0, get-intrinsic@^1.1.1: has "^1.0.3" has-symbols "^1.0.3" +get-package-type@^0.1.0: + version "0.1.0" + resolved "https://registry.yarnpkg.com/get-package-type/-/get-package-type-0.1.0.tgz#8de2d803cff44df3bc6c456e6668b36c3926e11a" + integrity sha512-pjzuKtY64GYfWizNAJ0fr9VqttZkNiK2iS430LtIHzjBEr6bX8Am2zm4sW4Ro5wjWW5cAlRL1qAMTcXbjNAO2Q== + +get-stream@^6.0.0: + version "6.0.1" + resolved "https://registry.yarnpkg.com/get-stream/-/get-stream-6.0.1.tgz#a262d8eef67aced57c2852ad6167526a43cbf7b7" + integrity sha512-ts6Wi+2j3jQjqi70w5AlN8DFnkSwC+MqmxEzdEALB2qXZYV3X/b1CTfgPLGJNMeAWxdPfU8FO1ms3NUfaHCPYg== + get-symbol-description@^1.0.0: version "1.0.0" resolved "https://registry.yarnpkg.com/get-symbol-description/-/get-symbol-description-1.0.0.tgz#7fdb81c900101fbd564dd5f1a30af5aadc1e58d6" @@ -2539,7 +3727,14 @@ getpass@^0.1.1: dependencies: assert-plus "^1.0.0" -glob-parent@^6.0.2: +glob-parent@^5.1.2: + version "5.1.2" + resolved "https://registry.yarnpkg.com/glob-parent/-/glob-parent-5.1.2.tgz#869832c58034fe68a4093c17dc15e8340d8401c4" + integrity sha512-AOIgSQCepiJYwP3ARnGx+5VnTu2HBYdzbGP45eLw1vr3zB3vZLeyed1sC9hnbcOc9/SrMyM5RPQrkGz4aS9Zow== + dependencies: + is-glob "^4.0.1" + +glob-parent@^6.0.1: version "6.0.2" resolved "https://registry.yarnpkg.com/glob-parent/-/glob-parent-6.0.2.tgz#6d237d99083950c79290f24c7642a3de9a28f9e3" integrity sha512-XxwI8EOhVQgWp6iDL+3b0r86f4d6AX6zSU55HfB4ydCEuXLXc5FcYeOu+nnGftS4TEju/11rt4KJPTMgbfmv4A== @@ -2570,13 +3765,30 @@ glob@^7.1.2, glob@^7.1.3, glob@^7.1.4: once "^1.3.0" path-is-absolute "^1.0.0" -globals@^13.19.0: - version "13.20.0" - resolved "https://registry.yarnpkg.com/globals/-/globals-13.20.0.tgz#ea276a1e508ffd4f1612888f9d1bad1e2717bf82" - integrity sha512-Qg5QtVkCy/kv3FUSlu4ukeZDVf9ee0iXLAUYX13gbR17bnejFTzr4iS9bY7kwCf1NztRNm1t91fjOiyx4CSwPQ== +globals@^11.1.0: + version "11.12.0" + resolved "https://registry.yarnpkg.com/globals/-/globals-11.12.0.tgz#ab8795338868a0babd8525758018c2a7eb95c42e" + integrity sha512-WOBp/EEGUiIsJSp7wcv/y6MO+lV9UoncWqxuFfm8eBwzWNgyfBd6Gz+IeKQ9jCmyhoH99g15M3T+QaVHFjizVA== + +globals@^13.15.0: + version "13.17.0" + resolved "https://registry.yarnpkg.com/globals/-/globals-13.17.0.tgz#902eb1e680a41da93945adbdcb5a9f361ba69bd4" + integrity sha512-1C+6nQRb1GwGMKm2dH/E7enFAMxGTmGI7/dEdhy/DNelv85w9B72t3uc5frtMNXIbzrarJJ/lTCjcaZwbLJmyw== dependencies: type-fest "^0.20.2" +globby@^11.1.0: + version "11.1.0" + resolved "https://registry.yarnpkg.com/globby/-/globby-11.1.0.tgz#bd4be98bb042f83d796f7e3811991fbe82a0d34b" + integrity sha512-jhIXaOzy1sb8IyocaruWSn1TjmnBVs8Ayhcy83rmxNJ8q2uWKCAj3CnJY+KpGSXCueAPc0i05kVvVKtP1t9S3g== + dependencies: + array-union "^2.1.0" + dir-glob "^3.0.1" + fast-glob "^3.2.9" + ignore "^5.2.0" + merge2 "^1.4.1" + slash "^3.0.0" + google-auth-library@^1.3.1: version "1.6.1" resolved "https://registry.yarnpkg.com/google-auth-library/-/google-auth-library-1.6.1.tgz#9c73d831ad720c0c3048ab89d0ffdec714d07dd2" @@ -2608,7 +3820,7 @@ google-p12-pem@^1.0.0: node-forge "^0.10.0" pify "^4.0.0" -graceful-fs@^4.1.2, graceful-fs@^4.2.6: +graceful-fs@^4.1.2, graceful-fs@^4.2.6, graceful-fs@^4.2.9: version "4.2.10" resolved "https://registry.yarnpkg.com/graceful-fs/-/graceful-fs-4.2.10.tgz#147d3a006da4ca3ce14728c7aefc287c367d7a6c" integrity sha512-9ByhssR2fPVsNZj478qUUbKfmL0+t5BDVyjShtyZZLiK7ZDAArFFfopyOTj0M05wE2tJPisA4iTnnXl2YoPvOA== @@ -2747,6 +3959,11 @@ hosted-git-info@^2.1.4: resolved "https://registry.yarnpkg.com/hosted-git-info/-/hosted-git-info-2.8.9.tgz#dffc0bf9a21c02209090f2aa69429e1414daf3f9" integrity sha512-mxIDAb9Lsm6DoOJ7xH+5+X4y1LU/4Hi50L9C5sIswK3JzULS4bwk1FvjdBgvYR4bzT4tuUQiC15FE2f5HbLvYw== +html-escaper@^2.0.0: + version "2.0.2" + resolved "https://registry.yarnpkg.com/html-escaper/-/html-escaper-2.0.2.tgz#dfd60027da36a36dfcbe236262c00a5822681453" + integrity sha512-H2iMtd0I4Mt5eYiapRdIDjp+XzelXQ0tFE4JS7YFwFevXXMmOp9myNrUvCg0D6ws8iqkRPBfKHgbwig1SmlLfg== + http-cache-semantics@^4.1.0: version "4.1.0" resolved "https://registry.yarnpkg.com/http-cache-semantics/-/http-cache-semantics-4.1.0.tgz#49e91c5cbf36c9b94bcfcd71c23d5249ec74e390" @@ -2840,6 +4057,11 @@ https-proxy-agent@^5.0.0: agent-base "6" debug "4" +human-signals@^2.1.0: + version "2.1.0" + resolved "https://registry.yarnpkg.com/human-signals/-/human-signals-2.1.0.tgz#dc91fcba42e4d06e4abaed33b3e7a3c02f514ea0" + integrity sha512-B4FFZ6q/T2jhhksgkbEW3HBvWIfDW85snkQgawt07S7J5QXTk6BkNV+0yAeZrM5QpMAdYlocGoljn0sJ/WQkFw== + humanize-ms@^1.2.1: version "1.2.1" resolved "https://registry.yarnpkg.com/humanize-ms/-/humanize-ms-1.2.1.tgz#c46e3159a293f6b896da29316d8b6fe8bb79bbed" @@ -2872,9 +4094,9 @@ ieee754@^1.1.13, ieee754@^1.1.4: integrity sha512-dcyqhDvX1C46lXZcVqCpK+FtMRQVdIMN6/Df5js2zouUsqG7I6sFxitIC+7KYK29KdXOLHdu9zL4sFnoVQnqaA== ignore@^5.2.0: - version "5.2.4" - resolved "https://registry.yarnpkg.com/ignore/-/ignore-5.2.4.tgz#a291c0c6178ff1b960befe47fcdec301674a6324" - integrity sha512-MAb38BcSbH0eHNBxn7ql2NH/kX33OkB3lZ1BNdh7ENeRChHTYsTvWrMubiIAMNS2llXEEgZ1MUOBtXChP3kaFQ== + version "5.2.0" + resolved "https://registry.yarnpkg.com/ignore/-/ignore-5.2.0.tgz#6d3bac8fa7fe0d45d9f9be7bac2fc279577e345a" + integrity sha512-CmxgYGiEPCLhfLnpPp1MoRmifwEIOgjcHXxOBjv7mY96c+eWScsOP9c112ZyLdWHi0FxHjI+4uVhKYp/gcdRmQ== immediate@^3.2.3: version "3.3.0" @@ -2894,6 +4116,14 @@ import-fresh@^3.0.0, import-fresh@^3.2.1: parent-module "^1.0.0" resolve-from "^4.0.0" +import-local@^3.0.2: + version "3.1.0" + resolved "https://registry.yarnpkg.com/import-local/-/import-local-3.1.0.tgz#b4479df8a5fd44f6cdce24070675676063c95cb4" + integrity sha512-ASB07uLtnDs1o6EHjKpX34BKYDSqnFerfTOJL2HvMqF70LnxpjkzDB8J44oT9pu4AMPkQwf8jl6szgvNd2tRIg== + dependencies: + pkg-dir "^4.2.0" + resolve-cwd "^3.0.0" + imurmurhash@^0.1.4: version "0.1.4" resolved "https://registry.yarnpkg.com/imurmurhash/-/imurmurhash-0.1.4.tgz#9218b9b2b928a238b13dc4fb6b6d576f231453ea" @@ -3053,6 +4283,11 @@ is-fullwidth-code-point@^3.0.0: resolved "https://registry.yarnpkg.com/is-fullwidth-code-point/-/is-fullwidth-code-point-3.0.0.tgz#f116f8064fe90b3f7844a38997c0b75051269f1d" integrity sha512-zymm5+u+sCsSWyD9qNaejV3DFvhCKclKdizYaJUuHA83RLjb7nSuGnddCHGv0hk+KY7BMAlsWeK4Ueg6EV6XQg== +is-generator-fn@^2.0.0: + version "2.1.0" + resolved "https://registry.yarnpkg.com/is-generator-fn/-/is-generator-fn-2.1.0.tgz#7d140adc389aaf3011a8f2a2a4cfa6faadffb118" + integrity sha512-cTIB4yPYL/Grw0EaSzASzg6bBy9gqCofvWN8okThAYIxKJZC+udlRAmGbM0XLeniEJSs8uEgHPGuHSe1XsOLSQ== + is-generator-function@^1.0.7: version "1.0.10" resolved "https://registry.yarnpkg.com/is-generator-function/-/is-generator-function-1.0.10.tgz#f1558baf1ac17e0deea7c0415c438351ff2b3c72" @@ -3067,7 +4302,7 @@ is-glob@^2.0.0: dependencies: is-extglob "^1.0.0" -is-glob@^4.0.0, is-glob@^4.0.3: +is-glob@^4.0.0, is-glob@^4.0.1, is-glob@^4.0.3: version "4.0.3" resolved "https://registry.yarnpkg.com/is-glob/-/is-glob-4.0.3.tgz#64f61e42cbbb2eec2071a9dac0b28ba1e65d5084" integrity sha512-xelSayHH36ZgE7ZWhli7pW34hNbNl8Ojv5KVmkJD4hBdD3th8Tfk9vYasLM+mXWOZhFkgZfxhLSnrwRr4elSSg== @@ -3106,10 +4341,10 @@ is-number-object@^1.0.4: dependencies: has-tostringtag "^1.0.0" -is-path-inside@^3.0.3: - version "3.0.3" - resolved "https://registry.yarnpkg.com/is-path-inside/-/is-path-inside-3.0.3.tgz#d231362e53a07ff2b0e0ea7fed049161ffd16283" - integrity sha512-Fd4gABb+ycGAmKou8eMftCupSir5lRxqf4aD/vd0cD2qc4HL07OjCeuHMr8Ro4CoMaeCKDB0/ECBOVWjTwUvPQ== +is-number@^7.0.0: + version "7.0.0" + resolved "https://registry.yarnpkg.com/is-number/-/is-number-7.0.0.tgz#7535345b896734d5f80c4d06c50955527a14f12b" + integrity sha512-41Cifkg6e8TylSpdtTpeLVMqvSBEVzTttHvERD741+pnZ8ANv0004MRL43QKPDlK9cGvNp6NZWZUBlbGXYxxng== is-regex@^1.1.4: version "1.1.4" @@ -3131,81 +4366,805 @@ is-stream@^2.0.0: resolved "https://registry.yarnpkg.com/is-stream/-/is-stream-2.0.1.tgz#fac1e3d53b97ad5a9d0ae9cef2389f5810a5c077" integrity sha512-hFoiJiTl63nn+kstHGBtewWSKnQLpyb155KHheA1l39uvtO9nWIop1p3udqPcUd/xbF1VLMO4n7OI6p7RbngDg== -is-string@^1.0.5, is-string@^1.0.7: - version "1.0.7" - resolved "https://registry.yarnpkg.com/is-string/-/is-string-1.0.7.tgz#0dd12bf2006f255bb58f695110eff7491eebc0fd" - integrity sha512-tE2UXzivje6ofPW7l23cjDOMa09gb7xlAqG6jG5ej6uPV32TlWP3NKPigtaGeHNu9fohccRYvIiZMfOOnOYUtg== - dependencies: - has-tostringtag "^1.0.0" +is-string@^1.0.5, is-string@^1.0.7: + version "1.0.7" + resolved "https://registry.yarnpkg.com/is-string/-/is-string-1.0.7.tgz#0dd12bf2006f255bb58f695110eff7491eebc0fd" + integrity sha512-tE2UXzivje6ofPW7l23cjDOMa09gb7xlAqG6jG5ej6uPV32TlWP3NKPigtaGeHNu9fohccRYvIiZMfOOnOYUtg== + dependencies: + has-tostringtag "^1.0.0" + +is-symbol@^1.0.2, is-symbol@^1.0.3: + version "1.0.4" + resolved "https://registry.yarnpkg.com/is-symbol/-/is-symbol-1.0.4.tgz#a6dac93b635b063ca6872236de88910a57af139c" + integrity sha512-C/CPBqKWnvdcxqIARxyOh4v1UUEOCHpgDa0WYgpKDFMszcrPcffg5uhwSgPCLD2WWxmq6isisz87tzT01tuGhg== + dependencies: + has-symbols "^1.0.2" + +is-typed-array@^1.1.3, is-typed-array@^1.1.9: + version "1.1.9" + resolved "https://registry.yarnpkg.com/is-typed-array/-/is-typed-array-1.1.9.tgz#246d77d2871e7d9f5aeb1d54b9f52c71329ece67" + integrity sha512-kfrlnTTn8pZkfpJMUgYD7YZ3qzeJgWUn8XfVYBARc4wnmNOmLbmuuaAs3q5fvB0UJOn6yHAKaGTPM7d6ezoD/A== + dependencies: + available-typed-arrays "^1.0.5" + call-bind "^1.0.2" + es-abstract "^1.20.0" + for-each "^0.3.3" + has-tostringtag "^1.0.0" + +is-typedarray@^1.0.0, is-typedarray@~1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/is-typedarray/-/is-typedarray-1.0.0.tgz#e479c80858df0c1b11ddda6940f96011fcda4a9a" + integrity sha512-cyA56iCMHAh5CdzjJIa4aohJyeO1YbwLi3Jc35MmRU6poroFjIGZzUzupGiRPOjgHg9TLu43xbpwXk523fMxKA== + +is-valid-path@^0.1.1: + version "0.1.1" + resolved "https://registry.yarnpkg.com/is-valid-path/-/is-valid-path-0.1.1.tgz#110f9ff74c37f663e1ec7915eb451f2db93ac9df" + integrity sha512-+kwPrVDu9Ms03L90Qaml+79+6DZHqHyRoANI6IsZJ/g8frhnfchDOBCa0RbQ6/kdHt5CS5OeIEyrYznNuVN+8A== + dependencies: + is-invalid-path "^0.1.0" + +is-weakref@^1.0.2: + version "1.0.2" + resolved "https://registry.yarnpkg.com/is-weakref/-/is-weakref-1.0.2.tgz#9529f383a9338205e89765e0392efc2f100f06f2" + integrity sha512-qctsuLZmIQ0+vSSMfoVvyFe2+GSEvnmZ2ezTup1SBse9+twCCeial6EEi3Nc2KFcf6+qz2FBPnjXsk8xhKSaPQ== + dependencies: + call-bind "^1.0.2" + +is-wsl@^2.2.0: + version "2.2.0" + resolved "https://registry.yarnpkg.com/is-wsl/-/is-wsl-2.2.0.tgz#74a4c76e77ca9fd3f932f290c17ea326cd157271" + integrity sha512-fKzAra0rGJUUBwGBgNkHZuToZcn+TtXHpeCgmkMJMMYx1sQDYaCSyjJBSCa2nH1DGm7s3n1oBnohoVTBaN7Lww== + dependencies: + is-docker "^2.0.0" + +isarray@0.0.1: + version "0.0.1" + resolved "https://registry.yarnpkg.com/isarray/-/isarray-0.0.1.tgz#8a18acfca9a8f4177e09abfc6038939b05d1eedf" + integrity sha512-D2S+3GLxWH+uhrNEcoh/fnmYeP8E8/zHl644d/jdA0g2uyXvy3sb0qxotE+ne0LtccHknQzWwZEzhak7oJ0COQ== + +isarray@2.0.1: + version "2.0.1" + resolved "https://registry.yarnpkg.com/isarray/-/isarray-2.0.1.tgz#a37d94ed9cda2d59865c9f76fe596ee1f338741e" + integrity sha512-c2cu3UxbI+b6kR3fy0nRnAhodsvR9dx7U5+znCOzdj6IfP3upFURTr0Xl5BlQZNKZjEtxrmVyfSdeE3O57smoQ== + +isarray@^1.0.0, isarray@~1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/isarray/-/isarray-1.0.0.tgz#bb935d48582cba168c06834957a54a3e07124f11" + integrity sha512-VLghIWNM6ELQzo7zwmcg0NmTVyWKYjvIeM83yjp0wRDTmUnrM678fQbcKBo6n2CJEF0szoG//ytg+TKla89ALQ== + +isexe@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/isexe/-/isexe-2.0.0.tgz#e8fbf374dc556ff8947a10dcb0572d633f2cfa10" + integrity sha512-RHxMLp9lnKHGHRng9QFhRCMbYAcVpn69smSGcq3f36xjgVVWThj4qqLbTLlq7Ssj8B+fIQ1EuCEGI2lKsyQeIw== + +isstream@~0.1.2: + version "0.1.2" + resolved "https://registry.yarnpkg.com/isstream/-/isstream-0.1.2.tgz#47e63f7af55afa6f92e1500e690eb8b8529c099a" + integrity sha512-Yljz7ffyPbrLpLngrMtZ7NduUgVvi6wG9RJ9IUcyCd59YQ911PBJphODUcbOVbqYfxe1wuYf/LJ8PauMRwsM/g== + +istanbul-lib-coverage@^3.0.0, istanbul-lib-coverage@^3.2.0: + version "3.2.0" + resolved "https://registry.yarnpkg.com/istanbul-lib-coverage/-/istanbul-lib-coverage-3.2.0.tgz#189e7909d0a39fa5a3dfad5b03f71947770191d3" + integrity sha512-eOeJ5BHCmHYvQK7xt9GkdHuzuCGS1Y6g9Gvnx3Ym33fz/HpLRYxiS0wHNr+m/MBC8B647Xt608vCDEvhl9c6Mw== + +istanbul-lib-instrument@^5.0.4: + version "5.2.1" + resolved "https://registry.yarnpkg.com/istanbul-lib-instrument/-/istanbul-lib-instrument-5.2.1.tgz#d10c8885c2125574e1c231cacadf955675e1ce3d" + integrity sha512-pzqtp31nLv/XFOzXGuvhCb8qhjmTVo5vjVk19XE4CRlSWz0KoeJ3bw9XsA7nOp9YBf4qHjwBxkDzKcME/J29Yg== + dependencies: + "@babel/core" "^7.12.3" + "@babel/parser" "^7.14.7" + "@istanbuljs/schema" "^0.1.2" + istanbul-lib-coverage "^3.2.0" + semver "^6.3.0" + +istanbul-lib-instrument@^6.0.0: + version "6.0.1" + resolved "https://registry.yarnpkg.com/istanbul-lib-instrument/-/istanbul-lib-instrument-6.0.1.tgz#71e87707e8041428732518c6fb5211761753fbdf" + integrity sha512-EAMEJBsYuyyztxMxW3g7ugGPkrZsV57v0Hmv3mm1uQsmB+QnZuepg731CRaIgeUVSdmsTngOkSnauNF8p7FIhA== + dependencies: + "@babel/core" "^7.12.3" + "@babel/parser" "^7.14.7" + "@istanbuljs/schema" "^0.1.2" + istanbul-lib-coverage "^3.2.0" + semver "^7.5.4" + +istanbul-lib-report@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/istanbul-lib-report/-/istanbul-lib-report-3.0.0.tgz#7518fe52ea44de372f460a76b5ecda9ffb73d8a6" + integrity sha512-wcdi+uAKzfiGT2abPpKZ0hSU1rGQjUQnLvtY5MpQ7QCTahD3VODhcu4wcfY1YtkGaDD5yuydOLINXsfbus9ROw== + dependencies: + istanbul-lib-coverage "^3.0.0" + make-dir "^3.0.0" + supports-color "^7.1.0" + +istanbul-lib-source-maps@^4.0.0: + version "4.0.1" + resolved "https://registry.yarnpkg.com/istanbul-lib-source-maps/-/istanbul-lib-source-maps-4.0.1.tgz#895f3a709fcfba34c6de5a42939022f3e4358551" + integrity sha512-n3s8EwkdFIJCG3BPKBYvskgXGoy88ARzvegkitk60NxRdwltLOTaH7CUiMRXvwYorl0Q712iEjcWB+fK/MrWVw== + dependencies: + debug "^4.1.1" + istanbul-lib-coverage "^3.0.0" + source-map "^0.6.1" + +istanbul-reports@^3.1.3: + version "3.1.5" + resolved "https://registry.yarnpkg.com/istanbul-reports/-/istanbul-reports-3.1.5.tgz#cc9a6ab25cb25659810e4785ed9d9fb742578bae" + integrity sha512-nUsEMa9pBt/NOHqbcbeJEgqIlY/K7rVWUX6Lql2orY5e9roQOthbR3vtY4zzf2orPELg80fnxxk9zUyPlgwD1w== + dependencies: + html-escaper "^2.0.0" + istanbul-lib-report "^3.0.0" + +jest-changed-files@^29.7.0: + version "29.7.0" + resolved "https://registry.yarnpkg.com/jest-changed-files/-/jest-changed-files-29.7.0.tgz#1c06d07e77c78e1585d020424dedc10d6e17ac3a" + integrity sha512-fEArFiwf1BpQ+4bXSprcDc3/x4HSzL4al2tozwVpDFpsxALjLYdyiIK4e5Vz66GQJIbXJ82+35PtysofptNX2w== + dependencies: + execa "^5.0.0" + jest-util "^29.7.0" + p-limit "^3.1.0" + +jest-circus@^29.1.2: + version "29.1.2" + resolved "https://registry.yarnpkg.com/jest-circus/-/jest-circus-29.1.2.tgz#4551068e432f169a53167fe1aef420cf51c8a735" + integrity sha512-ajQOdxY6mT9GtnfJRZBRYS7toNIJayiiyjDyoZcnvPRUPwJ58JX0ci0PKAKUo2C1RyzlHw0jabjLGKksO42JGA== + dependencies: + "@jest/environment" "^29.1.2" + "@jest/expect" "^29.1.2" + "@jest/test-result" "^29.1.2" + "@jest/types" "^29.1.2" + "@types/node" "*" + chalk "^4.0.0" + co "^4.6.0" + dedent "^0.7.0" + is-generator-fn "^2.0.0" + jest-each "^29.1.2" + jest-matcher-utils "^29.1.2" + jest-message-util "^29.1.2" + jest-runtime "^29.1.2" + jest-snapshot "^29.1.2" + jest-util "^29.1.2" + p-limit "^3.1.0" + pretty-format "^29.1.2" + slash "^3.0.0" + stack-utils "^2.0.3" + +jest-circus@^29.7.0: + version "29.7.0" + resolved "https://registry.yarnpkg.com/jest-circus/-/jest-circus-29.7.0.tgz#b6817a45fcc835d8b16d5962d0c026473ee3668a" + integrity sha512-3E1nCMgipcTkCocFwM90XXQab9bS+GMsjdpmPrlelaxwD93Ad8iVEjX/vvHPdLPnFf+L40u+5+iutRdA1N9myw== + dependencies: + "@jest/environment" "^29.7.0" + "@jest/expect" "^29.7.0" + "@jest/test-result" "^29.7.0" + "@jest/types" "^29.6.3" + "@types/node" "*" + chalk "^4.0.0" + co "^4.6.0" + dedent "^1.0.0" + is-generator-fn "^2.0.0" + jest-each "^29.7.0" + jest-matcher-utils "^29.7.0" + jest-message-util "^29.7.0" + jest-runtime "^29.7.0" + jest-snapshot "^29.7.0" + jest-util "^29.7.0" + p-limit "^3.1.0" + pretty-format "^29.7.0" + pure-rand "^6.0.0" + slash "^3.0.0" + stack-utils "^2.0.3" + +jest-cli@^29.1.2: + version "29.1.2" + resolved "https://registry.yarnpkg.com/jest-cli/-/jest-cli-29.1.2.tgz#423b9c5d3ea20a50b1354b8bf3f2a20e72110e89" + integrity sha512-vsvBfQ7oS2o4MJdAH+4u9z76Vw5Q8WBQF5MchDbkylNknZdrPTX1Ix7YRJyTlOWqRaS7ue/cEAn+E4V1MWyMzw== + dependencies: + "@jest/core" "^29.1.2" + "@jest/test-result" "^29.1.2" + "@jest/types" "^29.1.2" + chalk "^4.0.0" + exit "^0.1.2" + graceful-fs "^4.2.9" + import-local "^3.0.2" + jest-config "^29.1.2" + jest-util "^29.1.2" + jest-validate "^29.1.2" + prompts "^2.0.1" + yargs "^17.3.1" + +jest-config@^29.1.2: + version "29.1.2" + resolved "https://registry.yarnpkg.com/jest-config/-/jest-config-29.1.2.tgz#7d004345ca4c09f5d8f802355f54494e90842f4d" + integrity sha512-EC3Zi86HJUOz+2YWQcJYQXlf0zuBhJoeyxLM6vb6qJsVmpP7KcCP1JnyF0iaqTaXdBP8Rlwsvs7hnKWQWWLwwA== + dependencies: + "@babel/core" "^7.11.6" + "@jest/test-sequencer" "^29.1.2" + "@jest/types" "^29.1.2" + babel-jest "^29.1.2" + chalk "^4.0.0" + ci-info "^3.2.0" + deepmerge "^4.2.2" + glob "^7.1.3" + graceful-fs "^4.2.9" + jest-circus "^29.1.2" + jest-environment-node "^29.1.2" + jest-get-type "^29.0.0" + jest-regex-util "^29.0.0" + jest-resolve "^29.1.2" + jest-runner "^29.1.2" + jest-util "^29.1.2" + jest-validate "^29.1.2" + micromatch "^4.0.4" + parse-json "^5.2.0" + pretty-format "^29.1.2" + slash "^3.0.0" + strip-json-comments "^3.1.1" + +jest-config@^29.7.0: + version "29.7.0" + resolved "https://registry.yarnpkg.com/jest-config/-/jest-config-29.7.0.tgz#bcbda8806dbcc01b1e316a46bb74085a84b0245f" + integrity sha512-uXbpfeQ7R6TZBqI3/TxCU4q4ttk3u0PJeC+E0zbfSoSjq6bJ7buBPxzQPL0ifrkY4DNu4JUdk0ImlBUYi840eQ== + dependencies: + "@babel/core" "^7.11.6" + "@jest/test-sequencer" "^29.7.0" + "@jest/types" "^29.6.3" + babel-jest "^29.7.0" + chalk "^4.0.0" + ci-info "^3.2.0" + deepmerge "^4.2.2" + glob "^7.1.3" + graceful-fs "^4.2.9" + jest-circus "^29.7.0" + jest-environment-node "^29.7.0" + jest-get-type "^29.6.3" + jest-regex-util "^29.6.3" + jest-resolve "^29.7.0" + jest-runner "^29.7.0" + jest-util "^29.7.0" + jest-validate "^29.7.0" + micromatch "^4.0.4" + parse-json "^5.2.0" + pretty-format "^29.7.0" + slash "^3.0.0" + strip-json-comments "^3.1.1" + +jest-diff@^29.1.2: + version "29.1.2" + resolved "https://registry.yarnpkg.com/jest-diff/-/jest-diff-29.1.2.tgz#bb7aaf5353227d6f4f96c5e7e8713ce576a607dc" + integrity sha512-4GQts0aUopVvecIT4IwD/7xsBaMhKTYoM4/njE/aVw9wpw+pIUVp8Vab/KnSzSilr84GnLBkaP3JLDnQYCKqVQ== + dependencies: + chalk "^4.0.0" + diff-sequences "^29.0.0" + jest-get-type "^29.0.0" + pretty-format "^29.1.2" + +jest-diff@^29.7.0: + version "29.7.0" + resolved "https://registry.yarnpkg.com/jest-diff/-/jest-diff-29.7.0.tgz#017934a66ebb7ecf6f205e84699be10afd70458a" + integrity sha512-LMIgiIrhigmPrs03JHpxUh2yISK3vLFPkAodPeo0+BuF7wA2FoQbkEg1u8gBYBThncu7e1oEDUfIXVuTqLRUjw== + dependencies: + chalk "^4.0.0" + diff-sequences "^29.6.3" + jest-get-type "^29.6.3" + pretty-format "^29.7.0" + +jest-docblock@^29.0.0: + version "29.0.0" + resolved "https://registry.yarnpkg.com/jest-docblock/-/jest-docblock-29.0.0.tgz#3151bcc45ed7f5a8af4884dcc049aee699b4ceae" + integrity sha512-s5Kpra/kLzbqu9dEjov30kj1n4tfu3e7Pl8v+f8jOkeWNqM6Ds8jRaJfZow3ducoQUrf2Z4rs2N5S3zXnb83gw== + dependencies: + detect-newline "^3.0.0" + +jest-docblock@^29.7.0: + version "29.7.0" + resolved "https://registry.yarnpkg.com/jest-docblock/-/jest-docblock-29.7.0.tgz#8fddb6adc3cdc955c93e2a87f61cfd350d5d119a" + integrity sha512-q617Auw3A612guyaFgsbFeYpNP5t2aoUNLwBUbc/0kD1R4t9ixDbyFTHd1nok4epoVFpr7PmeWHrhvuV3XaJ4g== + dependencies: + detect-newline "^3.0.0" + +jest-each@^29.1.2: + version "29.1.2" + resolved "https://registry.yarnpkg.com/jest-each/-/jest-each-29.1.2.tgz#d4c8532c07a846e79f194f7007ce7cb1987d1cd0" + integrity sha512-AmTQp9b2etNeEwMyr4jc0Ql/LIX/dhbgP21gHAizya2X6rUspHn2gysMXaj6iwWuOJ2sYRgP8c1P4cXswgvS1A== + dependencies: + "@jest/types" "^29.1.2" + chalk "^4.0.0" + jest-get-type "^29.0.0" + jest-util "^29.1.2" + pretty-format "^29.1.2" + +jest-each@^29.7.0: + version "29.7.0" + resolved "https://registry.yarnpkg.com/jest-each/-/jest-each-29.7.0.tgz#162a9b3f2328bdd991beaabffbb74745e56577d1" + integrity sha512-gns+Er14+ZrEoC5fhOfYCY1LOHHr0TI+rQUHZS8Ttw2l7gl+80eHc/gFf2Ktkw0+SIACDTeWvpFcv3B04VembQ== + dependencies: + "@jest/types" "^29.6.3" + chalk "^4.0.0" + jest-get-type "^29.6.3" + jest-util "^29.7.0" + pretty-format "^29.7.0" + +jest-environment-node@^29.1.2: + version "29.1.2" + resolved "https://registry.yarnpkg.com/jest-environment-node/-/jest-environment-node-29.1.2.tgz#005e05cc6ea4b9b5ba55906ab1ce53c82f6907a7" + integrity sha512-C59yVbdpY8682u6k/lh8SUMDJPbOyCHOTgLVVi1USWFxtNV+J8fyIwzkg+RJIVI30EKhKiAGNxYaFr3z6eyNhQ== + dependencies: + "@jest/environment" "^29.1.2" + "@jest/fake-timers" "^29.1.2" + "@jest/types" "^29.1.2" + "@types/node" "*" + jest-mock "^29.1.2" + jest-util "^29.1.2" + +jest-environment-node@^29.7.0: + version "29.7.0" + resolved "https://registry.yarnpkg.com/jest-environment-node/-/jest-environment-node-29.7.0.tgz#0b93e111dda8ec120bc8300e6d1fb9576e164376" + integrity sha512-DOSwCRqXirTOyheM+4d5YZOrWcdu0LNZ87ewUoywbcb2XR4wKgqiG8vNeYwhjFMbEkfju7wx2GYH0P2gevGvFw== + dependencies: + "@jest/environment" "^29.7.0" + "@jest/fake-timers" "^29.7.0" + "@jest/types" "^29.6.3" + "@types/node" "*" + jest-mock "^29.7.0" + jest-util "^29.7.0" + +jest-get-type@^29.0.0: + version "29.0.0" + resolved "https://registry.yarnpkg.com/jest-get-type/-/jest-get-type-29.0.0.tgz#843f6c50a1b778f7325df1129a0fd7aa713aef80" + integrity sha512-83X19z/HuLKYXYHskZlBAShO7UfLFXu/vWajw9ZNJASN32li8yHMaVGAQqxFW1RCFOkB7cubaL6FaJVQqqJLSw== + +jest-get-type@^29.6.3: + version "29.6.3" + resolved "https://registry.yarnpkg.com/jest-get-type/-/jest-get-type-29.6.3.tgz#36f499fdcea197c1045a127319c0481723908fd1" + integrity sha512-zrteXnqYxfQh7l5FHyL38jL39di8H8rHoecLH3JNxH3BwOrBsNeabdap5e0I23lD4HHI8W5VFBZqG4Eaq5LNcw== + +jest-haste-map@^29.1.2: + version "29.1.2" + resolved "https://registry.yarnpkg.com/jest-haste-map/-/jest-haste-map-29.1.2.tgz#93f3634aa921b6b654e7c94137b24e02e7ca6ac9" + integrity sha512-xSjbY8/BF11Jh3hGSPfYTa/qBFrm3TPM7WU8pU93m2gqzORVLkHFWvuZmFsTEBPRKndfewXhMOuzJNHyJIZGsw== + dependencies: + "@jest/types" "^29.1.2" + "@types/graceful-fs" "^4.1.3" + "@types/node" "*" + anymatch "^3.0.3" + fb-watchman "^2.0.0" + graceful-fs "^4.2.9" + jest-regex-util "^29.0.0" + jest-util "^29.1.2" + jest-worker "^29.1.2" + micromatch "^4.0.4" + walker "^1.0.8" + optionalDependencies: + fsevents "^2.3.2" + +jest-haste-map@^29.7.0: + version "29.7.0" + resolved "https://registry.yarnpkg.com/jest-haste-map/-/jest-haste-map-29.7.0.tgz#3c2396524482f5a0506376e6c858c3bbcc17b104" + integrity sha512-fP8u2pyfqx0K1rGn1R9pyE0/KTn+G7PxktWidOBTqFPLYX0b9ksaMFkhK5vrS3DVun09pckLdlx90QthlW7AmA== + dependencies: + "@jest/types" "^29.6.3" + "@types/graceful-fs" "^4.1.3" + "@types/node" "*" + anymatch "^3.0.3" + fb-watchman "^2.0.0" + graceful-fs "^4.2.9" + jest-regex-util "^29.6.3" + jest-util "^29.7.0" + jest-worker "^29.7.0" + micromatch "^4.0.4" + walker "^1.0.8" + optionalDependencies: + fsevents "^2.3.2" + +jest-leak-detector@^29.1.2: + version "29.1.2" + resolved "https://registry.yarnpkg.com/jest-leak-detector/-/jest-leak-detector-29.1.2.tgz#4c846db14c58219430ccbc4f01a1ec52ebee4fc2" + integrity sha512-TG5gAZJpgmZtjb6oWxBLf2N6CfQ73iwCe6cofu/Uqv9iiAm6g502CAnGtxQaTfpHECBdVEMRBhomSXeLnoKjiQ== + dependencies: + jest-get-type "^29.0.0" + pretty-format "^29.1.2" + +jest-leak-detector@^29.7.0: + version "29.7.0" + resolved "https://registry.yarnpkg.com/jest-leak-detector/-/jest-leak-detector-29.7.0.tgz#5b7ec0dadfdfec0ca383dc9aa016d36b5ea4c728" + integrity sha512-kYA8IJcSYtST2BY9I+SMC32nDpBT3J2NvWJx8+JCuCdl/CR1I4EKUJROiP8XtCcxqgTTBGJNdbB1A8XRKbTetw== + dependencies: + jest-get-type "^29.6.3" + pretty-format "^29.7.0" + +jest-matcher-utils@^29.1.2: + version "29.1.2" + resolved "https://registry.yarnpkg.com/jest-matcher-utils/-/jest-matcher-utils-29.1.2.tgz#e68c4bcc0266e70aa1a5c13fb7b8cd4695e318a1" + integrity sha512-MV5XrD3qYSW2zZSHRRceFzqJ39B2z11Qv0KPyZYxnzDHFeYZGJlgGi0SW+IXSJfOewgJp/Km/7lpcFT+cgZypw== + dependencies: + chalk "^4.0.0" + jest-diff "^29.1.2" + jest-get-type "^29.0.0" + pretty-format "^29.1.2" + +jest-matcher-utils@^29.7.0: + version "29.7.0" + resolved "https://registry.yarnpkg.com/jest-matcher-utils/-/jest-matcher-utils-29.7.0.tgz#ae8fec79ff249fd592ce80e3ee474e83a6c44f12" + integrity sha512-sBkD+Xi9DtcChsI3L3u0+N0opgPYnCRPtGcQYrgXmR+hmt/fYfWAL0xRXYU8eWOdfuLgBe0YCW3AFtnRLagq/g== + dependencies: + chalk "^4.0.0" + jest-diff "^29.7.0" + jest-get-type "^29.6.3" + pretty-format "^29.7.0" + +jest-message-util@^29.1.2: + version "29.1.2" + resolved "https://registry.yarnpkg.com/jest-message-util/-/jest-message-util-29.1.2.tgz#c21a33c25f9dc1ebfcd0f921d89438847a09a501" + integrity sha512-9oJ2Os+Qh6IlxLpmvshVbGUiSkZVc2FK+uGOm6tghafnB2RyjKAxMZhtxThRMxfX1J1SOMhTn9oK3/MutRWQJQ== + dependencies: + "@babel/code-frame" "^7.12.13" + "@jest/types" "^29.1.2" + "@types/stack-utils" "^2.0.0" + chalk "^4.0.0" + graceful-fs "^4.2.9" + micromatch "^4.0.4" + pretty-format "^29.1.2" + slash "^3.0.0" + stack-utils "^2.0.3" + +jest-message-util@^29.7.0: + version "29.7.0" + resolved "https://registry.yarnpkg.com/jest-message-util/-/jest-message-util-29.7.0.tgz#8bc392e204e95dfe7564abbe72a404e28e51f7f3" + integrity sha512-GBEV4GRADeP+qtB2+6u61stea8mGcOT4mCtrYISZwfu9/ISHFJ/5zOMXYbpBE9RsS5+Gb63DW4FgmnKJ79Kf6w== + dependencies: + "@babel/code-frame" "^7.12.13" + "@jest/types" "^29.6.3" + "@types/stack-utils" "^2.0.0" + chalk "^4.0.0" + graceful-fs "^4.2.9" + micromatch "^4.0.4" + pretty-format "^29.7.0" + slash "^3.0.0" + stack-utils "^2.0.3" + +jest-mock@^29.1.2: + version "29.1.2" + resolved "https://registry.yarnpkg.com/jest-mock/-/jest-mock-29.1.2.tgz#de47807edbb9d4abf8423f1d8d308d670105678c" + integrity sha512-PFDAdjjWbjPUtQPkQufvniXIS3N9Tv7tbibePEjIIprzjgo0qQlyUiVMrT4vL8FaSJo1QXifQUOuPH3HQC/aMA== + dependencies: + "@jest/types" "^29.1.2" + "@types/node" "*" + jest-util "^29.1.2" + +jest-mock@^29.7.0: + version "29.7.0" + resolved "https://registry.yarnpkg.com/jest-mock/-/jest-mock-29.7.0.tgz#4e836cf60e99c6fcfabe9f99d017f3fdd50a6347" + integrity sha512-ITOMZn+UkYS4ZFh83xYAOzWStloNzJFO2s8DWrE4lhtGD+AorgnbkiKERe4wQVBydIGPx059g6riW5Btp6Llnw== + dependencies: + "@jest/types" "^29.6.3" + "@types/node" "*" + jest-util "^29.7.0" + +jest-pnp-resolver@^1.2.2: + version "1.2.2" + resolved "https://registry.yarnpkg.com/jest-pnp-resolver/-/jest-pnp-resolver-1.2.2.tgz#b704ac0ae028a89108a4d040b3f919dfddc8e33c" + integrity sha512-olV41bKSMm8BdnuMsewT4jqlZ8+3TCARAXjZGT9jcoSnrfUnRCqnMoF9XEeoWjbzObpqF9dRhHQj0Xb9QdF6/w== + +jest-regex-util@^29.0.0: + version "29.0.0" + resolved "https://registry.yarnpkg.com/jest-regex-util/-/jest-regex-util-29.0.0.tgz#b442987f688289df8eb6c16fa8df488b4cd007de" + integrity sha512-BV7VW7Sy0fInHWN93MMPtlClweYv2qrSCwfeFWmpribGZtQPWNvRSq9XOVgOEjU1iBGRKXUZil0o2AH7Iy9Lug== + +jest-regex-util@^29.6.3: + version "29.6.3" + resolved "https://registry.yarnpkg.com/jest-regex-util/-/jest-regex-util-29.6.3.tgz#4a556d9c776af68e1c5f48194f4d0327d24e8a52" + integrity sha512-KJJBsRCyyLNWCNBOvZyRDnAIfUiRJ8v+hOBQYGn8gDyF3UegwiP4gwRR3/SDa42g1YbVycTidUF3rKjyLFDWbg== + +jest-resolve-dependencies@^29.7.0: + version "29.7.0" + resolved "https://registry.yarnpkg.com/jest-resolve-dependencies/-/jest-resolve-dependencies-29.7.0.tgz#1b04f2c095f37fc776ff40803dc92921b1e88428" + integrity sha512-un0zD/6qxJ+S0et7WxeI3H5XSe9lTBBR7bOHCHXkKR6luG5mwDDlIzVQ0V5cZCuoTgEdcdwzTghYkTWfubi+nA== + dependencies: + jest-regex-util "^29.6.3" + jest-snapshot "^29.7.0" + +jest-resolve@^29.1.2: + version "29.1.2" + resolved "https://registry.yarnpkg.com/jest-resolve/-/jest-resolve-29.1.2.tgz#9dd8c2fc83e59ee7d676b14bd45a5f89e877741d" + integrity sha512-7fcOr+k7UYSVRJYhSmJHIid3AnDBcLQX3VmT9OSbPWsWz1MfT7bcoerMhADKGvKCoMpOHUQaDHtQoNp/P9JMGg== + dependencies: + chalk "^4.0.0" + graceful-fs "^4.2.9" + jest-haste-map "^29.1.2" + jest-pnp-resolver "^1.2.2" + jest-util "^29.1.2" + jest-validate "^29.1.2" + resolve "^1.20.0" + resolve.exports "^1.1.0" + slash "^3.0.0" + +jest-resolve@^29.7.0: + version "29.7.0" + resolved "https://registry.yarnpkg.com/jest-resolve/-/jest-resolve-29.7.0.tgz#64d6a8992dd26f635ab0c01e5eef4399c6bcbc30" + integrity sha512-IOVhZSrg+UvVAshDSDtHyFCCBUl/Q3AAJv8iZ6ZjnZ74xzvwuzLXid9IIIPgTnY62SJjfuupMKZsZQRsCvxEgA== + dependencies: + chalk "^4.0.0" + graceful-fs "^4.2.9" + jest-haste-map "^29.7.0" + jest-pnp-resolver "^1.2.2" + jest-util "^29.7.0" + jest-validate "^29.7.0" + resolve "^1.20.0" + resolve.exports "^2.0.0" + slash "^3.0.0" + +jest-runner@^29.1.2: + version "29.1.2" + resolved "https://registry.yarnpkg.com/jest-runner/-/jest-runner-29.1.2.tgz#f18b2b86101341e047de8c2f51a5fdc4e97d053a" + integrity sha512-yy3LEWw8KuBCmg7sCGDIqKwJlULBuNIQa2eFSVgVASWdXbMYZ9H/X0tnXt70XFoGf92W2sOQDOIFAA6f2BG04Q== + dependencies: + "@jest/console" "^29.1.2" + "@jest/environment" "^29.1.2" + "@jest/test-result" "^29.1.2" + "@jest/transform" "^29.1.2" + "@jest/types" "^29.1.2" + "@types/node" "*" + chalk "^4.0.0" + emittery "^0.10.2" + graceful-fs "^4.2.9" + jest-docblock "^29.0.0" + jest-environment-node "^29.1.2" + jest-haste-map "^29.1.2" + jest-leak-detector "^29.1.2" + jest-message-util "^29.1.2" + jest-resolve "^29.1.2" + jest-runtime "^29.1.2" + jest-util "^29.1.2" + jest-watcher "^29.1.2" + jest-worker "^29.1.2" + p-limit "^3.1.0" + source-map-support "0.5.13" + +jest-runner@^29.7.0: + version "29.7.0" + resolved "https://registry.yarnpkg.com/jest-runner/-/jest-runner-29.7.0.tgz#809af072d408a53dcfd2e849a4c976d3132f718e" + integrity sha512-fsc4N6cPCAahybGBfTRcq5wFR6fpLznMg47sY5aDpsoejOcVYFb07AHuSnR0liMcPTgBsA3ZJL6kFOjPdoNipQ== + dependencies: + "@jest/console" "^29.7.0" + "@jest/environment" "^29.7.0" + "@jest/test-result" "^29.7.0" + "@jest/transform" "^29.7.0" + "@jest/types" "^29.6.3" + "@types/node" "*" + chalk "^4.0.0" + emittery "^0.13.1" + graceful-fs "^4.2.9" + jest-docblock "^29.7.0" + jest-environment-node "^29.7.0" + jest-haste-map "^29.7.0" + jest-leak-detector "^29.7.0" + jest-message-util "^29.7.0" + jest-resolve "^29.7.0" + jest-runtime "^29.7.0" + jest-util "^29.7.0" + jest-watcher "^29.7.0" + jest-worker "^29.7.0" + p-limit "^3.1.0" + source-map-support "0.5.13" + +jest-runtime@^29.1.2: + version "29.1.2" + resolved "https://registry.yarnpkg.com/jest-runtime/-/jest-runtime-29.1.2.tgz#dbcd57103d61115479108d5864bdcd661d9c6783" + integrity sha512-jr8VJLIf+cYc+8hbrpt412n5jX3tiXmpPSYTGnwcvNemY+EOuLNiYnHJ3Kp25rkaAcTWOEI4ZdOIQcwYcXIAZw== + dependencies: + "@jest/environment" "^29.1.2" + "@jest/fake-timers" "^29.1.2" + "@jest/globals" "^29.1.2" + "@jest/source-map" "^29.0.0" + "@jest/test-result" "^29.1.2" + "@jest/transform" "^29.1.2" + "@jest/types" "^29.1.2" + "@types/node" "*" + chalk "^4.0.0" + cjs-module-lexer "^1.0.0" + collect-v8-coverage "^1.0.0" + glob "^7.1.3" + graceful-fs "^4.2.9" + jest-haste-map "^29.1.2" + jest-message-util "^29.1.2" + jest-mock "^29.1.2" + jest-regex-util "^29.0.0" + jest-resolve "^29.1.2" + jest-snapshot "^29.1.2" + jest-util "^29.1.2" + slash "^3.0.0" + strip-bom "^4.0.0" + +jest-runtime@^29.7.0: + version "29.7.0" + resolved "https://registry.yarnpkg.com/jest-runtime/-/jest-runtime-29.7.0.tgz#efecb3141cf7d3767a3a0cc8f7c9990587d3d817" + integrity sha512-gUnLjgwdGqW7B4LvOIkbKs9WGbn+QLqRQQ9juC6HndeDiezIwhDP+mhMwHWCEcfQ5RUXa6OPnFF8BJh5xegwwQ== + dependencies: + "@jest/environment" "^29.7.0" + "@jest/fake-timers" "^29.7.0" + "@jest/globals" "^29.7.0" + "@jest/source-map" "^29.6.3" + "@jest/test-result" "^29.7.0" + "@jest/transform" "^29.7.0" + "@jest/types" "^29.6.3" + "@types/node" "*" + chalk "^4.0.0" + cjs-module-lexer "^1.0.0" + collect-v8-coverage "^1.0.0" + glob "^7.1.3" + graceful-fs "^4.2.9" + jest-haste-map "^29.7.0" + jest-message-util "^29.7.0" + jest-mock "^29.7.0" + jest-regex-util "^29.6.3" + jest-resolve "^29.7.0" + jest-snapshot "^29.7.0" + jest-util "^29.7.0" + slash "^3.0.0" + strip-bom "^4.0.0" + +jest-snapshot@^29.1.2: + version "29.1.2" + resolved "https://registry.yarnpkg.com/jest-snapshot/-/jest-snapshot-29.1.2.tgz#7dd277e88c45f2d2ff5888de1612e63c7ceb575b" + integrity sha512-rYFomGpVMdBlfwTYxkUp3sjD6usptvZcONFYNqVlaz4EpHPnDvlWjvmOQ9OCSNKqYZqLM2aS3wq01tWujLg7gg== + dependencies: + "@babel/core" "^7.11.6" + "@babel/generator" "^7.7.2" + "@babel/plugin-syntax-jsx" "^7.7.2" + "@babel/plugin-syntax-typescript" "^7.7.2" + "@babel/traverse" "^7.7.2" + "@babel/types" "^7.3.3" + "@jest/expect-utils" "^29.1.2" + "@jest/transform" "^29.1.2" + "@jest/types" "^29.1.2" + "@types/babel__traverse" "^7.0.6" + "@types/prettier" "^2.1.5" + babel-preset-current-node-syntax "^1.0.0" + chalk "^4.0.0" + expect "^29.1.2" + graceful-fs "^4.2.9" + jest-diff "^29.1.2" + jest-get-type "^29.0.0" + jest-haste-map "^29.1.2" + jest-matcher-utils "^29.1.2" + jest-message-util "^29.1.2" + jest-util "^29.1.2" + natural-compare "^1.4.0" + pretty-format "^29.1.2" + semver "^7.3.5" + +jest-snapshot@^29.7.0: + version "29.7.0" + resolved "https://registry.yarnpkg.com/jest-snapshot/-/jest-snapshot-29.7.0.tgz#c2c574c3f51865da1bb329036778a69bf88a6be5" + integrity sha512-Rm0BMWtxBcioHr1/OX5YCP8Uov4riHvKPknOGs804Zg9JGZgmIBkbtlxJC/7Z4msKYVbIJtfU+tKb8xlYNfdkw== + dependencies: + "@babel/core" "^7.11.6" + "@babel/generator" "^7.7.2" + "@babel/plugin-syntax-jsx" "^7.7.2" + "@babel/plugin-syntax-typescript" "^7.7.2" + "@babel/types" "^7.3.3" + "@jest/expect-utils" "^29.7.0" + "@jest/transform" "^29.7.0" + "@jest/types" "^29.6.3" + babel-preset-current-node-syntax "^1.0.0" + chalk "^4.0.0" + expect "^29.7.0" + graceful-fs "^4.2.9" + jest-diff "^29.7.0" + jest-get-type "^29.6.3" + jest-matcher-utils "^29.7.0" + jest-message-util "^29.7.0" + jest-util "^29.7.0" + natural-compare "^1.4.0" + pretty-format "^29.7.0" + semver "^7.5.3" -is-symbol@^1.0.2, is-symbol@^1.0.3: - version "1.0.4" - resolved "https://registry.yarnpkg.com/is-symbol/-/is-symbol-1.0.4.tgz#a6dac93b635b063ca6872236de88910a57af139c" - integrity sha512-C/CPBqKWnvdcxqIARxyOh4v1UUEOCHpgDa0WYgpKDFMszcrPcffg5uhwSgPCLD2WWxmq6isisz87tzT01tuGhg== +jest-util@^29.0.0, jest-util@^29.7.0: + version "29.7.0" + resolved "https://registry.yarnpkg.com/jest-util/-/jest-util-29.7.0.tgz#23c2b62bfb22be82b44de98055802ff3710fc0bc" + integrity sha512-z6EbKajIpqGKU56y5KBUgy1dt1ihhQJgWzUlZHArA/+X2ad7Cb5iF+AK1EWVL/Bo7Rz9uurpqw6SiBCefUbCGA== dependencies: - has-symbols "^1.0.2" + "@jest/types" "^29.6.3" + "@types/node" "*" + chalk "^4.0.0" + ci-info "^3.2.0" + graceful-fs "^4.2.9" + picomatch "^2.2.3" -is-typed-array@^1.1.3, is-typed-array@^1.1.9: - version "1.1.9" - resolved "https://registry.yarnpkg.com/is-typed-array/-/is-typed-array-1.1.9.tgz#246d77d2871e7d9f5aeb1d54b9f52c71329ece67" - integrity sha512-kfrlnTTn8pZkfpJMUgYD7YZ3qzeJgWUn8XfVYBARc4wnmNOmLbmuuaAs3q5fvB0UJOn6yHAKaGTPM7d6ezoD/A== +jest-util@^29.1.2: + version "29.1.2" + resolved "https://registry.yarnpkg.com/jest-util/-/jest-util-29.1.2.tgz#ac5798e93cb6a6703084e194cfa0898d66126df1" + integrity sha512-vPCk9F353i0Ymx3WQq3+a4lZ07NXu9Ca8wya6o4Fe4/aO1e1awMMprZ3woPFpKwghEOW+UXgd15vVotuNN9ONQ== dependencies: - available-typed-arrays "^1.0.5" - call-bind "^1.0.2" - es-abstract "^1.20.0" - for-each "^0.3.3" - has-tostringtag "^1.0.0" - -is-typedarray@^1.0.0, is-typedarray@~1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/is-typedarray/-/is-typedarray-1.0.0.tgz#e479c80858df0c1b11ddda6940f96011fcda4a9a" - integrity sha512-cyA56iCMHAh5CdzjJIa4aohJyeO1YbwLi3Jc35MmRU6poroFjIGZzUzupGiRPOjgHg9TLu43xbpwXk523fMxKA== + "@jest/types" "^29.1.2" + "@types/node" "*" + chalk "^4.0.0" + ci-info "^3.2.0" + graceful-fs "^4.2.9" + picomatch "^2.2.3" -is-valid-path@^0.1.1: - version "0.1.1" - resolved "https://registry.yarnpkg.com/is-valid-path/-/is-valid-path-0.1.1.tgz#110f9ff74c37f663e1ec7915eb451f2db93ac9df" - integrity sha512-+kwPrVDu9Ms03L90Qaml+79+6DZHqHyRoANI6IsZJ/g8frhnfchDOBCa0RbQ6/kdHt5CS5OeIEyrYznNuVN+8A== +jest-validate@^29.1.2: + version "29.1.2" + resolved "https://registry.yarnpkg.com/jest-validate/-/jest-validate-29.1.2.tgz#83a728b8f6354da2e52346878c8bc7383516ca51" + integrity sha512-k71pOslNlV8fVyI+mEySy2pq9KdXdgZtm7NHrBX8LghJayc3wWZH0Yr0mtYNGaCU4F1OLPXRkwZR0dBm/ClshA== dependencies: - is-invalid-path "^0.1.0" + "@jest/types" "^29.1.2" + camelcase "^6.2.0" + chalk "^4.0.0" + jest-get-type "^29.0.0" + leven "^3.1.0" + pretty-format "^29.1.2" -is-weakref@^1.0.2: - version "1.0.2" - resolved "https://registry.yarnpkg.com/is-weakref/-/is-weakref-1.0.2.tgz#9529f383a9338205e89765e0392efc2f100f06f2" - integrity sha512-qctsuLZmIQ0+vSSMfoVvyFe2+GSEvnmZ2ezTup1SBse9+twCCeial6EEi3Nc2KFcf6+qz2FBPnjXsk8xhKSaPQ== +jest-validate@^29.7.0: + version "29.7.0" + resolved "https://registry.yarnpkg.com/jest-validate/-/jest-validate-29.7.0.tgz#7bf705511c64da591d46b15fce41400d52147d9c" + integrity sha512-ZB7wHqaRGVw/9hST/OuFUReG7M8vKeq0/J2egIGLdvjHCmYqGARhzXmtgi+gVeZ5uXFF219aOc3Ls2yLg27tkw== dependencies: - call-bind "^1.0.2" + "@jest/types" "^29.6.3" + camelcase "^6.2.0" + chalk "^4.0.0" + jest-get-type "^29.6.3" + leven "^3.1.0" + pretty-format "^29.7.0" -is-wsl@^2.2.0: - version "2.2.0" - resolved "https://registry.yarnpkg.com/is-wsl/-/is-wsl-2.2.0.tgz#74a4c76e77ca9fd3f932f290c17ea326cd157271" - integrity sha512-fKzAra0rGJUUBwGBgNkHZuToZcn+TtXHpeCgmkMJMMYx1sQDYaCSyjJBSCa2nH1DGm7s3n1oBnohoVTBaN7Lww== +jest-watcher@^29.1.2: + version "29.1.2" + resolved "https://registry.yarnpkg.com/jest-watcher/-/jest-watcher-29.1.2.tgz#de21439b7d889e2fcf62cc2a4779ef1a3f1f3c62" + integrity sha512-6JUIUKVdAvcxC6bM8/dMgqY2N4lbT+jZVsxh0hCJRbwkIEnbr/aPjMQ28fNDI5lB51Klh00MWZZeVf27KBUj5w== dependencies: - is-docker "^2.0.0" - -isarray@0.0.1: - version "0.0.1" - resolved "https://registry.yarnpkg.com/isarray/-/isarray-0.0.1.tgz#8a18acfca9a8f4177e09abfc6038939b05d1eedf" - integrity sha512-D2S+3GLxWH+uhrNEcoh/fnmYeP8E8/zHl644d/jdA0g2uyXvy3sb0qxotE+ne0LtccHknQzWwZEzhak7oJ0COQ== + "@jest/test-result" "^29.1.2" + "@jest/types" "^29.1.2" + "@types/node" "*" + ansi-escapes "^4.2.1" + chalk "^4.0.0" + emittery "^0.10.2" + jest-util "^29.1.2" + string-length "^4.0.1" -isarray@2.0.1: - version "2.0.1" - resolved "https://registry.yarnpkg.com/isarray/-/isarray-2.0.1.tgz#a37d94ed9cda2d59865c9f76fe596ee1f338741e" - integrity sha512-c2cu3UxbI+b6kR3fy0nRnAhodsvR9dx7U5+znCOzdj6IfP3upFURTr0Xl5BlQZNKZjEtxrmVyfSdeE3O57smoQ== +jest-watcher@^29.7.0: + version "29.7.0" + resolved "https://registry.yarnpkg.com/jest-watcher/-/jest-watcher-29.7.0.tgz#7810d30d619c3a62093223ce6bb359ca1b28a2f2" + integrity sha512-49Fg7WXkU3Vl2h6LbLtMQ/HyB6rXSIX7SqvBLQmssRBGN9I0PNvPmAmCWSOY6SOvrjhI/F7/bGAv9RtnsPA03g== + dependencies: + "@jest/test-result" "^29.7.0" + "@jest/types" "^29.6.3" + "@types/node" "*" + ansi-escapes "^4.2.1" + chalk "^4.0.0" + emittery "^0.13.1" + jest-util "^29.7.0" + string-length "^4.0.1" -isarray@^1.0.0, isarray@~1.0.0: - version "1.0.0" - resolved "https://registry.yarnpkg.com/isarray/-/isarray-1.0.0.tgz#bb935d48582cba168c06834957a54a3e07124f11" - integrity sha512-VLghIWNM6ELQzo7zwmcg0NmTVyWKYjvIeM83yjp0wRDTmUnrM678fQbcKBo6n2CJEF0szoG//ytg+TKla89ALQ== +jest-worker@^29.1.2: + version "29.1.2" + resolved "https://registry.yarnpkg.com/jest-worker/-/jest-worker-29.1.2.tgz#a68302af61bce82b42a9a57285ca7499d29b2afc" + integrity sha512-AdTZJxKjTSPHbXT/AIOjQVmoFx0LHFcVabWu0sxI7PAy7rFf8c0upyvgBKgguVXdM4vY74JdwkyD4hSmpTW8jA== + dependencies: + "@types/node" "*" + jest-util "^29.1.2" + merge-stream "^2.0.0" + supports-color "^8.0.0" -isexe@^2.0.0: - version "2.0.0" - resolved "https://registry.yarnpkg.com/isexe/-/isexe-2.0.0.tgz#e8fbf374dc556ff8947a10dcb0572d633f2cfa10" - integrity sha512-RHxMLp9lnKHGHRng9QFhRCMbYAcVpn69smSGcq3f36xjgVVWThj4qqLbTLlq7Ssj8B+fIQ1EuCEGI2lKsyQeIw== +jest-worker@^29.7.0: + version "29.7.0" + resolved "https://registry.yarnpkg.com/jest-worker/-/jest-worker-29.7.0.tgz#acad073acbbaeb7262bd5389e1bcf43e10058d4a" + integrity sha512-eIz2msL/EzL9UFTFFx7jBTkeZfku0yUAyZZZmJ93H2TYEiroIx2PQjEXcwYtYl8zXCxb+PAmA2hLIt/6ZEkPHw== + dependencies: + "@types/node" "*" + jest-util "^29.7.0" + merge-stream "^2.0.0" + supports-color "^8.0.0" -isstream@~0.1.2: - version "0.1.2" - resolved "https://registry.yarnpkg.com/isstream/-/isstream-0.1.2.tgz#47e63f7af55afa6f92e1500e690eb8b8529c099a" - integrity sha512-Yljz7ffyPbrLpLngrMtZ7NduUgVvi6wG9RJ9IUcyCd59YQ911PBJphODUcbOVbqYfxe1wuYf/LJ8PauMRwsM/g== +jest@^29.1.2: + version "29.1.2" + resolved "https://registry.yarnpkg.com/jest/-/jest-29.1.2.tgz#f821a1695ffd6cd0efc3b59d2dfcc70a98582499" + integrity sha512-5wEIPpCezgORnqf+rCaYD1SK+mNN7NsstWzIsuvsnrhR/hSxXWd82oI7DkrbJ+XTD28/eG8SmxdGvukrGGK6Tw== + dependencies: + "@jest/core" "^29.1.2" + "@jest/types" "^29.1.2" + import-local "^3.0.2" + jest-cli "^29.1.2" jmespath@0.15.0: version "0.15.0" @@ -3237,9 +5196,14 @@ joi@^17.6.0: "@sideway/pinpoint" "^2.0.0" js-sdsl@^4.1.4: - version "4.3.0" - resolved "https://registry.yarnpkg.com/js-sdsl/-/js-sdsl-4.3.0.tgz#aeefe32a451f7af88425b11fdb5f58c90ae1d711" - integrity sha512-mifzlm2+5nZ+lEcLJMoBK0/IH/bDg8XnJfd/Wq6IP+xoCjLZsTOnV2QpxlVbX9bMnkl5PdEjNtBJ9Cj1NjifhQ== + version "4.1.5" + resolved "https://registry.yarnpkg.com/js-sdsl/-/js-sdsl-4.1.5.tgz#1ff1645e6b4d1b028cd3f862db88c9d887f26e2a" + integrity sha512-08bOAKweV2NUC1wqTtf3qZlnpOX/R2DU9ikpjOHs0H+ibQv3zpncVQg6um4uYtRtrwIX8M4Nh3ytK4HGlYAq7Q== + +js-tokens@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/js-tokens/-/js-tokens-4.0.0.tgz#19203fb59991df98e3a287050d4647cdeaf32499" + integrity sha512-RdJUflcE3cUzKiMqQgsCu06FPu9UdIJO0beYbPhHN4k6apgJtifcoCtT9bcxOpYBtpD2kCM6Sbzg4CausW/PKQ== js-yaml@^3.13.1, js-yaml@^3.14.0: version "3.14.1" @@ -3261,6 +5225,11 @@ jsbn@~0.1.0: resolved "https://registry.yarnpkg.com/jsbn/-/jsbn-0.1.1.tgz#a5e654c2e5a2deb5f201d96cefbca80c0ef2f513" integrity sha512-UVU9dibq2JcFWxQPA6KCqj5O42VOmAY3zQUfEKxU0KpTGXwNoCjkX1e13eHNvw/xPynt6pU0rZ1htjWTNTSXsg== +jsesc@^2.5.1: + version "2.5.2" + resolved "https://registry.yarnpkg.com/jsesc/-/jsesc-2.5.2.tgz#80564d2e483dacf6e8ef209650a67df3f0c283a4" + integrity sha512-OYu7XEzjkCQ3C5Ps3QIZsQfNpqoJyZZA99wd9aWd05NCtC5pWOkShK2mkL6HXQR6/Cy2lbNdPlZBpuQHXE63gA== + json-edm-parser@~0.1.2: version "0.1.2" resolved "https://registry.yarnpkg.com/json-edm-parser/-/json-edm-parser-0.1.2.tgz#1e60b0fef1bc0af67bc0d146dfdde5486cd615b4" @@ -3273,6 +5242,11 @@ json-parse-better-errors@^1.0.1: resolved "https://registry.yarnpkg.com/json-parse-better-errors/-/json-parse-better-errors-1.0.2.tgz#bb867cfb3450e69107c131d1c514bab3dc8bcaa9" integrity sha512-mrqyZKfX5EhL7hvqcV6WG1yYjnjeuYDzDhhcAAUrq8Po85NBQBJP+ZDUT75qZQ98IkUoBqdkExkukOU7Ts2wrw== +json-parse-even-better-errors@^2.3.0: + version "2.3.1" + resolved "https://registry.yarnpkg.com/json-parse-even-better-errors/-/json-parse-even-better-errors-2.3.1.tgz#7c47805a94319928e05777405dc12e1f7a4ee02d" + integrity sha512-xyFwyhro/JEof6Ghe2iz2NcXoj2sloNsWr/XsERDK/oiPCfaNhl5ONfp+jQdAZRQQ0IJWNzH9zIZF7li91kh2w== + json-schema-deref-sync@^0.14.0: version "0.14.0" resolved "https://registry.yarnpkg.com/json-schema-deref-sync/-/json-schema-deref-sync-0.14.0.tgz#d55cdb8e3d7c26f63f28ffe67601ad2f69bfd779" @@ -3319,6 +5293,11 @@ json5@^1.0.1: dependencies: minimist "^1.2.0" +json5@^2.2.1, json5@^2.2.3: + version "2.2.3" + resolved "https://registry.yarnpkg.com/json5/-/json5-2.2.3.tgz#78cd6f1a19bdc12b73db5ad0c61efd66c1e29283" + integrity sha512-XmOWe7eyHYH14cLdVPoyg+GOH3rYX++KpzrylJwSW98t3Nk+U8XOl8FWKOgwtzdb8lXGf6zYwDUzeHMWfxasyg== + jsonparse@^1.2.0: version "1.3.1" resolved "https://registry.yarnpkg.com/jsonparse/-/jsonparse-1.3.1.tgz#3f4dae4a91fac315f71062f8521cc239f1366280" @@ -3394,6 +5373,11 @@ jws@^4.0.0: jwa "^2.0.0" safe-buffer "^5.0.1" +kleur@^3.0.3: + version "3.0.3" + resolved "https://registry.yarnpkg.com/kleur/-/kleur-3.0.3.tgz#a79c9ecc86ee1ce3fa6206d1216c501f147fc07e" + integrity sha512-eTIzlVOSUR+JxdDFepEYcBMtZ9Qqdef+rnzWdRZuMbOywu5tO2w2N7rqjoANZ5k9vywhL6Br1VRjUIgTQx4E8w== + kuler@^2.0.0: version "2.0.0" resolved "https://registry.yarnpkg.com/kuler/-/kuler-2.0.0.tgz#e2c570a3800388fb44407e851531c1d670b061b3" @@ -3525,6 +5509,11 @@ levelup@~0.19.0: semver "~5.1.0" xtend "~3.0.0" +leven@^3.1.0: + version "3.1.0" + resolved "https://registry.yarnpkg.com/leven/-/leven-3.1.0.tgz#77891de834064cccba82ae7842bb6b14a13ed7f2" + integrity sha512-qsda+H8jTaUaN/x5vzW2rzc+8Rw4TAQ/4KjB46IwK5VH+IlVeeeje/EoZRpiXvIqjFgK84QffqPztGI3VBLG1A== + levn@^0.4.1: version "0.4.1" resolved "https://registry.yarnpkg.com/levn/-/levn-0.4.1.tgz#ae4562c007473b932a6200d403268dd2fffc6ade" @@ -3533,6 +5522,11 @@ levn@^0.4.1: prelude-ls "^1.2.1" type-check "~0.4.0" +lines-and-columns@^1.1.6: + version "1.2.4" + resolved "https://registry.yarnpkg.com/lines-and-columns/-/lines-and-columns-1.2.4.tgz#eca284f75d2965079309dc0ad9255abb2ebc1632" + integrity sha512-7ylylesZQ/PV29jhEDl3Ufjo6ZX7gCqJr5F7PKrqc93v7fzSymt1BpwEU8nAUXs8qzzvqhbjhK5QZg6Mt/HkBg== + linkify-it@^2.0.0: version "2.2.0" resolved "https://registry.yarnpkg.com/linkify-it/-/linkify-it-2.2.0.tgz#e3b54697e78bf915c70a38acd78fd09e0058b1cf" @@ -3558,6 +5552,13 @@ locate-path@^2.0.0: p-locate "^2.0.0" path-exists "^3.0.0" +locate-path@^5.0.0: + version "5.0.0" + resolved "https://registry.yarnpkg.com/locate-path/-/locate-path-5.0.0.tgz#1afba396afd676a6d42504d0a67a3a7eb9f62aa0" + integrity sha512-t7hw9pI+WvuwNJXwk5zVHpyhIqzg2qTlklJOf0mVxGSbe3Fp2VieZcduNYjaLDoy6p9uGpQEGWG87WpMKlNq8g== + dependencies: + p-locate "^4.1.0" + locate-path@^6.0.0: version "6.0.0" resolved "https://registry.yarnpkg.com/locate-path/-/locate-path-6.0.0.tgz#55321eb309febbc59c4801d931a72452a681d286" @@ -3681,6 +5682,11 @@ lodash.keys@^3.0.0: lodash.isarguments "^3.0.0" lodash.isarray "^3.0.0" +lodash.memoize@4.x: + version "4.1.2" + resolved "https://registry.yarnpkg.com/lodash.memoize/-/lodash.memoize-4.1.2.tgz#bcc6c49a42a2840ed997f323eada5ecd182e0bfe" + integrity sha512-t7j+NzmgnQzTAYXcsHYLgimltOV1MXHtlOWf6GjL9Kj8GK5FInw5JotxvbOs+IvV1/Dzo04/fCGfLVs7aXb4Ag== + lodash.merge@^4.6.2: version "4.6.2" resolved "https://registry.yarnpkg.com/lodash.merge/-/lodash.merge-4.6.2.tgz#558aa53b43b661e1925a0afdfa36a9a1085fe57a" @@ -3745,6 +5751,13 @@ lru-cache@^4.1.3: pseudomap "^1.0.2" yallist "^2.1.2" +lru-cache@^5.1.1: + version "5.1.1" + resolved "https://registry.yarnpkg.com/lru-cache/-/lru-cache-5.1.1.tgz#1da27e6710271947695daf6848e847f01d84b920" + integrity sha512-KpNARQA3Iwv+jTA0utUVVbrh+Jlrr1Fv0e56GGzAFOXN7dk/FviaDW8LHmK52DlcH4WP2n6gI8vN1aesBFgo9w== + dependencies: + yallist "^3.0.2" + lru-cache@^6.0.0: version "6.0.0" resolved "https://registry.yarnpkg.com/lru-cache/-/lru-cache-6.0.0.tgz#6d6fe6570ebd96aaf90fcad1dafa3b2566db3a94" @@ -3762,6 +5775,18 @@ ltgt@~2.1.1: resolved "https://registry.yarnpkg.com/ltgt/-/ltgt-2.1.3.tgz#10851a06d9964b971178441c23c9e52698eece34" integrity sha512-5VjHC5GsENtIi5rbJd+feEpDKhfr7j0odoUR2Uh978g+2p93nd5o34cTjQWohXsPsCZeqoDnIqEf88mPCe0Pfw== +make-dir@^3.0.0: + version "3.1.0" + resolved "https://registry.yarnpkg.com/make-dir/-/make-dir-3.1.0.tgz#415e967046b3a7f1d185277d84aa58203726a13f" + integrity sha512-g3FeP20LNwhALb/6Cz6Dd4F2ngze0jz7tbzrD2wAV+o9FeNHe4rL+yK2md0J/fiSf1sa1ADhXqi5+oVwOM/eGw== + dependencies: + semver "^6.0.0" + +make-error@1.x, make-error@^1.1.1: + version "1.3.6" + resolved "https://registry.yarnpkg.com/make-error/-/make-error-1.3.6.tgz#2eb2e37ea9b67c4891f684a1394799af484cf7a2" + integrity sha512-s8UhlNe7vPKomQhC1qFelMokr/Sc3AgNbso3n74mVPA5LTZwkB9NlXf4XPamLxJE8h0gh73rM94xvwRT2CVInw== + make-fetch-happen@^9.1.0: version "9.1.0" resolved "https://registry.yarnpkg.com/make-fetch-happen/-/make-fetch-happen-9.1.0.tgz#53085a09e7971433e6765f7971bf63f4e05cb968" @@ -3784,6 +5809,13 @@ make-fetch-happen@^9.1.0: socks-proxy-agent "^6.0.0" ssri "^8.0.0" +makeerror@1.0.12: + version "1.0.12" + resolved "https://registry.yarnpkg.com/makeerror/-/makeerror-1.0.12.tgz#3e5dd2079a82e812e983cc6610c4a2cb0eaa801a" + integrity sha512-JmqCvUhmt43madlpFzG4BQzG2Z3m6tvQDNKdClZnO3VbIudJYmxsT0FNJMeiB2+JTSlTQTSbU8QdesVmwJcmLg== + dependencies: + tmpl "1.0.5" + markdown-it@10.0.0: version "10.0.0" resolved "https://registry.yarnpkg.com/markdown-it/-/markdown-it-10.0.0.tgz#abfc64f141b1722d663402044e43927f1f50a8dc" @@ -3862,11 +5894,29 @@ merge-descriptors@1.0.1: resolved "https://registry.yarnpkg.com/merge-descriptors/-/merge-descriptors-1.0.1.tgz#b00aaa556dd8b44568150ec9d1b953f3f90cbb61" integrity sha512-cCi6g3/Zr1iqQi6ySbseM1Xvooa98N0w31jzUYrXPX2xqObmFGHJ0tQ5u74H3mVh7wLouTseZyYIq39g8cNp1w== +merge-stream@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/merge-stream/-/merge-stream-2.0.0.tgz#52823629a14dd00c9770fb6ad47dc6310f2c1f60" + integrity sha512-abv/qOcuPfk3URPfDzmZU1LKmuw8kT+0nIHvKrKgFrwifol/doWcdA4ZqsWQ8ENrFKkd67Mfpo/LovbIUsbt3w== + +merge2@^1.3.0, merge2@^1.4.1: + version "1.4.1" + resolved "https://registry.yarnpkg.com/merge2/-/merge2-1.4.1.tgz#4368892f885e907455a6fd7dc55c0c9d404990ae" + integrity sha512-8q7VEgMJW4J8tcfVPy8g09NcQwZdbwFEqhe/WZkoIzjn/3TGDwtOCYtXGxA3O8tPzpczCCDgv+P2P5y00ZJOOg== + methods@~1.1.2: version "1.1.2" resolved "https://registry.yarnpkg.com/methods/-/methods-1.1.2.tgz#5529a4d67654134edcc5266656835b0f851afcee" integrity sha512-iclAHeNqNm68zFtnZ0e+1L2yUIdvzNoauKU4WBA3VvH/vPFieF7qfRlwUZU+DA9P9bPXIS90ulxoUoCH23sV2w== +micromatch@^4.0.4: + version "4.0.5" + resolved "https://registry.yarnpkg.com/micromatch/-/micromatch-4.0.5.tgz#bc8999a7cbbf77cdc89f132f6e467051b49090c6" + integrity sha512-DMy+ERcEW2q8Z2Po+WNXuw3c5YaUSFjAO5GsJqfEl7UjvtIuFKO6ZrKvcItdy98dwFI2N1tg3zNIdKaQT+aNdA== + dependencies: + braces "^3.0.2" + picomatch "^2.3.1" + mime-db@1.52.0: version "1.52.0" resolved "https://registry.yarnpkg.com/mime-db/-/mime-db-1.52.0.tgz#bbabcdc02859f4987301c856e3387ce5ec43bf70" @@ -3889,7 +5939,12 @@ mime@^2.2.0: resolved "https://registry.yarnpkg.com/mime/-/mime-2.6.0.tgz#a2a682a95cd4d0cb1d6257e28f83da7e35800367" integrity sha512-USPkMeET31rOMiarsBNIHZKLGgvKc/LrjofAnBlOttf5ajRvqiRA8QsenbcooctK6d6Ts6aqZXBA+XbkKthiQg== -minimatch@^3.0.2, minimatch@^3.0.4, minimatch@^3.0.5, minimatch@^3.1.1, minimatch@^3.1.2: +mimic-fn@^2.1.0: + version "2.1.0" + resolved "https://registry.yarnpkg.com/mimic-fn/-/mimic-fn-2.1.0.tgz#7ed2c2ccccaf84d3ffcb7a69b57711fc2083401b" + integrity sha512-OqbOk5oEQeAZ8WXWydlu9HJjz9WVdEIvamMCcXmuqUYjTknH/sqsWvhQ3vgwKFRR1HpjvNBKQ37nbJgYzGqGcg== + +minimatch@^3.0.2, minimatch@^3.0.4, minimatch@^3.1.1, minimatch@^3.1.2: version "3.1.2" resolved "https://registry.yarnpkg.com/minimatch/-/minimatch-3.1.2.tgz#19cd194bfd3e428f049a70817c038d89ab4be35b" integrity sha512-J7p63hRiAjw1NDEww1W7i37+ByIrOWO5XQQAzZ3VOcL0PNybwpfmV/N05zFAzwQ9USyEcX6t3UO+K5aqBQOIHw== @@ -4164,6 +6219,11 @@ node-gyp@^8.0.0: tar "^6.1.2" which "^2.0.2" +node-int64@^0.4.0: + version "0.4.0" + resolved "https://registry.yarnpkg.com/node-int64/-/node-int64-0.4.0.tgz#87a9065cdb355d3182d8f94ce11188b825c68a3b" + integrity sha512-O5lz91xSOeoXP6DulyHfllpq+Eg00MWitZIbtPfoSEvqIHdl5gfcY6hYzDWnj0qD5tz52PI08u9qUvSVeUBeHw== + node-rdkafka-prometheus@^1.0.0: version "1.0.0" resolved "https://registry.yarnpkg.com/node-rdkafka-prometheus/-/node-rdkafka-prometheus-1.0.0.tgz#cb4e0e28043574afd8320e4fba846454880c8994" @@ -4180,6 +6240,11 @@ node-rdkafka@^2.12.0: bindings "^1.3.1" nan "^2.14.0" +node-releases@^2.0.13: + version "2.0.13" + resolved "https://registry.yarnpkg.com/node-releases/-/node-releases-2.0.13.tgz#d5ed1627c23e3461e819b02e57b75e4899b1c81d" + integrity sha512-uYr7J37ae/ORWdZeQ1xxMJe3NtdmqMC/JZK+geofDrkLUApKRHPd18/TxtBOJ4A0/+uUIliorNrfYV6s1b02eQ== + node-schedule@^1.2.0, node-schedule@^1.3.2: version "1.3.3" resolved "https://registry.yarnpkg.com/node-schedule/-/node-schedule-1.3.3.tgz#f8e01c5fb9597f09ecf9c4c25d6938e5e7a06f48" @@ -4222,6 +6287,11 @@ normalize-package-data@^2.3.2: semver "2 || 3 || 4 || 5" validate-npm-package-license "^3.0.1" +normalize-path@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/normalize-path/-/normalize-path-3.0.0.tgz#0dcd69ff23a1c9b11fd0978316644a0388216a65" + integrity sha512-6eZs5Ls3WtCisHWp9S2GUy8dqkpGi4BVSz3GaqiE6ezub0512ESztXUwUB6C6IKbQkY2Pnb/mD4WYojCRwcwLA== + notation@^1.3.6: version "1.3.6" resolved "https://registry.yarnpkg.com/notation/-/notation-1.3.6.tgz#bc87b88d1f1159e2931e7f9317a3020313790321" @@ -4242,6 +6312,13 @@ npm-run-all@~4.1.5: shell-quote "^1.6.1" string.prototype.padend "^3.0.0" +npm-run-path@^4.0.1: + version "4.0.1" + resolved "https://registry.yarnpkg.com/npm-run-path/-/npm-run-path-4.0.1.tgz#b7ecd1e5ed53da8e37a55e1c2269e0b97ed748ea" + integrity sha512-S48WzZW777zhNIrn7gxOlISNAqi9ZC/uQFnRdbeIHhZhCA6UqpkOT8T1G7BvfdgP4Er8gF4sUbaS0i7QvIfCWw== + dependencies: + path-key "^3.0.0" + npmlog@^6.0.0: version "6.0.2" resolved "https://registry.yarnpkg.com/npmlog/-/npmlog-6.0.2.tgz#c8166017a42f2dea92d6453168dd865186a70830" @@ -4291,7 +6368,17 @@ object-keys@^1.1.1: resolved "https://registry.yarnpkg.com/object-keys/-/object-keys-1.1.1.tgz#1c47f272df277f3b1daf061677d9c82e2322c60e" integrity sha512-NuAESUOUMrlIXOfHKzD6bpPu3tYt3xvjNdRIQ+FeT0lNb4K8WR70CaDxhuNguS2XG+GjkyMwOzsN5ZktImfhLA== -object.assign@^4.1.0, object.assign@^4.1.2: +object.assign@^4.1.0: + version "4.1.4" + resolved "https://registry.yarnpkg.com/object.assign/-/object.assign-4.1.4.tgz#9673c7c7c351ab8c4d0b516f4343ebf4dfb7799f" + integrity sha512-1mxKf0e58bvyjSCtKYY4sRe9itRk3PJpquJOjeIkz885CczcI4IvJJDLPS72oowuSh+pBxUFROpX+TU++hxhZQ== + dependencies: + call-bind "^1.0.2" + define-properties "^1.1.4" + has-symbols "^1.0.3" + object-keys "^1.1.1" + +object.assign@^4.1.2: version "4.1.2" resolved "https://registry.yarnpkg.com/object.assign/-/object.assign-4.1.2.tgz#0ed54a342eceb37b38ff76eb831a0e788cb63940" integrity sha512-ixT2L5THXsApyiUPYKmW+2EHpXXe5Ii3M+f4e+aJFAHao5amFRW6J0OO6c/LU8Be47utCx2GL89hxGB6XSmKuQ== @@ -4340,6 +6427,13 @@ one-time@^1.0.0: dependencies: fn.name "1.x.x" +onetime@^5.1.2: + version "5.1.2" + resolved "https://registry.yarnpkg.com/onetime/-/onetime-5.1.2.tgz#d0e96ebb56b07476df1dd9c4806e5237985ca45e" + integrity sha512-kbpaSSGJTWdAY5KPVeMOKXSrPtr8C8C7wodJbcsd51jRnmD+GZu8Y0VoU6Dm5Z4vWr0Ig/1NKuWRKf7j5aaYSg== + dependencies: + mimic-fn "^2.1.0" + open@^8.0.0: version "8.4.0" resolved "https://registry.yarnpkg.com/open/-/open-8.4.0.tgz#345321ae18f8138f82565a910fdc6b39e8c244f8" @@ -4380,7 +6474,14 @@ p-limit@^1.1.0: dependencies: p-try "^1.0.0" -p-limit@^3.0.2: +p-limit@^2.2.0: + version "2.3.0" + resolved "https://registry.yarnpkg.com/p-limit/-/p-limit-2.3.0.tgz#3dd33c647a214fdfffd835933eb086da0dc21db1" + integrity sha512-//88mFWSJx8lxCzwdAABTJL2MyWB12+eIY7MDL2SqLmAkeKU9qxRvWuSyTjm3FUmpBEMuFfckAIqEaVGUDxb6w== + dependencies: + p-try "^2.0.0" + +p-limit@^3.0.2, p-limit@^3.1.0: version "3.1.0" resolved "https://registry.yarnpkg.com/p-limit/-/p-limit-3.1.0.tgz#e1daccbe78d0d1388ca18c64fea38e3e57e3706b" integrity sha512-TYOanM3wGwNGsZN2cVTYPArw454xnXj5qmWF1bEoAc4+cU/ol7GVh7odevjp1FNHduHc3KZMcFduxU5Xc6uJRQ== @@ -4394,6 +6495,13 @@ p-locate@^2.0.0: dependencies: p-limit "^1.1.0" +p-locate@^4.1.0: + version "4.1.0" + resolved "https://registry.yarnpkg.com/p-locate/-/p-locate-4.1.0.tgz#a3428bb7088b3a60292f66919278b7c297ad4f07" + integrity sha512-R79ZZ/0wAxKGu3oYMlz8jy/kbhsNrS7SKZ7PxEHBgJ5+F2mtFW2fK2cOtBh1cHYkQsbzFV7I+EoRKe6Yt0oK7A== + dependencies: + p-limit "^2.2.0" + p-locate@^5.0.0: version "5.0.0" resolved "https://registry.yarnpkg.com/p-locate/-/p-locate-5.0.0.tgz#83c8315c6785005e3bd021839411c9e110e6d834" @@ -4418,6 +6526,11 @@ p-try@^1.0.0: resolved "https://registry.yarnpkg.com/p-try/-/p-try-1.0.0.tgz#cbc79cdbaf8fd4228e13f621f2b1a237c1b207b3" integrity sha512-U1etNYuMJoIz3ZXSrrySFjsXQTWOx2/jdi86L+2pRvph/qMKL6sbcCYdH23fqsbm8TH2Gn0OybpT4eSFlCVHww== +p-try@^2.0.0: + version "2.2.0" + resolved "https://registry.yarnpkg.com/p-try/-/p-try-2.2.0.tgz#cb2868540e313d61de58fafbe35ce9004d5540e6" + integrity sha512-R4nPAVTAU0B9D35/Gk3uJf/7XYbQcyohSKdvAxIRSNghFl4e71hVoGnBNQz9cWaXxO2I10KTC+3jMdvvoKw6dQ== + parent-module@^1.0.0: version "1.0.1" resolved "https://registry.yarnpkg.com/parent-module/-/parent-module-1.0.1.tgz#691d2709e78c79fae3a156622452d00762caaaa2" @@ -4433,6 +6546,16 @@ parse-json@^4.0.0: error-ex "^1.3.1" json-parse-better-errors "^1.0.1" +parse-json@^5.2.0: + version "5.2.0" + resolved "https://registry.yarnpkg.com/parse-json/-/parse-json-5.2.0.tgz#c76fc66dee54231c962b22bcc8a72cf2f99753cd" + integrity sha512-ayCKvm/phCGxOkYRSCM82iDwct8/EonSEgCSxWxD7ve6jHggsFl4fZVQBPRNgQoKiuV/odhFrGzQXZwbifC8Rg== + dependencies: + "@babel/code-frame" "^7.0.0" + error-ex "^1.3.1" + json-parse-even-better-errors "^2.3.0" + lines-and-columns "^1.1.6" + parseqs@0.0.6: version "0.0.6" resolved "https://registry.yarnpkg.com/parseqs/-/parseqs-0.0.6.tgz#8e4bb5a19d1cdc844a08ac974d34e273afa670d5" @@ -4468,7 +6591,7 @@ path-key@^2.0.1: resolved "https://registry.yarnpkg.com/path-key/-/path-key-2.0.1.tgz#411cadb574c5a140d3a4b1910d40d80cc9f40b40" integrity sha512-fEHGKCSmUSDPv4uoj8AlD+joPlq3peND+HRYyxFz4KPw4z926S/b8rIuFs2FYJg3BwsxJf6A9/3eIdLaYC+9Dw== -path-key@^3.1.0: +path-key@^3.0.0, path-key@^3.1.0: version "3.1.1" resolved "https://registry.yarnpkg.com/path-key/-/path-key-3.1.1.tgz#581f6ade658cbba65a0d3380de7753295054f375" integrity sha512-ojmeN0qd+y0jszEtoY48r0Peq5dwMEkIlCOu6Q5f41lfkswXuKtYrhgoTpLnyIcHm24Uhqx+5Tqm2InSwLhE6Q== @@ -4497,11 +6620,26 @@ path-type@^3.0.0: dependencies: pify "^3.0.0" +path-type@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/path-type/-/path-type-4.0.0.tgz#84ed01c0a7ba380afe09d90a8c180dcd9d03043b" + integrity sha512-gDKb8aZMDeD/tZWs9P6+q0J9Mwkdl6xMV8TjnGP3qJVJ06bdMgkbBlLU8IdfOsIsFz2BW1rNVT3XuNEl8zPAvw== + performance-now@^2.1.0: version "2.1.0" resolved "https://registry.yarnpkg.com/performance-now/-/performance-now-2.1.0.tgz#6309f4e0e5fa913ec1c69307ae364b4b377c9e7b" integrity sha512-7EAHlyLHI56VEIdK57uwHdHKIaAGbnXPiw0yWbarQZOKaKpvUIgW0jWRVLiatnM+XXlSwsanIBH/hzGMJulMow== +picocolors@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/picocolors/-/picocolors-1.0.0.tgz#cb5bdc74ff3f51892236eaf79d68bc44564ab81c" + integrity sha512-1fygroTLlHu66zi26VoTDv8yRgm0Fccecssto+MhsZ0D/DGW2sm8E8AjW7NU5VVTRt5GxbeZ5qBuJr+HyLYkjQ== + +picomatch@^2.0.4, picomatch@^2.2.3, picomatch@^2.3.1: + version "2.3.1" + resolved "https://registry.yarnpkg.com/picomatch/-/picomatch-2.3.1.tgz#3ba3833733646d9d3e4995946c1365a67fb07a42" + integrity sha512-JU3teHTNjmE2VCGFzuY8EXzCDVwEqB2a8fsIvwaStHhAWJEeVd1o1QD80CU6+ZdEXXSLbSsuLwJjkCBWqRQUVA== + pidtree@^0.3.0: version "0.3.1" resolved "https://registry.yarnpkg.com/pidtree/-/pidtree-0.3.1.tgz#ef09ac2cc0533df1f3250ccf2c4d366b0d12114a" @@ -4517,11 +6655,41 @@ pify@^4.0.0: resolved "https://registry.yarnpkg.com/pify/-/pify-4.0.1.tgz#4b2cd25c50d598735c50292224fd8c6df41e3231" integrity sha512-uB80kBFb/tfd68bVleG9T5GGsGPjJrLAUpR5PZIrhBnIaRTQRjqdJSsIKkOP6OAIFbj7GOrcudc5pNjZ+geV2g== +pirates@^4.0.4: + version "4.0.6" + resolved "https://registry.yarnpkg.com/pirates/-/pirates-4.0.6.tgz#3018ae32ecfcff6c29ba2267cbf21166ac1f36b9" + integrity sha512-saLsH7WeYYPiD25LDuLRRY/i+6HaPYr6G1OUlN39otzkSTxKnubR9RTxS3/Kk50s1g2JTgFwWQDQyplC5/SHZg== + +pkg-dir@^4.2.0: + version "4.2.0" + resolved "https://registry.yarnpkg.com/pkg-dir/-/pkg-dir-4.2.0.tgz#f099133df7ede422e81d1d8448270eeb3e4261f3" + integrity sha512-HRDzbaKjC+AOWVXxAU/x54COGeIv9eb+6CkDSQoNTt4XyWoIJvuPsXizxu/Fr23EiekbtZwmh1IcIG/l/a10GQ== + dependencies: + find-up "^4.0.0" + prelude-ls@^1.2.1: version "1.2.1" resolved "https://registry.yarnpkg.com/prelude-ls/-/prelude-ls-1.2.1.tgz#debc6489d7a6e6b0e7611888cec880337d316396" integrity sha512-vkcDPrRZo1QZLbn5RLGPpg/WmIQ65qoWWhcGKf/b5eplkkarX0m9z8ppCat4mlOqUsWpyNuYgO3VRyrYHSzX5g== +pretty-format@^29.1.2: + version "29.1.2" + resolved "https://registry.yarnpkg.com/pretty-format/-/pretty-format-29.1.2.tgz#b1f6b75be7d699be1a051f5da36e8ae9e76a8e6a" + integrity sha512-CGJ6VVGXVRP2o2Dorl4mAwwvDWT25luIsYhkyVQW32E4nL+TgW939J7LlKT/npq5Cpq6j3s+sy+13yk7xYpBmg== + dependencies: + "@jest/schemas" "^29.0.0" + ansi-styles "^5.0.0" + react-is "^18.0.0" + +pretty-format@^29.7.0: + version "29.7.0" + resolved "https://registry.yarnpkg.com/pretty-format/-/pretty-format-29.7.0.tgz#ca42c758310f365bfa71a0bda0a807160b776812" + integrity sha512-Pdlw/oPxN+aXdmM9R00JVC9WVFoCLTKJvDVLgmJ+qAffBMxsV85l/Lu7sNx4zSzPyoL2euImuEwHhOXdEgNFZQ== + dependencies: + "@jest/schemas" "^29.6.3" + ansi-styles "^5.0.0" + react-is "^18.0.0" + process-nextick-args@~1.0.6: version "1.0.7" resolved "https://registry.yarnpkg.com/process-nextick-args/-/process-nextick-args-1.0.7.tgz#150e20b756590ad3f91093f25a4f2ad8bff30ba3" @@ -4578,6 +6746,14 @@ promise-retry@^2.0.1: err-code "^2.0.2" retry "^0.12.0" +prompts@^2.0.1: + version "2.4.2" + resolved "https://registry.yarnpkg.com/prompts/-/prompts-2.4.2.tgz#7b57e73b3a48029ad10ebd44f74b01722a4cb069" + integrity sha512-NxNv/kLguCA7p3jE8oL2aEBsrJWgAakBpgmgK6lpPWV+WuOmY6r2/zbAVnP+T8bQlA0nzHXSJSJW0Hq7ylaD2Q== + dependencies: + kleur "^3.0.3" + sisteransi "^1.0.5" + proxy-addr@~2.0.7: version "2.0.7" resolved "https://registry.yarnpkg.com/proxy-addr/-/proxy-addr-2.0.7.tgz#f19fe69ceab311eeb94b42e70e8c2070f9ba1025" @@ -4664,6 +6840,11 @@ punycode@^2.1.0, punycode@^2.1.1: resolved "https://registry.yarnpkg.com/punycode/-/punycode-2.1.1.tgz#b58b010ac40c22c5657616c8d2c2c02c7bf479ec" integrity sha512-XRsRjdf+j5ml+y/6GKHPZbrF/8p2Yga0JPtdqTIY2Xe5ohJPD9saDJJLPvp9+NSBprVvevdXZybnj2cv8OEd0A== +pure-rand@^6.0.0: + version "6.0.4" + resolved "https://registry.yarnpkg.com/pure-rand/-/pure-rand-6.0.4.tgz#50b737f6a925468679bff00ad20eade53f37d5c7" + integrity sha512-LA0Y9kxMYv47GIPJy6MI84fqTd2HmYZI83W/kM/SkKfDlajnZYfmXFTxkbY+xSBPkLJxltMa9hIkmdc29eguMA== + qs@6.10.3: version "6.10.3" resolved "https://registry.yarnpkg.com/qs/-/qs-6.10.3.tgz#d6cde1b2ffca87b5aa57889816c5f81535e22e8e" @@ -4691,6 +6872,11 @@ queue-microtask@^1.2.2: resolved "https://registry.yarnpkg.com/queue-microtask/-/queue-microtask-1.2.3.tgz#4929228bbc724dfac43e0efb058caf7b6cfb6243" integrity sha512-NuaNSa6flKT5JaSYQzJok04JzTL1CA6aGhv5rfLW3PgqA+M2ChpZQnAC8h8i4ZFkBS8X5RqkDBHA7r4hej3K9A== +rambda@^7.4.0: + version "7.5.0" + resolved "https://registry.yarnpkg.com/rambda/-/rambda-7.5.0.tgz#1865044c59bc0b16f63026c6e5a97e4b1bbe98fe" + integrity sha512-y/M9weqWAH4iopRd7EHDEQQvpFPHj1AA3oHozE9tfITHUtTR7Z9PSlIRRG2l1GuW7sefC1cXFfIcF+cgnShdBA== + range-parser@~1.2.1: version "1.2.1" resolved "https://registry.yarnpkg.com/range-parser/-/range-parser-1.2.1.tgz#3cf37023d199e1c24d1a55b84800c2f3e6468031" @@ -4706,6 +6892,11 @@ raw-body@2.5.1: iconv-lite "0.4.24" unpipe "1.0.0" +react-is@^18.0.0: + version "18.2.0" + resolved "https://registry.yarnpkg.com/react-is/-/react-is-18.2.0.tgz#199431eeaaa2e09f86427efbb4f1473edb47609b" + integrity sha512-xWGDIW6x921xtzPkhiULtthJHoJvBbF3q26fzloPCK0hsvxtPVelvftw3zjbHWSkR2km9Z+4uxbDDK/6Zw9B8w== + read-pkg@^3.0.0: version "3.0.0" resolved "https://registry.yarnpkg.com/read-pkg/-/read-pkg-3.0.0.tgz#9cbc686978fee65d16c00e2b19c237fcf6e38389" @@ -4822,6 +7013,11 @@ require-at@^1.0.6: resolved "https://registry.yarnpkg.com/require-at/-/require-at-1.0.6.tgz#9eb7e3c5e00727f5a4744070a7f560d4de4f6e6a" integrity sha512-7i1auJbMUrXEAZCOQ0VNJgmcT2VOKPRl2YGJwgpHpC9CE91Mv4/4UYIUm4chGJaI381ZDq1JUicFii64Hapd8g== +require-directory@^2.1.1: + version "2.1.1" + resolved "https://registry.yarnpkg.com/require-directory/-/require-directory-2.1.1.tgz#8c64ad5fd30dab1c976e2344ffe7f792a6a6df42" + integrity sha512-fGxEI7+wsG9xrvdjsrlmL22OMTTiHRwAMroiEeMgq8gzoLC/PQr7RsRDSTLUg/bZAZtF+TVIkHc6/4RIKrui+Q== + require_optional@~1.0.0: version "1.0.1" resolved "https://registry.yarnpkg.com/require_optional/-/require_optional-1.0.1.tgz#4cf35a4247f64ca3df8c2ef208cc494b1ca8fc2e" @@ -4835,6 +7031,13 @@ requires-port@^1.0.0: resolved "https://registry.yarnpkg.com/requires-port/-/requires-port-1.0.0.tgz#925d2601d39ac485e091cf0da5c6e694dc3dcaff" integrity sha512-KigOCHcocU3XODJxsu8i/j8T9tzT4adHiecwORRQ0ZZFcp7ahwXuRU1m+yuO90C5ZUyGeGfocHDI14M3L3yDAQ== +resolve-cwd@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/resolve-cwd/-/resolve-cwd-3.0.0.tgz#0f0075f1bb2544766cf73ba6a6e2adfebcb13f2d" + integrity sha512-OrZaX2Mb+rJCpH/6CpSqt9xFVpN++x01XnN2ie9g6P5/3xelLAkXWVADpdz1IHD/KFfEXyE6V0U01OQ3UO2rEg== + dependencies: + resolve-from "^5.0.0" + resolve-from@^2.0.0: version "2.0.0" resolved "https://registry.yarnpkg.com/resolve-from/-/resolve-from-2.0.0.tgz#9480ab20e94ffa1d9e80a804c7ea147611966b57" @@ -4845,6 +7048,21 @@ resolve-from@^4.0.0: resolved "https://registry.yarnpkg.com/resolve-from/-/resolve-from-4.0.0.tgz#4abcd852ad32dd7baabfe9b40e00a36db5f392e6" integrity sha512-pb/MYmXstAkysRFx8piNI1tGFNQIFA3vkE3Gq4EuA1dF6gHp/+vgZqsCGJapvy8N3Q+4o7FwvquPJcnZ7RYy4g== +resolve-from@^5.0.0: + version "5.0.0" + resolved "https://registry.yarnpkg.com/resolve-from/-/resolve-from-5.0.0.tgz#c35225843df8f776df21c57557bc087e9dfdfc69" + integrity sha512-qYg9KP24dD5qka9J47d0aVky0N+b4fTU89LN9iDnjB5waksiC49rvMB0PrUJQGoTmH50XPiqOvAjDfaijGxYZw== + +resolve.exports@^1.1.0: + version "1.1.0" + resolved "https://registry.yarnpkg.com/resolve.exports/-/resolve.exports-1.1.0.tgz#5ce842b94b05146c0e03076985d1d0e7e48c90c9" + integrity sha512-J1l+Zxxp4XK3LUDZ9m60LRJF/mAe4z6a4xyabPHk7pvK5t35dACV32iIjJDFeWZFfZlO29w6SZ67knR0tHzJtQ== + +resolve.exports@^2.0.0: + version "2.0.2" + resolved "https://registry.yarnpkg.com/resolve.exports/-/resolve.exports-2.0.2.tgz#f8c934b8e6a13f539e38b7098e2e36134f01e800" + integrity sha512-X2UW6Nw3n/aMgDVy+0rSqgHlv39WZAlZrXCdnbyEiKm17DSqHX4MmQMaST3FbeWR5FTuRcUwYAziZajji0Y7mg== + resolve@^1.10.0, resolve@^1.20.0, resolve@^1.22.0: version "1.22.1" resolved "https://registry.yarnpkg.com/resolve/-/resolve-1.22.1.tgz#27cb2ebb53f91abb49470a928bba7558066ac177" @@ -4940,6 +7158,23 @@ sax@>=0.6.0, sax@^1.2.4: resolved "https://registry.yarnpkg.com/semver/-/semver-5.7.1.tgz#a954f931aeba508d307bbf069eff0c01c96116f7" integrity sha512-sauaDf/PZdVgrLTNYHRtpXa1iRiKcaebiKQ1BJdpQlWH2lCvexQdX55snPFyK7QzpudqbCI0qXFfOasHdyNDGQ== +semver@7.x, semver@^7.3.7: + version "7.3.8" + resolved "https://registry.yarnpkg.com/semver/-/semver-7.3.8.tgz#07a78feafb3f7b32347d725e33de7e2a2df67798" + integrity sha512-NB1ctGL5rlHrPJtFDVIVzTyQylMLu9N9VICA6HSFJo8MCGVTMW6gfpicwKmmK/dAjTOrqu5l63JJOpDSrAis3A== + dependencies: + lru-cache "^6.0.0" + +semver@^6.0.0, semver@^6.3.0: + version "6.3.0" + resolved "https://registry.yarnpkg.com/semver/-/semver-6.3.0.tgz#ee0a64c8af5e8ceea67687b133761e1becbd1d3d" + integrity sha512-b39TBaTSfV6yBrapU89p5fKekE2m/NwnDocOVruQFS1/veMgdzuPcnOM34M6CwxW8jH/lxEa5rBoDeUwu5HHTw== + +semver@^6.3.1: + version "6.3.1" + resolved "https://registry.yarnpkg.com/semver/-/semver-6.3.1.tgz#556d2ef8689146e46dcea4bfdd095f3434dffcb4" + integrity sha512-BR7VvDCVHO+q2xBEWskxS6DJE1qRnb7DxzUrogb71CWoSficBxYsiAGd+Kl0mmq/MprG9yArRkyrQxTO6XjMzA== + semver@^7.3.5: version "7.3.7" resolved "https://registry.yarnpkg.com/semver/-/semver-7.3.7.tgz#12c5b649afdbf9049707796e22a4028814ce523f" @@ -4947,6 +7182,13 @@ semver@^7.3.5: dependencies: lru-cache "^6.0.0" +semver@^7.5.3, semver@^7.5.4: + version "7.5.4" + resolved "https://registry.yarnpkg.com/semver/-/semver-7.5.4.tgz#483986ec4ed38e1c6c48c34894a9182dbff68a6e" + integrity sha512-1bCSESV6Pv+i21Hvpxp3Dx+pSD8lIPt8uVjRrxAUt/nbswYc+tK6Y2btiULjd4+fnq15PX+nqQDC7Oft7WkwcA== + dependencies: + lru-cache "^6.0.0" + semver@~5.1.0: version "5.1.1" resolved "https://registry.yarnpkg.com/semver/-/semver-5.1.1.tgz#a3292a373e6f3e0798da0b20641b9a9c5bc47e19" @@ -5029,7 +7271,7 @@ side-channel@^1.0.4: get-intrinsic "^1.0.2" object-inspect "^1.9.0" -signal-exit@^3.0.7: +signal-exit@^3.0.3, signal-exit@^3.0.7: version "3.0.7" resolved "https://registry.yarnpkg.com/signal-exit/-/signal-exit-3.0.7.tgz#a9a1767f8af84155114eaabd73f99273c8f59ad9" integrity sha512-wnD2ZE+l+SPC/uoS0vXeE9L1+0wuaMqKlfz9AMUo38JsyLSBWSFcHR1Rri62LZc12vLr1gb3jl7iwQhgwpAbGQ== @@ -5063,6 +7305,16 @@ sinon@^10.0.0: nise "^5.0.1" supports-color "^7.1.0" +sisteransi@^1.0.5: + version "1.0.5" + resolved "https://registry.yarnpkg.com/sisteransi/-/sisteransi-1.0.5.tgz#134d681297756437cc05ca01370d3a7a571075ed" + integrity sha512-bLGGlR1QxBcynn2d5YmDX4MGjlZvy2MRBDRNHLJ8VI6l6+9FUiyTFNJ0IveOSP0bcXgVDPRcfGqA0pjaqUpfVg== + +slash@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/slash/-/slash-3.0.0.tgz#6539be870c165adbd5240220dbe361f1bc4d4634" + integrity sha512-g9Q1haeby36OSStwb4ntCGGGaKsaVSjQ68fBxoQcutl5fS1vuY18H3wSt3jFyFtrkx+Kz0V1G85A4MyAdDMi2Q== + smart-buffer@^4.2.0: version "4.2.0" resolved "https://registry.yarnpkg.com/smart-buffer/-/smart-buffer-4.2.0.tgz#6e1d71fa4f18c05f7d0ff216dd16a481d0e8d9ae" @@ -5187,6 +7439,19 @@ sorted-array-functions@^1.3.0: resolved "https://registry.yarnpkg.com/sorted-array-functions/-/sorted-array-functions-1.3.0.tgz#8605695563294dffb2c9796d602bd8459f7a0dd5" integrity sha512-2sqgzeFlid6N4Z2fUQ1cvFmTOLRi/sEDzSQ0OKYchqgoPmQBVyM3959qYx3fpS6Esef80KjmpgPeEr028dP3OA== +source-map-support@0.5.13: + version "0.5.13" + resolved "https://registry.yarnpkg.com/source-map-support/-/source-map-support-0.5.13.tgz#31b24a9c2e73c2de85066c0feb7d44767ed52932" + integrity sha512-SHSKFHadjVA5oR4PPqhtAVdcBWwRYVd6g6cAXnIbRiIwc2EhPrTuKUBdSLvlEKyIP3GCf89fltvcZiP9MMFA1w== + dependencies: + buffer-from "^1.0.0" + source-map "^0.6.0" + +source-map@^0.6.0, source-map@^0.6.1: + version "0.6.1" + resolved "https://registry.yarnpkg.com/source-map/-/source-map-0.6.1.tgz#74722af32e9614e9c287a8d0bbde48b5e2f1a263" + integrity sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g== + sparse-bitfield@^3.0.3: version "3.0.3" resolved "https://registry.yarnpkg.com/sparse-bitfield/-/sparse-bitfield-3.0.3.tgz#ff4ae6e68656056ba4b3e792ab3334d38273ca11" @@ -5283,6 +7548,13 @@ stack-trace@0.0.x: resolved "https://registry.yarnpkg.com/stack-trace/-/stack-trace-0.0.10.tgz#547c70b347e8d32b4e108ea1a2a159e5fdde19c0" integrity sha512-KGzahc7puUKkzyMt+IqAep+TVNbKP+k2Lmwhub39m1AsTSkaDutx56aDCo+HLDzf/D26BIHTJWNiTG1KAJiQCg== +stack-utils@^2.0.3: + version "2.0.6" + resolved "https://registry.yarnpkg.com/stack-utils/-/stack-utils-2.0.6.tgz#aaf0748169c02fc33c8232abccf933f54a1cc34f" + integrity sha512-XlkWvfIm6RmsWtNJx+uqtKLS8eqFbxUg0ZzLXqY0caEy9l7hruX8IpiDnjsLavoBgqCCR71TqWO8MaXYheJ3RQ== + dependencies: + escape-string-regexp "^2.0.0" + standard-as-callback@^2.1.0: version "2.1.0" resolved "https://registry.yarnpkg.com/standard-as-callback/-/standard-as-callback-2.1.0.tgz#8953fc05359868a77b5b9739a665c5977bb7df45" @@ -5306,7 +7578,15 @@ stream-to-pull-stream@^1.7.1: looper "^3.0.0" pull-stream "^3.2.3" -"string-width@^1.0.2 || 2 || 3 || 4", string-width@^4.2.3: +string-length@^4.0.1: + version "4.0.2" + resolved "https://registry.yarnpkg.com/string-length/-/string-length-4.0.2.tgz#a8a8dc7bd5c1a82b9b3c8b87e125f66871b6e57a" + integrity sha512-+l6rNN5fYHNhZZy41RXsYptCjA2Igmq4EG7kZAYFQI1E1VTXarr6ZPXBg6eq7Y6eK4FEhY6AJlyuFIb/v/S0VQ== + dependencies: + char-regex "^1.0.2" + strip-ansi "^6.0.0" + +"string-width@^1.0.2 || 2 || 3 || 4", string-width@^4.1.0, string-width@^4.2.0, string-width@^4.2.3: version "4.2.3" resolved "https://registry.yarnpkg.com/string-width/-/string-width-4.2.3.tgz#269c7117d27b05ad2e536830a8ec895ef9c6d010" integrity sha512-wKyQRQpjJ0sIp62ErSZdGsjMJWsap5oRNihHhu6G7JVO/9jIB6UyevL+tXuOqrng8j/cxKTWyWUwvSTriiZz/g== @@ -5368,7 +7648,7 @@ string_decoder@~1.1.1: dependencies: safe-buffer "~5.1.0" -strip-ansi@^6.0.1: +strip-ansi@^6.0.0, strip-ansi@^6.0.1: version "6.0.1" resolved "https://registry.yarnpkg.com/strip-ansi/-/strip-ansi-6.0.1.tgz#9e26c63d30f53443e9489495b2105d37b67a85d9" integrity sha512-Y38VPSHcqkFrCpFnQ9vuSXmquuv5oXOKpGeT6aGrr3o3Gc9AlVa6JBfUSOCnbxGGZF+/0ooI7KrPuUSztUdU5A== @@ -5380,6 +7660,16 @@ strip-bom@^3.0.0: resolved "https://registry.yarnpkg.com/strip-bom/-/strip-bom-3.0.0.tgz#2334c18e9c759f7bdd56fdef7e9ae3d588e68ed3" integrity sha512-vavAMRXOgBVNF6nyEEmL3DBK19iRpDcoIwW+swQ+CbGiu7lju6t+JklA1MHweoWtadgt4ISVUsXLyDq34ddcwA== +strip-bom@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/strip-bom/-/strip-bom-4.0.0.tgz#9c3505c1db45bcedca3d9cf7a16f5c5aa3901878" + integrity sha512-3xurFv5tEgii33Zi8Jtp55wEIILR9eh34FAW00PZf+JnSsTmV/ioewSgQl97JHvgjoRGwPShsWm+IdrxB35d0w== + +strip-final-newline@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/strip-final-newline/-/strip-final-newline-2.0.0.tgz#89b852fb2fcbe936f6f4b3187afb0a12c1ab58ad" + integrity sha512-BrpvfNAE3dcvq7ll3xVumzjKjZQ5tI1sEUIKr3Uoks0XUl45St3FlatVqef9prk4jRDzhW6WZg+3bk93y6pLjA== + strip-json-comments@^3.1.0, strip-json-comments@^3.1.1: version "3.1.1" resolved "https://registry.yarnpkg.com/strip-json-comments/-/strip-json-comments-3.1.1.tgz#31f1281b3832630434831c310c01cccda8cbe006" @@ -5406,6 +7696,13 @@ supports-color@^7.1.0: dependencies: has-flag "^4.0.0" +supports-color@^8.0.0: + version "8.1.1" + resolved "https://registry.yarnpkg.com/supports-color/-/supports-color-8.1.1.tgz#cd6fc17e28500cff56c1b86c0a7fd4a54a73005c" + integrity sha512-MpUEN2OodtUzxvKQl72cUF7RQ5EiHsGvSsVG0ia9c5RbWGL2CI4C7EpPS8UTBIplnlzZiNuV56w+FuNxy3ty2Q== + dependencies: + has-flag "^4.0.0" + supports-preserve-symlinks-flag@^1.0.0: version "1.0.0" resolved "https://registry.yarnpkg.com/supports-preserve-symlinks-flag/-/supports-preserve-symlinks-flag-1.0.0.tgz#6eda4bd344a3c94aea376d4cc31bc77311039e09" @@ -5430,6 +7727,15 @@ tdigest@^0.1.1: dependencies: bintrees "1.0.2" +test-exclude@^6.0.0: + version "6.0.0" + resolved "https://registry.yarnpkg.com/test-exclude/-/test-exclude-6.0.0.tgz#04a8698661d805ea6fa293b6cb9e63ac044ef15e" + integrity sha512-cAGWPIyOHU6zlmg88jwm7VRyXnMN7iV68OGAbYDk/Mh/xC/pzVPlQtY6ngoIH/5/tciuhGfvESU8GrHrcxD56w== + dependencies: + "@istanbuljs/schema" "^0.1.2" + glob "^7.1.4" + minimatch "^3.0.4" + text-hex@1.0.x: version "1.0.0" resolved "https://registry.yarnpkg.com/text-hex/-/text-hex-1.0.0.tgz#69dc9c1b17446ee79a92bf5b884bb4b9127506f5" @@ -5450,11 +7756,28 @@ tiny-each-async@2.0.3: resolved "https://registry.yarnpkg.com/tiny-each-async/-/tiny-each-async-2.0.3.tgz#8ebbbfd6d6295f1370003fbb37162afe5a0a51d1" integrity sha512-5ROII7nElnAirvFn8g7H7MtpfV1daMcyfTGQwsn/x2VtyV+VPiO5CjReCJtWLvoKTDEDmZocf3cNPraiMnBXLA== +tmpl@1.0.5: + version "1.0.5" + resolved "https://registry.yarnpkg.com/tmpl/-/tmpl-1.0.5.tgz#8683e0b902bb9c20c4f726e3c0b69f36518c07cc" + integrity sha512-3f0uOEAQwIqGuWW2MVzYg8fV/QNnc/IpuJNG837rLuczAaLVHslWHZQj4IGiEl5Hs3kkbhwL9Ab7Hrsmuj+Smw== + to-array@0.1.4: version "0.1.4" resolved "https://registry.yarnpkg.com/to-array/-/to-array-0.1.4.tgz#17e6c11f73dd4f3d74cda7a4ff3238e9ad9bf890" integrity sha512-LhVdShQD/4Mk4zXNroIQZJC+Ap3zgLcDuwEdcmLv9CCO73NWockQDwyUnW/m8VX/EElfL6FcYx7EeutN4HJA6A== +to-fast-properties@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/to-fast-properties/-/to-fast-properties-2.0.0.tgz#dc5e698cbd079265bc73e0377681a4e4e83f616e" + integrity sha512-/OaKK0xYrs3DmxRYqL/yDc+FxFUVYhDlXMhRmv3z915w2HF1tnN1omB354j8VUGO/hbRzyD6Y3sA7v7GS/ceog== + +to-regex-range@^5.0.1: + version "5.0.1" + resolved "https://registry.yarnpkg.com/to-regex-range/-/to-regex-range-5.0.1.tgz#1648c44aae7c8d988a326018ed72f5b4dd0392e4" + integrity sha512-65P7iz6X5yEr1cwcgvQxbbIw7Uk3gOy5dIdtZ4rDveLqhrdJP+Li/Hx6tyK0NEb+2GCyneCMJiGqrADCSNk8sQ== + dependencies: + is-number "^7.0.0" + toidentifier@1.0.1: version "1.0.1" resolved "https://registry.yarnpkg.com/toidentifier/-/toidentifier-1.0.1.tgz#3be34321a88a820ed1bd80dfaa33e479fbb8dd35" @@ -5505,6 +7828,39 @@ triple-beam@^1.3.0: resolved "https://registry.yarnpkg.com/triple-beam/-/triple-beam-1.3.0.tgz#a595214c7298db8339eeeee083e4d10bd8cb8dd9" integrity sha512-XrHUvV5HpdLmIj4uVMxHggLbFSZYIn7HEWsqePZcI50pco+MPqJ50wMGY794X7AOOhxOBAjbkqfAbEe/QMp2Lw== +ts-jest@^29.0.3: + version "29.0.3" + resolved "https://registry.yarnpkg.com/ts-jest/-/ts-jest-29.0.3.tgz#63ea93c5401ab73595440733cefdba31fcf9cb77" + integrity sha512-Ibygvmuyq1qp/z3yTh9QTwVVAbFdDy/+4BtIQR2sp6baF2SJU/8CKK/hhnGIDY2L90Az2jIqTwZPnN2p+BweiQ== + dependencies: + bs-logger "0.x" + fast-json-stable-stringify "2.x" + jest-util "^29.0.0" + json5 "^2.2.1" + lodash.memoize "4.x" + make-error "1.x" + semver "7.x" + yargs-parser "^21.0.1" + +ts-node@^10.9.1: + version "10.9.1" + resolved "https://registry.yarnpkg.com/ts-node/-/ts-node-10.9.1.tgz#e73de9102958af9e1f0b168a6ff320e25adcff4b" + integrity sha512-NtVysVPkxxrwFGUUxGYhfux8k78pQB3JqYBXlLRZgdGUqTO5wU/UyHop5p70iEbGhB7q5KmiZiU0Y3KlJrScEw== + dependencies: + "@cspotcode/source-map-support" "^0.8.0" + "@tsconfig/node10" "^1.0.7" + "@tsconfig/node12" "^1.0.7" + "@tsconfig/node14" "^1.0.0" + "@tsconfig/node16" "^1.0.2" + acorn "^8.4.1" + acorn-walk "^8.1.1" + arg "^4.1.0" + create-require "^1.1.0" + diff "^4.0.1" + make-error "^1.1.1" + v8-compile-cache-lib "^3.0.1" + yn "3.1.1" + tsconfig-paths@^3.14.1: version "3.14.1" resolved "https://registry.yarnpkg.com/tsconfig-paths/-/tsconfig-paths-3.14.1.tgz#ba0734599e8ea36c862798e920bcf163277b137a" @@ -5515,11 +7871,23 @@ tsconfig-paths@^3.14.1: minimist "^1.2.6" strip-bom "^3.0.0" +tslib@^1.8.1: + version "1.14.1" + resolved "https://registry.yarnpkg.com/tslib/-/tslib-1.14.1.tgz#cf2d38bdc34a134bcaf1091c41f6619e2f672d00" + integrity sha512-Xni35NKzjgMrwevysHTCArtLDpPvye8zV/0E4EyYn43P7/7qvQwPh9BGkHewbMulVntbigmcT7rdX3BNo9wRJg== + tslib@^2.2.0: version "2.4.1" resolved "https://registry.yarnpkg.com/tslib/-/tslib-2.4.1.tgz#0d0bfbaac2880b91e22df0768e55be9753a5b17e" integrity sha512-tGyy4dAjRIEwI7BzsB0lynWgOpfqjUdq91XXAlIWD2OwKBH7oCl/GZG/HT4BOHrTlPMOASlMQ7veyTqpmRcrNA== +tsutils@^3.21.0: + version "3.21.0" + resolved "https://registry.yarnpkg.com/tsutils/-/tsutils-3.21.0.tgz#b48717d394cea6c1e096983eed58e9d61715b623" + integrity sha512-mHKK3iUXL+3UF6xL5k0PEhKRUBKPBCv/+RkEOpjRWxxx27KKRBmmA60A9pgOUvMi8GKhRMPEmjBRPzs2W7O1OA== + dependencies: + tslib "^1.8.1" + tunnel-agent@^0.6.0: version "0.6.0" resolved "https://registry.yarnpkg.com/tunnel-agent/-/tunnel-agent-0.6.0.tgz#27a5dea06b36b04a0a9966774b290868f0fc40fd" @@ -5554,6 +7922,11 @@ type-fest@^0.20.2: resolved "https://registry.yarnpkg.com/type-fest/-/type-fest-0.20.2.tgz#1bf207f4b28f91583666cb5fbd327887301cd5f4" integrity sha512-Ne+eE4r0/iWnpAxD852z3A+N0Bt5RN//NjJwRd2VFHEmrywxf5vsZlh4R6lixl6B+wz/8d+maTSAkN1FIkI3LQ== +type-fest@^0.21.3: + version "0.21.3" + resolved "https://registry.yarnpkg.com/type-fest/-/type-fest-0.21.3.tgz#d260a24b0198436e133fa26a524a6d65fa3b2e37" + integrity sha512-t0rzBq87m3fVcduHDUFhKmyyX+9eo6WQjZvf51Ea/M0Q7+T374Jp1aUiyUl0GKxp8M/OETVHSDvmkyPgvX+X2w== + type-is@~1.6.18: version "1.6.18" resolved "https://registry.yarnpkg.com/type-is/-/type-is-1.6.18.tgz#4e552cd05df09467dcbc4ef739de89f2cf37c131" @@ -5650,6 +8023,14 @@ unpipe@1.0.0, unpipe@~1.0.0: resolved "https://registry.yarnpkg.com/unpipe/-/unpipe-1.0.0.tgz#b2bf4ee8514aae6165b4817829d21b2ef49904ec" integrity sha512-pjy2bYhSsufwWlKwPc+l3cN7+wuJlK6uz0YdJEOlQDbl6jo/YlPi4mb8agUkVC8BF7V8NuzeyPNqRksA3hztKQ== +update-browserslist-db@^1.0.13: + version "1.0.13" + resolved "https://registry.yarnpkg.com/update-browserslist-db/-/update-browserslist-db-1.0.13.tgz#3c5e4f5c083661bd38ef64b6328c26ed6c8248c4" + integrity sha512-xebP81SNcPuNpPP3uzeW1NYXxI3rxyJzF3pD6sH4jE7o/IX+WtSpwnVU+qIsDPyk0d3hmFQ7mjqc6AtV604hbg== + dependencies: + escalade "^3.1.1" + picocolors "^1.0.0" + uri-js@^4.2.2: version "4.4.1" resolved "https://registry.yarnpkg.com/uri-js/-/uri-js-4.4.1.tgz#9b1a52595225859e55f669d928f88c6c57f2a77e" @@ -5762,6 +8143,20 @@ uuid@^8.3.0, uuid@^8.3.2: resolved "https://registry.yarnpkg.com/uuid/-/uuid-8.3.2.tgz#80d5b5ced271bb9af6c445f21a1a04c606cefbe2" integrity sha512-+NYs2QeMWy+GWFOEm9xnn6HCDp0l7QBD7ml8zLUmJ+93Q5NF0NocErnwkTkXVFNiX3/fpC6afS8Dhb/gz7R7eg== +v8-compile-cache-lib@^3.0.1: + version "3.0.1" + resolved "https://registry.yarnpkg.com/v8-compile-cache-lib/-/v8-compile-cache-lib-3.0.1.tgz#6336e8d71965cb3d35a1bbb7868445a7c05264bf" + integrity sha512-wa7YjyUGfNZngI/vtK0UHAN+lgDCxBPCylVXGp0zu59Fz5aiGtNXaq3DhIov063MorB+VfufLh3JlF2KdTK3xg== + +v8-to-istanbul@^9.0.1: + version "9.0.1" + resolved "https://registry.yarnpkg.com/v8-to-istanbul/-/v8-to-istanbul-9.0.1.tgz#b6f994b0b5d4ef255e17a0d17dc444a9f5132fa4" + integrity sha512-74Y4LqY74kLE6IFyIjPtkSTWzUZmj8tdHT9Ii/26dvQ6K9Dl2NbEfj0XgU2sHCtKgt5VupqhlO/5aWuqS+IY1w== + dependencies: + "@jridgewell/trace-mapping" "^0.3.12" + "@types/istanbul-lib-coverage" "^2.0.1" + convert-source-map "^1.6.0" + valid-url@~1.0.9: version "1.0.9" resolved "https://registry.yarnpkg.com/valid-url/-/valid-url-1.0.9.tgz#1c14479b40f1397a75782f115e4086447433a200" @@ -5824,6 +8219,13 @@ verror@1.10.0: core-util-is "1.0.2" extsprintf "^1.2.0" +walker@^1.0.8: + version "1.0.8" + resolved "https://registry.yarnpkg.com/walker/-/walker-1.0.8.tgz#bd498db477afe573dc04185f011d3ab8a8d7653f" + integrity sha512-ts/8E8l5b7kY0vlWLewOkDXMmPdLcVV4GmOQLyxuSswIJsweeFZtAsMF7k1Nszz+TYBQrlYRmzOnr398y1JemQ== + dependencies: + makeerror "1.0.12" + webidl-conversions@^3.0.0: version "3.0.1" resolved "https://registry.yarnpkg.com/webidl-conversions/-/webidl-conversions-3.0.1.tgz#24534275e2a7bc6be7bc86611cc16ae0a5654871" @@ -5961,11 +8363,28 @@ word-wrap@^1.2.3: resolved "https://registry.yarnpkg.com/word-wrap/-/word-wrap-1.2.3.tgz#610636f6b1f703891bd34771ccb17fb93b47079c" integrity sha512-Hz/mrNwitNRh/HUAtM/VT/5VH+ygD6DV7mYKZAtHOrbs8U7lvPS6xf7EJKMF0uW1KJCl0H701g3ZGus+muE5vQ== +wrap-ansi@^7.0.0: + version "7.0.0" + resolved "https://registry.yarnpkg.com/wrap-ansi/-/wrap-ansi-7.0.0.tgz#67e145cff510a6a6984bdf1152911d69d2eb9e43" + integrity sha512-YVGIj2kamLSTxw6NsZjoBxfSwsn0ycdesmc4p+Q21c5zPuZ1pl+NfxVdxPtdHvmNVOQ6XSYG4AUtyt/Fi7D16Q== + dependencies: + ansi-styles "^4.0.0" + string-width "^4.1.0" + strip-ansi "^6.0.0" + wrappy@1: version "1.0.2" resolved "https://registry.yarnpkg.com/wrappy/-/wrappy-1.0.2.tgz#b5243d8f3ec1aa35f1364605bc0d1036e30ab69f" integrity sha512-l4Sp/DRseor9wL6EvV2+TuQn63dMkPjZ/sp9XkghTEbV9KlPS1xUsZ3u7/IQO4wxtcFB4bgpQPRcR3QCvezPcQ== +write-file-atomic@^4.0.2: + version "4.0.2" + resolved "https://registry.yarnpkg.com/write-file-atomic/-/write-file-atomic-4.0.2.tgz#a9df01ae5b77858a027fd2e80768ee433555fcfd" + integrity sha512-7KxauUdBmSdWnmpaGFg+ppNjKF8uNLry8LyzjauQDOVONfFLNKrKvQOxZ/VuTIcS/gge/YNahf5RIIQWTSarlg== + dependencies: + imurmurhash "^0.1.4" + signal-exit "^3.0.7" + ws@^5.1.0: version "5.2.3" resolved "https://registry.yarnpkg.com/ws/-/ws-5.2.3.tgz#05541053414921bc29c63bee14b8b0dd50b07b3d" @@ -6036,21 +8455,54 @@ xtend@~3.0.0: resolved "https://registry.yarnpkg.com/xtend/-/xtend-3.0.0.tgz#5cce7407baf642cba7becda568111c493f59665a" integrity sha512-sp/sT9OALMjRW1fKDlPeuSZlDQpkqReA0pyJukniWbTGoEKefHxhGJynE3PNhUMlcM8qWIjPwecwCw4LArS5Eg== +y18n@^5.0.5: + version "5.0.8" + resolved "https://registry.yarnpkg.com/y18n/-/y18n-5.0.8.tgz#7f4934d0f7ca8c56f95314939ddcd2dd91ce1d55" + integrity sha512-0pfFzegeDWJHJIAmTLRP2DwHjdF5s7jo9tuztdQxAhINCdvS+3nGINqPd00AphqJR/0LhANUS6/+7SCb98YOfA== + yallist@^2.1.2: version "2.1.2" resolved "https://registry.yarnpkg.com/yallist/-/yallist-2.1.2.tgz#1c11f9218f076089a47dd512f93c6699a6a81d52" integrity sha512-ncTzHV7NvsQZkYe1DW7cbDLm0YpzHmZF5r/iyP3ZnQtMiJ+pjzisCiMNI+Sj+xQF5pXhSHxSB3uDbsBTzY/c2A== +yallist@^3.0.2: + version "3.1.1" + resolved "https://registry.yarnpkg.com/yallist/-/yallist-3.1.1.tgz#dbb7daf9bfd8bac9ab45ebf602b8cbad0d5d08fd" + integrity sha512-a4UGQaWPH59mOXUYnAG2ewncQS4i4F43Tv3JoAM+s2VDAmS9NsK8GpDMLrCHPksFT7h3K6TOoUNn2pb7RoXx4g== + yallist@^4.0.0: version "4.0.0" resolved "https://registry.yarnpkg.com/yallist/-/yallist-4.0.0.tgz#9bb92790d9c0effec63be73519e11a35019a3a72" integrity sha512-3wdGidZyq5PB084XLES5TpOSRA3wjXAlIWMhum2kRcv/41Sn2emQ0dycQW4uZXLejwKvg6EsvbdlVL+FYEct7A== +yargs-parser@^21.0.0, yargs-parser@^21.0.1: + version "21.1.1" + resolved "https://registry.yarnpkg.com/yargs-parser/-/yargs-parser-21.1.1.tgz#9096bceebf990d21bb31fa9516e0ede294a77d35" + integrity sha512-tVpsJW7DdjecAiFpbIB1e3qxIQsE6NoPc5/eTdrbbIC4h0LVsWhnoa3g+m2HclBIujHzsxZ4VJVA+GUuc2/LBw== + +yargs@^17.3.1: + version "17.6.0" + resolved "https://registry.yarnpkg.com/yargs/-/yargs-17.6.0.tgz#e134900fc1f218bc230192bdec06a0a5f973e46c" + integrity sha512-8H/wTDqlSwoSnScvV2N/JHfLWOKuh5MVla9hqLjK3nsfyy6Y4kDSYSvkU5YCUEPOSnRXfIyx3Sq+B/IWudTo4g== + dependencies: + cliui "^8.0.1" + escalade "^3.1.1" + get-caller-file "^2.0.5" + require-directory "^2.1.1" + string-width "^4.2.3" + y18n "^5.0.5" + yargs-parser "^21.0.0" + yeast@0.1.2: version "0.1.2" resolved "https://registry.yarnpkg.com/yeast/-/yeast-0.1.2.tgz#008e06d8094320c372dbc2f8ed76a0ca6c8ac419" integrity sha512-8HFIh676uyGYP6wP13R/j6OJ/1HwJ46snpvzE7aHAN3Ryqh2yX6Xox2B4CUmTwwOIzlG3Bs7ocsP5dZH/R1Qbg== +yn@3.1.1: + version "3.1.1" + resolved "https://registry.yarnpkg.com/yn/-/yn-3.1.1.tgz#1e87401a09d767c1d5eab26a6e4c185182d2eb50" + integrity sha512-Ux4ygGWsu2c7isFWe8Yu1YluJmqVhxqK2cLXNQA5AcC3QfbGNpM7fu0Y8b/z16pXLnFxZYvWhd3fhBY9DLmC6Q== + yocto-queue@^0.1.0: version "0.1.0" resolved "https://registry.yarnpkg.com/yocto-queue/-/yocto-queue-0.1.0.tgz#0294eb3dee05028d31ee1a5fa2c556a6aaf10a1b"