From f29bc1546dcacd89fc2d6421722cc1365421a659 Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 28 Nov 2023 11:25:22 +0530 Subject: [PATCH 01/26] Add topic configuration and fix config types --- examples/kafkajs/consumer.js | 7 +- examples/kafkajs/eos.js | 7 +- lib/kafkajs/_common.js | 47 +++--- lib/kafkajs/_consumer.js | 135 ++++++++--------- lib/kafkajs/_kafka.js | 57 ++++---- lib/kafkajs/_producer.js | 275 ++++++++++++++++++----------------- lib/kafkajs/index.js | 4 +- types/kafkajs.d.ts | 5 +- 8 files changed, 283 insertions(+), 254 deletions(-) diff --git a/examples/kafkajs/consumer.js b/examples/kafkajs/consumer.js index 22d5ccc7..cd8df5c7 100644 --- a/examples/kafkajs/consumer.js +++ b/examples/kafkajs/consumer.js @@ -1,4 +1,4 @@ -const { Kafka } = require('../..').KafkaJS +const { Kafka } = require('../..').KafkaJS; //const { Kafka } = require('kafkajs') async function consumerStart() { @@ -32,7 +32,12 @@ async function consumerStart() { } }, rdKafka: { + globalConfig: { 'enable.auto.commit': false + }, + topicConfig: { + 'auto.offset.reset': 'earliest' + }, } }); diff --git a/examples/kafkajs/eos.js b/examples/kafkajs/eos.js index 6c9f85ca..be06485e 100644 --- a/examples/kafkajs/eos.js +++ b/examples/kafkajs/eos.js @@ -15,7 +15,9 @@ async function eosStart() { const consumer = kafka.consumer({ groupId: 'groupId', rdKafka: { - "enable.auto.commit": false, + globalConfig: { + "enable.auto.commit": false, + } }, }); @@ -34,7 +36,8 @@ async function eosStart() { // The run method acts like a consume-transform-produce loop. consumer.run({ eachMessage: async ({ topic, partition, message }) => { - const msgAckString = JSON.stringify({topic, + const msgAckString = JSON.stringify({ + topic, partition, offset: message.offset, key: message.key?.toString(), diff --git a/lib/kafkajs/_common.js b/lib/kafkajs/_common.js index 4ffc3f4f..6d1c7571 100644 --- a/lib/kafkajs/_common.js +++ b/lib/kafkajs/_common.js @@ -1,38 +1,47 @@ +/** + * @function kafkaJSToRdKafkaConfig() + * @param {object} config + * @returns {{globalConfig: import("../../types/config").ConsumerGlobalConfig|import("../../types/config").ProducerTopicConfig, topicConfig: import("../../types/config").ConsumerTopicConfig|import("../../types/config").ProducerTopicConfig}} + */ async function kafkaJSToRdKafkaConfig(config) { - const ret = { - 'allow.auto.create.topics': 'false' - } - ret['bootstrap.servers'] = config['brokers'].join(','); + const globalConfig = { + "allow.auto.create.topics": "false", + }; + const topicConfig = {}; + globalConfig["bootstrap.servers"] = config["brokers"].join(","); let withSASL = false; if (config.sasl) { - const sasl = config.sasl; - if (sasl.mechanism === 'plain' && - typeof sasl.username === 'string' && - typeof sasl.password === 'string') { - ret['sasl.mechanism'] = 'PLAIN'; - ret['sasl.username'] = sasl.username; - ret['sasl.password'] = sasl.password; - withSASL = true; + const sasl = config.sasl; + if ( + sasl.mechanism === "plain" && + typeof sasl.username === "string" && + typeof sasl.password === "string" + ) { + globalConfig["sasl.mechanism"] = "PLAIN"; + globalConfig["sasl.username"] = sasl.username; + globalConfig["sasl.password"] = sasl.password; + withSASL = true; } } if (config.ssl === true && withSASL) { - ret['security.protocol'] = 'sasl_ssl'; + globalConfig["security.protocol"] = "sasl_ssl"; } else if (withSASL) { - ret['security.protocol'] = 'sasl_plaintext'; + globalConfig["security.protocol"] = "sasl_plaintext"; } if (config.rdKafka) { if (config.rdKafka.constructor === Function) { - await config.rdKafka(ret); + await config.rdKafka(globalConfig, topicConfig); } else { - Object.assign(ret, config.rdKafka); + Object.assign(globalConfig, config.rdKafka.globalConfig); + Object.assign(topicConfig, config.rdKafka.topicConfig); } } - return ret; + return { globalConfig, topicConfig }; } function topicPartitionOffsetToRdKafka(tpo) { @@ -40,7 +49,7 @@ function topicPartitionOffsetToRdKafka(tpo) { topic: tpo.topic, partition: tpo.partition, offset: Number(tpo.offset), - } + }; } -module.exports = { kafkaJSToRdKafkaConfig, topicPartitionOffsetToRdKafka } +module.exports = { kafkaJSToRdKafkaConfig, topicPartitionOffsetToRdKafka }; diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 9cfdba0c..776d9d4b 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -3,8 +3,8 @@ const RdKafka = require('../rdkafka'); const { kafkaJSToRdKafkaConfig, topicPartitionOffsetToRdKafka } = require('./_common'); const ConsumerState = Object.freeze({ - INIT: 0, - CONNECTING: 1, + INIT: 0, + CONNECTING: 1, CONNECTED: 2, DISCONNECTING: 3, DISCONNECTED: 4, @@ -17,38 +17,42 @@ class Consumer { #connectPromiseFunc = {}; #state = ConsumerState.INIT; + /** + * @constructor + * @param {import("../../types/kafkajs").ConsumerConfig} kJSConfig + */ constructor(kJSConfig) { this.#kJSConfig = kJSConfig; } - #config() { + async #config() { if (!this.#rdKafkaConfig) - this.#rdKafkaConfig = this.#finalizedConfig(); + this.#rdKafkaConfig = await this.#finalizedConfig(); return this.#rdKafkaConfig; } async #finalizedConfig() { - const config = await kafkaJSToRdKafkaConfig(this.#kJSConfig); + const { globalConfig, topicConfig } = await kafkaJSToRdKafkaConfig(this.#kJSConfig); if (this.#kJSConfig.groupId) { - config['group.id'] = this.#kJSConfig.groupId; + globalConfig['group.id'] = this.#kJSConfig.groupId; } - config['offset_commit_cb'] = true; + globalConfig['offset_commit_cb'] = true; if (this.#kJSConfig.rebalanceListener) { - config['rebalance_cb'] = (err, assignment) => { + globalConfig['rebalance_cb'] = (err, assignment) => { // Create the librdkafka error err = LibrdKafkaError.create(err); let call; - switch(err.code) { + switch (err.code) { case LibrdKafkaError.codes.ERR__ASSIGN_PARTITIONS: call = (this.#kJSConfig.rebalanceListener.onPartitionsAssigned ? - this.#kJSConfig.rebalanceListener.onPartitionsAssigned(assignment) : - Promise.resolve()).catch(console.error); + this.#kJSConfig.rebalanceListener.onPartitionsAssigned(assignment) : + Promise.resolve()).catch(console.error); break; case LibrdKafkaError.codes.ERR__REVOKE_PARTITIONS: call = (this.#kJSConfig.rebalanceListener.onPartitionsRevoked ? - this.#kJSConfig.rebalanceListener.onPartitionsRevoked(assignment) : - Promise.resolve()).catch(console.error); + this.#kJSConfig.rebalanceListener.onPartitionsRevoked(assignment) : + Promise.resolve()).catch(console.error); break; default: call = Promise.reject().catch(() => { @@ -58,46 +62,46 @@ class Consumer { } call - .finally(() => { - // Emit the event - this.#internalClient.emit('rebalance', err, assignment); - - try { - if (err.code === LibrdKafkaError.codes.ERR__ASSIGN_PARTITIONS) { - this.#internalClient.assign(assignment); - } else { - this.#internalClient.unassign(); + .finally(() => { + // Emit the event + this.#internalClient.emit('rebalance', err, assignment); + + try { + if (err.code === LibrdKafkaError.codes.ERR__ASSIGN_PARTITIONS) { + this.#internalClient.assign(assignment); + } else { + this.#internalClient.unassign(); + } + } catch (e) { + // Ignore exceptions if we are not connected + if (this.#internalClient.isConnected()) { + this.#internalClient.emit('rebalance.error', e); + } } - } catch (e) { - // Ignore exceptions if we are not connected - if (this.#internalClient.isConnected()) { - this.#internalClient.emit('rebalance.error', e); - } - } - }); + }); }; } - return config; + return { globalConfig, topicConfig }; } #readyCb(arg) { - if (this.#state !== ConsumerState.CONNECTING) { - // I really don't know how to handle this now. - return; - } - this.#state = ConsumerState.CONNECTED; + if (this.#state !== ConsumerState.CONNECTING) { + // I really don't know how to handle this now. + return; + } + this.#state = ConsumerState.CONNECTED; - // Resolve the promise. - this.#connectPromiseFunc['resolve'](); + // Resolve the promise. + this.#connectPromiseFunc['resolve'](); } #errorCb(args) { - console.log('error', args); - if (this.#state === ConsumerState.CONNECTING) { - this.#connectPromiseFunc['reject'](args); - } else { - // do nothing for now. - } + console.log('error', args); + if (this.#state === ConsumerState.CONNECTING) { + this.#connectPromiseFunc['reject'](args); + } else { + // do nothing for now. + } } #notImplemented() { @@ -111,7 +115,7 @@ class Consumer { } let timestamp = message.timestamp ? new Date(message.timestamp).toISOString() - : ''; + : ''; var headers = undefined; if (message.headers) { @@ -139,14 +143,14 @@ class Consumer { size: message.size, headers }, - heartbeat: async () => {}, - pause: () => {} + heartbeat: async () => { }, + pause: () => { } } } async #consumeSingle() { return new Promise((resolve, reject) => { - this.#internalClient.consume(1, function(err, messages) { + this.#internalClient.consume(1, function (err, messages) { if (err) { reject(`Consume error code ${err.code}`); return; @@ -168,7 +172,7 @@ class Consumer { }); else { for (let partition of topic.partitions) { - ret.push({topic: topic.topic, partition}); + ret.push({ topic: topic.topic, partition }); } } } @@ -180,22 +184,23 @@ class Consumer { } async connect() { - if (this.#state !== ConsumerState.INIT) { - return Promise.reject('Connect has already been called elsewhere.'); - } + if (this.#state !== ConsumerState.INIT) { + return Promise.reject('Connect has already been called elsewhere.'); + } - this.#state = ConsumerState.CONNECTING; - this.#internalClient = new RdKafka.KafkaConsumer(await this.#config()); - this.#internalClient.on('ready', this.#readyCb.bind(this)); - this.#internalClient.on('event.error', this.#errorCb.bind(this)); - this.#internalClient.on('event.log', console.log); - - return new Promise((resolve, reject) => { - this.#connectPromiseFunc = {resolve, reject}; - console.log('Connecting....'); - this.#internalClient.connect(); - console.log('connect() called'); - }); + this.#state = ConsumerState.CONNECTING; + const { globalConfig, topicConfig } = await this.#config(); + this.#internalClient = new RdKafka.KafkaConsumer(globalConfig, topicConfig); + this.#internalClient.on('ready', this.#readyCb.bind(this)); + this.#internalClient.on('event.error', this.#errorCb.bind(this)); + this.#internalClient.on('event.log', console.log); + + return new Promise((resolve, reject) => { + this.#connectPromiseFunc = { resolve, reject }; + console.log('Connecting....'); + this.#internalClient.connect(); + console.log('connect() called'); + }); } async subscribe(subscription) { @@ -208,7 +213,7 @@ class Consumer { async run(config) { if (this.#state !== ConsumerState.CONNECTED) { - throw new Error('Run must be called in state CONNECTED.'); + throw new Error('Run must be called in state CONNECTED.'); } while (this.#state === ConsumerState.CONNECTED) { @@ -240,7 +245,7 @@ class Consumer { seek(topicPartitionOffset) { return new Promise((resolve, reject) => { const rdKafkaTopicPartitionOffset = - topicPartitionOffsetToRdKafka(topicPartitionOffset); + topicPartitionOffsetToRdKafka(topicPartitionOffset); this.#internalClient.seek(rdKafkaTopicPartitionOffset, 0, (err) => { if (err) { reject(new Error(`Seek error code ${err.code}`)); diff --git a/lib/kafkajs/_kafka.js b/lib/kafkajs/_kafka.js index e8849138..86f937d4 100644 --- a/lib/kafkajs/_kafka.js +++ b/lib/kafkajs/_kafka.js @@ -1,36 +1,35 @@ -const { Producer } = require('./_producer'); -const { Consumer } = require('./_consumer'); - +const { Producer } = require("./_producer"); +const { Consumer } = require("./_consumer"); class Kafka { - #commonClientConfig = {}; - - constructor(config) { - this.#commonClientConfig = config; + #commonClientConfig = {}; + + constructor(config) { + this.#commonClientConfig = config; + } + + #mergeConfiguration(config) { + let baseConfig = Object.assign({}, this.#commonClientConfig); + config = Object.assign({}, config); + + let rdKafka = baseConfig.rdKafka; + Object.assign(baseConfig, config); + if (rdKafka && config.rdKafka) { + baseConfig.rdKafka = { + ...rdKafka, + ...config.rdKafka, + }; } + return baseConfig; + } - #mergeConfiguration(config) { - let baseConfig = Object.assign({}, this.#commonClientConfig); - config = Object.assign({}, config); - - let rdKafka = baseConfig.rdKafka; - Object.assign(baseConfig, config); - if (rdKafka && config.rdKafka) { - baseConfig.rdKafka = { - ...rdKafka, - ...config.rdKafka - } - } - return baseConfig; - } + producer(config) { + return new Producer(this.#mergeConfiguration(config)); + } - producer(config) { - return new Producer(this.#mergeConfiguration(config)); - } - - consumer(config) { - return new Consumer(this.#mergeConfiguration(config)); - } + consumer(config) { + return new Consumer(this.#mergeConfiguration(config)); + } } -module.exports = { Kafka } +module.exports = { Kafka }; diff --git a/lib/kafkajs/_producer.js b/lib/kafkajs/_producer.js index acd31932..52616bac 100644 --- a/lib/kafkajs/_producer.js +++ b/lib/kafkajs/_producer.js @@ -3,8 +3,8 @@ const { kafkaJSToRdKafkaConfig, topicPartitionOffsetToRdKafka } = require('./_co const { Consumer } = require('./_consumer'); const ProducerState = Object.freeze({ - INIT: 0, - CONNECTING: 1, + INIT: 0, + CONNECTING: 1, INITIALIZING_TRANSACTIONS: 2, INITIALIZED_TRANSACTIONS: 3, CONNECTED: 4, @@ -20,25 +20,29 @@ class Producer { #state = ProducerState.INIT; #ongoingTransaction = false; + /** + * @constructor + * @param {import("../../types/kafkajs").ProducerConfig} kJSConfig + */ constructor(kJSConfig) { this.#kJSConfig = kJSConfig; } - #config() { + async #config() { if (!this.#rdKafkaConfig) - this.#rdKafkaConfig = this.#finalizedConfig(); + this.#rdKafkaConfig = await this.#finalizedConfig(); return this.#rdKafkaConfig; } async #finalizedConfig() { - const config = await kafkaJSToRdKafkaConfig(this.#kJSConfig); - config.dr_cb = 'true'; + const { globalConfig, topicConfig } = await kafkaJSToRdKafkaConfig(this.#kJSConfig); + globalConfig.dr_cb = 'true'; if (this.#kJSConfig.hasOwnProperty('transactionalId')) { - config['transactional.id'] = this.#kJSConfig.transactionalId; + globalConfig['transactional.id'] = this.#kJSConfig.transactionalId; } - return config; + return { globalConfig, topicConfig }; } #flattenTopicPartitionOffsets(topics) { @@ -50,101 +54,102 @@ class Producer { } #readyTransactions(err) { - if (err) { - this.#connectPromiseFunc["reject"](err); - return; - } + if (err) { + this.#connectPromiseFunc["reject"](err); + return; + } - if (this.#state !== ProducerState.INITIALIZING_TRANSACTIONS) { - // FSM impossible state. We should add error handling for - // this later. - return; - } + if (this.#state !== ProducerState.INITIALIZING_TRANSACTIONS) { + // FSM impossible state. We should add error handling for + // this later. + return; + } - this.#state = ProducerState.INITIALIZED_TRANSACTIONS; - this.#readyCb(null); + this.#state = ProducerState.INITIALIZED_TRANSACTIONS; + this.#readyCb(null); } async #readyCb(arg) { - if (this.#state !== ProducerState.CONNECTING && this.#state !== ProducerState.INITIALIZED_TRANSACTIONS) { - // I really don't know how to handle this now. - return; - } + if (this.#state !== ProducerState.CONNECTING && this.#state !== ProducerState.INITIALIZED_TRANSACTIONS) { + // I really don't know how to handle this now. + return; + } - let config = await this.#config(); - if (config.hasOwnProperty('transactional.id') && this.#state !== ProducerState.INITIALIZED_TRANSACTIONS) { - this.#state = ProducerState.INITIALIZING_TRANSACTIONS; - this.#internalClient.initTransactions(5000 /* default: 5s */, this.#readyTransactions.bind(this)); - return; + let config = await this.#config(); + if (config.hasOwnProperty('transactional.id') && this.#state !== ProducerState.INITIALIZED_TRANSACTIONS) { + this.#state = ProducerState.INITIALIZING_TRANSACTIONS; + this.#internalClient.initTransactions(5000 /* default: 5s */, this.#readyTransactions.bind(this)); + return; + } + + this.#state = ProducerState.CONNECTED; + + // Start a loop to poll. + let pollInterval = setInterval(() => { + if (this.#state >= ProducerState.DISCONNECTING) { + clearInterval(pollInterval); + return; + } + this.#internalClient.poll(); + }, 500); + + this.#internalClient.on('delivery-report', function (err, report) { + //console.log('got delivery report', report, err); + const opaque = report.opaque; + if (!opaque) { + // not sure how to handle this. + return; + } + if (err) { + opaque.reject('err out'); + return; + } + //console.log('delivery-report: ' + JSON.stringify(report)); + delete report['opaque']; + + const recordMetadata = { + topicName: report.topic, + partition: report.partition, + errorCode: 0, + baseOffset: report.offset, + logAppendTime: null, + logStartOffset: null, } - this.#state = ProducerState.CONNECTED; - - // Start a loop to poll. - let pollInterval = setInterval(() => { - if (this.#state >= ProducerState.DISCONNECTING) { - clearInterval(pollInterval); - return; - } - this.#internalClient.poll(); - }, 500); - - this.#internalClient.on('delivery-report', function(err, report) { - //console.log('got delivery report', report, err); - const opaque = report.opaque; - if (!opaque) { - // not sure how to handle this. - return; - } - if (err) { - opaque.reject('err out'); - return; - } - //console.log('delivery-report: ' + JSON.stringify(report)); - delete report['opaque']; - - const recordMetadata = { - topicName: report.topic, - partition: report.partition, - errorCode: 0, - baseOffset: report.offset, - logAppendTime: null, - logStartOffset: null, - } - - opaque.resolve(recordMetadata); - }); + opaque.resolve(recordMetadata); + }); - // Resolve the promise. - this.#connectPromiseFunc["resolve"](); + // Resolve the promise. + this.#connectPromiseFunc["resolve"](); } #errorCb(args) { - console.log('error', args); - if (this.#state === ProducerState.CONNECTING) { - this.#connectPromiseFunc["reject"](args); - } else { - // do nothing for now. - } + console.log('error', args); + if (this.#state === ProducerState.CONNECTING) { + this.#connectPromiseFunc["reject"](args); + } else { + // do nothing for now. + } } async connect() { - if (this.#state !== ProducerState.INIT) { - return Promise.reject("Connect has already been called elsewhere."); - } + if (this.#state !== ProducerState.INIT) { + return Promise.reject("Connect has already been called elsewhere."); + } - this.#state = ProducerState.CONNECTING; - this.#internalClient = new RdKafka.Producer(await this.#config()); - this.#internalClient.on('ready', this.#readyCb.bind(this)); - this.#internalClient.on('event.error', this.#errorCb.bind(this)); - this.#internalClient.on('event.log', console.log); - - return new Promise((resolve, reject) => { - this.#connectPromiseFunc = {resolve, reject}; - console.log("Connecting...."); - this.#internalClient.connect(); - console.log("connect() called"); - }); + this.#state = ProducerState.CONNECTING; + const { globalConfig, topicConfig } = await this.#config(); + this.#internalClient = new RdKafka.Producer(globalConfig, topicConfig); + this.#internalClient.on('ready', this.#readyCb.bind(this)); + this.#internalClient.on('event.error', this.#errorCb.bind(this)); + this.#internalClient.on('event.log', console.log); + + return new Promise((resolve, reject) => { + this.#connectPromiseFunc = { resolve, reject }; + console.log("Connecting...."); + this.#internalClient.connect(); + console.log("connect() called"); + }); } async disconnect() { @@ -276,67 +281,67 @@ class Producer { } async send(sendOptions) { - if (this.#state !== ProducerState.CONNECTED) { - return Promise.reject("Cannot send message without awaiting connect()"); - } + if (this.#state !== ProducerState.CONNECTED) { + return Promise.reject("Cannot send message without awaiting connect()"); + } - if (sendOptions === null || !(sendOptions instanceof Object)) { - return Promise.reject("sendOptions must be set correctly"); - } + if (sendOptions === null || !(sendOptions instanceof Object)) { + return Promise.reject("sendOptions must be set correctly"); + } - // Ignore all properties except topic and messages. - // TODO: log a warning instead of ignoring. - if (!sendOptions.hasOwnProperty("topic") || !sendOptions.hasOwnProperty("messages") || !Array.isArray(sendOptions["messages"])) { - // TODO: add further validations. - return Promise.reject("sendOptions must be of the form {topic: string, messages: Message[]}"); - } + // Ignore all properties except topic and messages. + // TODO: log a warning instead of ignoring. + if (!sendOptions.hasOwnProperty("topic") || !sendOptions.hasOwnProperty("messages") || !Array.isArray(sendOptions["messages"])) { + // TODO: add further validations. + return Promise.reject("sendOptions must be of the form {topic: string, messages: Message[]}"); + } - const msgPromises = []; - for (let i = 0; i < sendOptions.messages.length; i++) { - const msg = sendOptions.messages[i]; + const msgPromises = []; + for (let i = 0; i < sendOptions.messages.length; i++) { + const msg = sendOptions.messages[i]; - if (!msg.hasOwnProperty("partition") || msg.partition === null) { - msg.partition = -1; - } + if (!msg.hasOwnProperty("partition") || msg.partition === null) { + msg.partition = -1; + } - if (typeof msg.value === 'string') { - msg.value = Buffer.from(msg.value); - } + if (typeof msg.value === 'string') { + msg.value = Buffer.from(msg.value); + } - msgPromises.push(new Promise((resolve, reject) => { - const opaque = {resolve, reject}; - this.#internalClient.produce(sendOptions.topic, msg.partition, msg.value, msg.key, msg.timestamp ?? Date.now(), opaque, msg.headers); - })); + msgPromises.push(new Promise((resolve, reject) => { + const opaque = { resolve, reject }; + this.#internalClient.produce(sendOptions.topic, msg.partition, msg.value, msg.key, msg.timestamp ?? Date.now(), opaque, msg.headers); + })); + } + const recordMetadataArr = await Promise.all(msgPromises); + + const topicPartitionRecordMetadata = new Map(); + for (const recordMetadata of recordMetadataArr) { + const key = `${recordMetadata.topicName},${recordMetadata.partition}`; + if (recordMetadata.baseOffset == null || !topicPartitionRecordMetadata.has(key)) { + topicPartitionRecordMetadata.set(key, recordMetadata); + continue; } - const recordMetadataArr = await Promise.all(msgPromises); - - const topicPartitionRecordMetadata = new Map(); - for (const recordMetadata of recordMetadataArr) { - const key = `${recordMetadata.topicName},${recordMetadata.partition}`; - if (recordMetadata.baseOffset == null || !topicPartitionRecordMetadata.has(key)) { - topicPartitionRecordMetadata.set(key, recordMetadata); - continue; - } - const currentRecordMetadata = topicPartitionRecordMetadata.get(key); + const currentRecordMetadata = topicPartitionRecordMetadata.get(key); - // Don't overwrite a null baseOffset - if (currentRecordMetadata.baseOffset == null) { - continue; - } - - if (currentRecordMetadata.baseOffset > recordMetadata.baseOffset) { - topicPartitionRecordMetadata.set(key, recordMetadata); - } + // Don't overwrite a null baseOffset + if (currentRecordMetadata.baseOffset == null) { + continue; } - const ret = []; - for (const [key, value] of topicPartitionRecordMetadata.entries()) { - value.baseOffset = value.baseOffset?.toString(); - ret.push(value); + if (currentRecordMetadata.baseOffset > recordMetadata.baseOffset) { + topicPartitionRecordMetadata.set(key, recordMetadata); } - return ret; + } + + const ret = []; + for (const [key, value] of topicPartitionRecordMetadata.entries()) { + value.baseOffset = value.baseOffset?.toString(); + ret.push(value); + } + return ret; } } diff --git a/lib/kafkajs/index.js b/lib/kafkajs/index.js index a41822d9..beac115f 100644 --- a/lib/kafkajs/index.js +++ b/lib/kafkajs/index.js @@ -1,3 +1,3 @@ -const { Kafka } = require('./_kafka'); +const { Kafka } = require("./_kafka"); -module.exports = { Kafka } +module.exports = { Kafka }; diff --git a/types/kafkajs.d.ts b/types/kafkajs.d.ts index 97e5e506..22e04c62 100644 --- a/types/kafkajs.d.ts +++ b/types/kafkajs.d.ts @@ -1,4 +1,5 @@ import * as tls from 'tls' +import { ConsumerGlobalConfig, ConsumerTopicConfig, ProducerGlobalConfig, ProducerTopicConfig } from './config' export type BrokersFunction = () => string[] | Promise @@ -37,6 +38,7 @@ export interface ProducerConfig { transactionalId?: string transactionTimeout?: number maxInFlightRequests?: number + rdKafka?: Function | { topicConfig?: ProducerTopicConfig, globalConfig?: ProducerGlobalConfig } } export interface IHeaders { @@ -124,6 +126,7 @@ export interface ConsumerConfig { maxInFlightRequests?: number readUncommitted?: boolean rackId?: string + rdKafka?: Function | { topicConfig?: ConsumerTopicConfig, globalConfig?: ConsumerGlobalConfig } } export type ConsumerEvents = { @@ -409,7 +412,7 @@ export type GroupDescription = { export type Consumer = { connect(): Promise disconnect(): Promise - subscribe(subscription: ConsumerSubscribeTopics ): Promise + subscribe(subscription: ConsumerSubscribeTopics): Promise stop(): Promise run(config?: ConsumerRunConfig): Promise commitOffsets(topicPartitions: Array): Promise From 1b20ce62560ab286e12133cee4c6c0dd977e17a7 Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 27 Nov 2023 17:19:04 +0530 Subject: [PATCH 02/26] Add errors for promisified API --- CONTRIBUTING.md | 2 +- LICENSE.txt | 1 + MIGRATION.md | 64 ++++++++++++++ lib/kafkajs/_common.js | 52 +++++++++++- lib/kafkajs/_error.js | 187 +++++++++++++++++++++++++++++++++++++++++ lib/kafkajs/_kafka.js | 7 +- 6 files changed, 308 insertions(+), 5 deletions(-) create mode 100644 MIGRATION.md create mode 100644 lib/kafkajs/_error.js diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index e2cabe1f..0d965dc5 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -195,7 +195,7 @@ You can add breakpoints and so on after that. ## Updating librdkafka version -The librdkafka should be periodically updated to the latest release in https://github.com/edenhill/librdkafka/releases +The librdkafka should be periodically updated to the latest release in https://github.com/confluentinc/librdkafka/releases Steps to update: 1. Update the `librdkafka` property in [`package.json`](https://github.com/confluentinc/confluent-kafka-js/blob/master/package.json) to the desired version. diff --git a/LICENSE.txt b/LICENSE.txt index d5ad6d41..ea9b1c3b 100644 --- a/LICENSE.txt +++ b/LICENSE.txt @@ -1,5 +1,6 @@ The MIT License (MIT) Copyright (c) 2016-2023 Blizzard Entertainment + 2023 Confluent, Inc. Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in diff --git a/MIGRATION.md b/MIGRATION.md new file mode 100644 index 00000000..34bc29eb --- /dev/null +++ b/MIGRATION.md @@ -0,0 +1,64 @@ +# Migration Guide + +## KafkaJS + +### Common + +1. Error Handling: Some possible subtypes of `KafkaJSError` have been removed, + and additional information has been added into `KafkaJSError`. + Internally, fields have been added denoting if the error is fatal, retriable, or abortable (the latter two only relevant for a + transactional producer). + Some error-specific fields have also been removed. An exhaustive list is at the bottom of this section. + + For compability, as many error types as possible have been retained, but it is + better to switch to checking the `error.code`. + + **Action**: Convert any checks based on `instanceof` and `error.name` or to error + checks based on `error.code` or `error.type`. + + **Example:**: + ```js + try { + await producer.send(/* args */); + } catch (error) { + if (!Kafka.isKafkaJSError(error)) { /* unrelated err handling */ } + else if (error.fatal) { /* fatal error, abandon producer */ } + else if (error.code === Kafka.ErrorCode.ERR__QUEUE_FULL) { /*...*/ } + else if (error.type === 'ERR_MSG_SIZE_TOO_LARGE') { /*...*/ } + /* and so on for specific errors */ + } + ``` + + Exhaustive list of error types and error fields removed: + 1. `KafkaJSNonRetriableError`: retriable errors are automatically retried by librdkafka, so there's no need for this type. + Note that `error.retriable` still exists, but it's applicable only for transactional producer, + where users are expected to retry an action themselves. + All error types using this as a superclass now use `KafkaJSError` as their superclass. + 2. `topic` and `partition` are removed from `KafkaJSOffsetOutOfRange`. + 3. `KafkaJSMemberIdRequired`: removed as automatically handled by librdkafka. + 4. `KafkaJSNumberOfRetriesExceeded`: removed as retries are handled by librdkafka. + 5. `broker, correlationId, createdAt, sentAt` and `pendingDuration` are removed from `KafkaJSNumberOfRetriesExceeded`. + 6. `KafkaJSMetadataNotLoaded`: removed as metadata is automatically reloaded by librdkafka. + 7. `KafkaJSTopicMetadataNotLoaded`: removed as topic metadata is automatically reloaded by librdkafka. + 8. `KafkaJSStaleTopicMetadataAssignment`: removed as it's automatically refreshed by librdkafka. + 9. `KafkaJSDeleteGroupsError`: removed, as the Admin Client doesn't have this yet. May be added back again, or changed. + 10. `KafkaJSServerDoesNotSupportApiKey`: removed, as this error isn't generally exposed to user in librdkafka. If raised, + it is subsumed into `KafkaJSError` where `error.code === Kafka.ErrorCode.ERR_UNSUPPORTED_VERSION`. + 11. `KafkaJSBrokerNotFound`: removed, as this error isn't exposed directly to the user in librdkafka. + 12. `KafkaJSLockTimeout`: removed, as such an error is not applicable while using librdkafka. + 13. `KafkaJSUnsupportedMagicByteInMessageSet`: removed. It is subsumed into `KafkaJSError` where `error.code === Kafka.ErrorCode.ERR_UNSUPPORTED_VERSION`. + 14. `KafkaJSDeleteTopicRecordsError`: removed, as the Admin Client doesn't have this yet. May be added back again, or changed. + 15. `KafkaJSInvariantViolation`: removed, as it's not applicable to librdkafka. Errors in internal state are subsumed into `KafkaJSError` where `error.code === Kafka.ErrorCode.ERR__STATE`. + 16. `KafkaJSInvalidVarIntError`: removed, as it's not exposed to the user in librdkafka. + 17. `KafkaJSInvalidLongError`: removed, as it's not exposed to the user in librdkafka. + 18. `KafkaJSCreateTopicError`: removed, as the Admin Client doesn't have this yet. May be added back again, or changed. + 19. `KafkaJSAlterPartitionReassignmentsError`: removed, as the RPC is not used in librdkafka. + 20. `KafkaJSFetcherRebalanceError`: removed, it's not exposed to the user in librdkafka. + 21. `broker` is removed from `KafkaJSConnectionError`. + 22. `KafkaJSConnectionClosedError`: removed, and subsumed into `KafkaJSConnectionError` as librdkafka treats them equivalently. + +### Producer + +### Consumer + +## node-rdkafka \ No newline at end of file diff --git a/lib/kafkajs/_common.js b/lib/kafkajs/_common.js index 6d1c7571..ea6d3d36 100644 --- a/lib/kafkajs/_common.js +++ b/lib/kafkajs/_common.js @@ -1,3 +1,6 @@ +const error = require("./_error"); +const LibrdKafkaError = require('../error'); + /** * @function kafkaJSToRdKafkaConfig() * @param {object} config @@ -52,4 +55,51 @@ function topicPartitionOffsetToRdKafka(tpo) { }; } -module.exports = { kafkaJSToRdKafkaConfig, topicPartitionOffsetToRdKafka }; +/** + * Convert a librdkafka error from node-rdkafka into a KafkaJSError. + * @param {LibrdKafkaError} librdKafkaError to convert from. + * @returns KafkaJSError + */ +function createKafkaJsErrorFromLibRdKafkaError(librdKafkaError) { + const properties = { + retriable: librdKafkaError.retriable, + fatal: librdKafkaError.fatal, + abortable: librdKafkaError.abortable, + stack: librdKafkaError.stack, + code: librdKafkaError.code, + }; + + let err = null; + + if (properties.code === error.ErrorCodes.ERR_OFFSET_OUT_OF_RANGE) { + err = new error.KafkaJSOffsetOutOfRange(e, properties); + } else if (properties.code === error.ErrorCodes.ERR_REQUEST_TIMED_OUT) { + err = new error.KafkaJSRequestTimeoutError(e, properties); + } else if (properties.code === error.ErrorCodes.ERR__PARTIAL) { + err = new error.KafkaJSPartialMessageError(e, properties); + } else if (properties.code === error.ErrorCodes.ERR__AUTHENTICATION) { + err = new error.KafkaJSSASLAuthenticationError(e, properties); + } else if (properties.code === error.ErrorCodes.ERR_GROUP_COORDINATOR_NOT_AVAILABLE) { + err = new error.KafkaJSGroupCoordinatorNotAvailableError(e, properties); + } else if (properties.code === error.ErrorCodes.ERR__NOT_IMPLEMENTED) { + err = new error.KafkaJSNotImplemented(e, properties); + } else if (properties.code === error.ErrorCodes.ERR__TIMED_OUT) { + err = new error.KafkaJSTimedOut(e, properties); + } else if (properties.code === error.ErrorCodes.ERR__ALL_BROKERS_DOWN) { + err = new error.KafkaJSNoBrokerAvailableError(e, properties); + } else if (properties.code === error.ErrorCodes.ERR__TRANSPORT) { + err = new error.KafkaJSConnectionError(e, properties); + } else if (properties.code > 0) { /* Indicates a non-local error */ + err = new error.KafkaJSProtocolError(e, properties); + } else { + err = new error.KafkaJSError(e, properties); + } + + return err; +} + +module.exports = { + kafkaJSToRdKafkaConfig, + topicPartitionOffsetToRdKafka, + createKafkaJsErrorFromLibRdKafkaError, +}; diff --git a/lib/kafkajs/_error.js b/lib/kafkajs/_error.js new file mode 100644 index 00000000..d182f0cb --- /dev/null +++ b/lib/kafkajs/_error.js @@ -0,0 +1,187 @@ +const LibrdKafkaError = require('../error'); + +/** + * @typedef {Object} KafkaJSError represents an error when using the promisified interface. + */ +class KafkaJSError extends Error { + /** + * @param {Error | string} error an Error or a string describing the error. + * @param {object} properties a set of optional error properties. + * @param {boolean} [properties.retriable=false] whether the error is retriable. Applies only to the transactional producer + * @param {boolean} [properties.fatal=false] whether the error is fatal. Applies only to the transactional producer. + * @param {boolean} [properties.abortable=false] whether the error is abortable. Applies only to the transactional producer. + * @param {string} [properties.stack] the stack trace of the error. + * @param {number} [properties.code=LibrdKafkaError.codes.ERR_UNKNOWN] the error code. + */ + constructor(e, { retriable = false, fatal = false, abortable = false, stack = null, code = LibrdKafkaError.codes.ERR_UNKNOWN } = {}) { + super(e, {}); + this.name = 'KafkaJSError'; + this.message = e.message || e; + this.retriable = retriable; + this.fatal = fatal; + this.abortable = abortable; + this.code = code; + + if (stack) { + this.stack = stack; + } else { + Error.captureStackTrace(this, this.constructor); + } + + const errTypes = Object + .keys(LibrdKafkaError.codes) + .filter(k => LibrdKafkaError.codes[k] === kjsErr.code); + + if (errTypes.length !== 1) { + this.type = LibrdKafkaError.codes.ERR_UNKNOWN; + } else { + this.type = errTypes[0]; + } + } +} + +/** + * @typedef {Object} KafkaJSProtocolError represents an error that is caused when a Kafka Protocol RPC has an embedded error. + */ +class KafkaJSProtocolError extends KafkaJSError { + constructor() { + super(...arguments); + this.name = 'KafkaJSProtocolError'; + } +} + +/** + * @typedef {Object} KafkaJSOffsetOutOfRange represents the error raised when fetching from an offset out of range. + */ +class KafkaJSOffsetOutOfRange extends KafkaJSProtocolError { + constructor() { + super(...arguments); + this.name = 'KafkaJSOffsetOutOfRange'; + } +} + +/** + * @typedef {Object} KafkaJSConnectionError represents the error raised when a connection to a broker cannot be established or is broken unexpectedly. + */ +class KafkaJSConnectionError extends KafkaJSError { + constructor() { + super(...arguments); + this.name = 'KafkaJSConnectionError'; + } +} + +/** + * @typedef {Object} KafkaJSRequestTimeoutError represents the error raised on a timeout for one request. + */ +class KafkaJSRequestTimeoutError extends KafkaJSError { + constructor() { + super(...arguments); + this.name = 'KafkaJSRequestTimeoutError'; + } +} + +/** + * @typedef {Object} KafkaJSPartialMessageError represents the error raised when a response does not contain all expected information. + */ +class KafkaJSPartialMessageError extends KafkaJSError { + constructor() { + super(...arguments); + this.name = 'KafkaJSPartialMessageError'; + } +} + +/** + * @typedef {Object} KafkaJSSASLAuthenticationError represents an error raised when authentication fails. + */ +class KafkaJSSASLAuthenticationError extends KafkaJSError { + constructor() { + super(...arguments); + this.name = 'KafkaJSSASLAuthenticationError'; + } +} + +/** + * @typedef {Object} KafkaJSGroupCoordinatorNotFound represents an error raised when the group coordinator is not found. + */ +class KafkaJSGroupCoordinatorNotFound extends KafkaJSError { + constructor() { + super(...arguments); + this.name = 'KafkaJSGroupCoordinatorNotFound'; + } +} + +/** + * @typedef {Object} KafkaJSNotImplemented represents an error raised when a feature is not implemented for this particular client. + */ +class KafkaJSNotImplemented extends KafkaJSError { + constructor() { + super(...arguments); + this.name = 'KafkaJSNotImplemented'; + } +} + +/** + * @typedef {Object} KafkaJSTimeout represents an error raised when a timeout for an operation occurs (including retries). + */ +class KafkaJSTimeout extends KafkaJSError { + constructor() { + super(...arguments); + this.name = 'KafkaJSTimeout'; + } +} + +/** + * @typedef {Object} KafkaJSAggregateError represents an error raised when multiple errors occur at once. + */ +class KafkaJSAggregateError extends Error { + constructor(message, errors) { + super(message); + this.errors = errors; + this.name = 'KafkaJSAggregateError'; + } +} + +/** + * @typedef {Object} KafkaJSNoBrokerAvailableError represents an error raised when no broker is available for the operation. + */ +class KafkaJSNoBrokerAvailableError extends KafkaJSError { + constructor() { + super(...arguments); + this.name = 'KafkaJSNoBrokerAvailableError'; + } +} + +/** + * @function isRebalancing + * @param {KafkaJSError} e + * @returns boolean representing whether the error is a rebalancing error. + */ +const isRebalancing = e => + e.type === 'REBALANCE_IN_PROGRESS' || + e.type === 'NOT_COORDINATOR_FOR_GROUP' || + e.type === 'ILLEGAL_GENERATION'; + +/** + * @function isKafkaJSError + * @param {any} e + * @returns boolean representing whether the error is a KafkaJSError. + */ +const isKafkaJSError = e => e instanceof KafkaJSError; + +module.exports = { + KafkaJSError, + KafkaJSPartialMessageError, + KafkaJSProtocolError, + KafkaJSConnectionError, + KafkaJSRequestTimeoutError, + KafkaJSSASLAuthenticationError, + KafkaJSOffsetOutOfRange, + KafkaJSGroupCoordinatorNotFound, + KafkaJSNotImplemented, + KafkaJSTimeout, + KafkaJSAggregateError, + KafkaJSNoBrokerAvailableError, + isRebalancing, + isKafkaJSError, + ErrorCodes: LibrdKafkaError.codes, +}; \ No newline at end of file diff --git a/lib/kafkajs/_kafka.js b/lib/kafkajs/_kafka.js index 86f937d4..5204a3fa 100644 --- a/lib/kafkajs/_kafka.js +++ b/lib/kafkajs/_kafka.js @@ -1,5 +1,6 @@ -const { Producer } = require("./_producer"); -const { Consumer } = require("./_consumer"); +const { Producer } = require('./_producer'); +const { Consumer } = require('./_consumer'); +const error = require('./_error'); class Kafka { #commonClientConfig = {}; @@ -32,4 +33,4 @@ class Kafka { } } -module.exports = { Kafka }; +module.exports = { Kafka, ...error }; From 446dee14af9afb2483dba6880011157197e7ce61 Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 29 Nov 2023 17:56:08 +0530 Subject: [PATCH 03/26] Convert delivery report error from string to code for richer error --- src/callbacks.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/callbacks.cc b/src/callbacks.cc index b5a1a7c7..b0d8cd62 100644 --- a/src/callbacks.cc +++ b/src/callbacks.cc @@ -291,7 +291,7 @@ void DeliveryReportDispatcher::Flush() { if (event.is_error) { // If it is an error we need the first argument to be set - argv[0] = Nan::Error(event.error_string.c_str()); + argv[0] = Nan::New(event.error_code); } else { argv[0] = Nan::Null(); } From 15755c1fa7a6f177b0f8cf3bee599f02b3505eac Mon Sep 17 00:00:00 2001 From: Milind L Date: Thu, 30 Nov 2023 12:50:40 +0530 Subject: [PATCH 04/26] Add error types to producer, and fix some errors --- MIGRATION.md | 50 ++++++++- lib/kafkajs/_common.js | 56 +++++++--- lib/kafkajs/_error.js | 4 +- lib/kafkajs/_kafka.js | 24 +++- lib/kafkajs/_producer.js | 236 ++++++++++++++++++++++++++++----------- types/kafkajs.d.ts | 3 +- 6 files changed, 285 insertions(+), 88 deletions(-) diff --git a/MIGRATION.md b/MIGRATION.md index 34bc29eb..5794f5ad 100644 --- a/MIGRATION.md +++ b/MIGRATION.md @@ -4,8 +4,10 @@ ### Common -1. Error Handling: Some possible subtypes of `KafkaJSError` have been removed, - and additional information has been added into `KafkaJSError`. +* Configuration changes + +* Error Handling: Some possible subtypes of `KafkaJSError` have been removed, + and additional information has been added into `KafkaJSError`. Internally, fields have been added denoting if the error is fatal, retriable, or abortable (the latter two only relevant for a transactional producer). Some error-specific fields have also been removed. An exhaustive list is at the bottom of this section. @@ -17,7 +19,7 @@ checks based on `error.code` or `error.type`. **Example:**: - ```js + ```javascript try { await producer.send(/* args */); } catch (error) { @@ -59,6 +61,46 @@ ### Producer +* `sendBatch` is currently unsupported - but will be supported. TODO. However, the actual batching semantics are handled by librdkafka. +* Changes to `send`: + 1. `acks`, `compression` and `timeout` are not set on a per-send basis. Rather, they must be configured in the configuration. + Before: + ```javascript + const kafka = new Kafka({/* ... */}); + const producer = kafka.producer(); + await producer.connect(); + + await producer.send({ + topic: 'test', + messages: [ /* ... */ ], + acks: 1, + compression: CompressionTypes.GZIP, + timeout: 30000, + }); + ``` + + After: + ```javascript + const kafka = new Kafka({/* ... */}); + const producer = kafka.producer({ + rdKafka: { + topicConfig: { + "acks": "1", + "compression.codec": "gzip", + "message.timeout.ms": "30000", + }, + } + }); + await producer.connect(); + + await producer.send({ + topic: 'test', + messages: [ /* ... */ ], + }); + ``` + + * Error-handling for a failed `send` is stricter. While sending multiple messages, even if one of the messages fails, the method throws an error. + ### Consumer -## node-rdkafka \ No newline at end of file +## node-rdkafka diff --git a/lib/kafkajs/_common.js b/lib/kafkajs/_common.js index ea6d3d36..5c8423a3 100644 --- a/lib/kafkajs/_common.js +++ b/lib/kafkajs/_common.js @@ -1,5 +1,4 @@ const error = require("./_error"); -const LibrdKafkaError = require('../error'); /** * @function kafkaJSToRdKafkaConfig() @@ -47,7 +46,13 @@ async function kafkaJSToRdKafkaConfig(config) { return { globalConfig, topicConfig }; } +/** + * Converts a topicPartitionOffset from KafkaJS to a format that can be used by node-rdkafka. + * @param {import("../../types/kafkajs").TopicPartitionOffset} tpo + * @returns {{topic: string, partition: number, offset: number}} + */ function topicPartitionOffsetToRdKafka(tpo) { + // TODO: do we need some checks for negative offsets and stuff? Or 'named' offsets? return { topic: tpo.topic, partition: tpo.partition, @@ -57,8 +62,8 @@ function topicPartitionOffsetToRdKafka(tpo) { /** * Convert a librdkafka error from node-rdkafka into a KafkaJSError. - * @param {LibrdKafkaError} librdKafkaError to convert from. - * @returns KafkaJSError + * @param {import("../error")} librdKafkaError to convert from. + * @returns {error.KafkaJSError} the converted error. */ function createKafkaJsErrorFromLibRdKafkaError(librdKafkaError) { const properties = { @@ -72,34 +77,57 @@ function createKafkaJsErrorFromLibRdKafkaError(librdKafkaError) { let err = null; if (properties.code === error.ErrorCodes.ERR_OFFSET_OUT_OF_RANGE) { - err = new error.KafkaJSOffsetOutOfRange(e, properties); + err = new error.KafkaJSOffsetOutOfRange(librdKafkaError, properties); } else if (properties.code === error.ErrorCodes.ERR_REQUEST_TIMED_OUT) { - err = new error.KafkaJSRequestTimeoutError(e, properties); + err = new error.KafkaJSRequestTimeoutError(librdKafkaError, properties); } else if (properties.code === error.ErrorCodes.ERR__PARTIAL) { - err = new error.KafkaJSPartialMessageError(e, properties); + err = new error.KafkaJSPartialMessageError(librdKafkaError, properties); } else if (properties.code === error.ErrorCodes.ERR__AUTHENTICATION) { - err = new error.KafkaJSSASLAuthenticationError(e, properties); + err = new error.KafkaJSSASLAuthenticationError(librdKafkaError, properties); } else if (properties.code === error.ErrorCodes.ERR_GROUP_COORDINATOR_NOT_AVAILABLE) { - err = new error.KafkaJSGroupCoordinatorNotAvailableError(e, properties); + err = new error.KafkaJSGroupCoordinatorNotAvailableError(librdKafkaError, properties); } else if (properties.code === error.ErrorCodes.ERR__NOT_IMPLEMENTED) { - err = new error.KafkaJSNotImplemented(e, properties); + err = new error.KafkaJSNotImplemented(librdKafkaError, properties); } else if (properties.code === error.ErrorCodes.ERR__TIMED_OUT) { - err = new error.KafkaJSTimedOut(e, properties); + err = new error.KafkaJSTimedOut(librdKafkaError, properties); } else if (properties.code === error.ErrorCodes.ERR__ALL_BROKERS_DOWN) { - err = new error.KafkaJSNoBrokerAvailableError(e, properties); + err = new error.KafkaJSNoBrokerAvailableError(librdKafkaError, properties); } else if (properties.code === error.ErrorCodes.ERR__TRANSPORT) { - err = new error.KafkaJSConnectionError(e, properties); + err = new error.KafkaJSConnectionError(librdKafkaError, properties); } else if (properties.code > 0) { /* Indicates a non-local error */ - err = new error.KafkaJSProtocolError(e, properties); + err = new error.KafkaJSProtocolError(librdKafkaError, properties); } else { - err = new error.KafkaJSError(e, properties); + err = new error.KafkaJSError(librdKafkaError, properties); } + console.log("Converted err = " + JSON.stringify(err, null, 2) + " librdkafka erro = " + JSON.stringify(librdKafkaError, null, 2)); return err; } +/** + * Converts KafkaJS headers to a format that can be used by node-rdkafka. + * @param {import("../../types/kafkajs").IHeaders|null} kafkaJSHeaders + * @returns {import("../../").MessageHeader[]|null} the converted headers. + */ +function convertToRdKafkaHeaders(kafkaJSHeaders) { + if (!kafkaJSHeaders) return null; + + const headers = []; + for (const [key, value] of Object.entries(kafkaJSHeaders)) { + if (value.constructor === Array) { + for (const v of value) { + headers.push({ key, value: v }); + } + } else { + headers.push({ key, value }); + } + } + return headers; +} + module.exports = { kafkaJSToRdKafkaConfig, topicPartitionOffsetToRdKafka, createKafkaJsErrorFromLibRdKafkaError, + convertToRdKafkaHeaders, }; diff --git a/lib/kafkajs/_error.js b/lib/kafkajs/_error.js index d182f0cb..fc1885af 100644 --- a/lib/kafkajs/_error.js +++ b/lib/kafkajs/_error.js @@ -30,7 +30,7 @@ class KafkaJSError extends Error { const errTypes = Object .keys(LibrdKafkaError.codes) - .filter(k => LibrdKafkaError.codes[k] === kjsErr.code); + .filter(k => LibrdKafkaError.codes[k] === this.code); if (errTypes.length !== 1) { this.type = LibrdKafkaError.codes.ERR_UNKNOWN; @@ -184,4 +184,4 @@ module.exports = { isRebalancing, isKafkaJSError, ErrorCodes: LibrdKafkaError.codes, -}; \ No newline at end of file +}; diff --git a/lib/kafkajs/_kafka.js b/lib/kafkajs/_kafka.js index 5204a3fa..58e888bd 100644 --- a/lib/kafkajs/_kafka.js +++ b/lib/kafkajs/_kafka.js @@ -5,17 +5,29 @@ const error = require('./_error'); class Kafka { #commonClientConfig = {}; + /** + * + * @param {import("../../types/kafkajs").KafkaConfig} config + */ constructor(config) { this.#commonClientConfig = config; } + /** + * Merge the producer/consumer specific configuration with the common configuration. + * @param {import("../../types/kafkajs").ProducerConfig|import("../../types/kafkajs").ConsumerConfig} config + * @returns + */ #mergeConfiguration(config) { let baseConfig = Object.assign({}, this.#commonClientConfig); config = Object.assign({}, config); + // TODO: there's some confusion around this, as we currently allow + // rdKafka to be a function, but here, we don't seem to treat it as such. + // Correct this, so that only objects are allowed for `rdKafka`. let rdKafka = baseConfig.rdKafka; Object.assign(baseConfig, config); - if (rdKafka && config.rdKafka) { + if (typeof rdKafka === 'object' && typeof config.rdKafka === 'object') { baseConfig.rdKafka = { ...rdKafka, ...config.rdKafka, @@ -24,10 +36,20 @@ class Kafka { return baseConfig; } + /** + * Creates a new producer. + * @param {import("../../types/kafkajs").ProducerConfig} config + * @returns {Producer} + */ producer(config) { return new Producer(this.#mergeConfiguration(config)); } + /** + * Creates a new consumer. + * @param {import("../../types/kafkajs").Consumer} config + * @returns {Consumer} + */ consumer(config) { return new Consumer(this.#mergeConfiguration(config)); } diff --git a/lib/kafkajs/_producer.js b/lib/kafkajs/_producer.js index 52616bac..a6b68588 100644 --- a/lib/kafkajs/_producer.js +++ b/lib/kafkajs/_producer.js @@ -1,6 +1,8 @@ const RdKafka = require('../rdkafka'); -const { kafkaJSToRdKafkaConfig, topicPartitionOffsetToRdKafka } = require('./_common'); +const { kafkaJSToRdKafkaConfig, topicPartitionOffsetToRdKafka, createKafkaJsErrorFromLibRdKafkaError, convertToRdKafkaHeaders } = require('./_common'); const { Consumer } = require('./_consumer'); +const error = require('./_error'); +const { Buffer } = require('buffer'); const ProducerState = Object.freeze({ INIT: 0, @@ -13,11 +15,40 @@ const ProducerState = Object.freeze({ }); class Producer { - #kJSConfig = null + /** + * kJSConfig is the kafkaJS config object. + * @type {import("../../types/kafkajs").ProducerConfig|null} + */ + #kJSConfig = null; + + /** + * rdKafkaConfig contains the config objects that will be passed to node-rdkafka. + * @type {{globalConfig: import("../../types/config").ProducerGlobalConfig, topicConfig: import("../../types/config").ProducerTopicConfig}|null} + */ #rdKafkaConfig = null; + + /** + * internalClient is the node-rdkafka client used by the API. + * @type {import("../rdkafka").Producer|null} + */ #internalClient = null; + + /** + * connectPromiseFunc is the set of promise functions used to resolve/reject the connect() promise. + * @type {{resolve: Function, reject: Function}|{}} + */ #connectPromiseFunc = {}; + + /** + * state is the current state of the producer. + * @type {ProducerState} + */ #state = ProducerState.INIT; + + /** + * ongoingTransaction is true if there is an ongoing transaction. + * @type {boolean} + */ #ongoingTransaction = false; /** @@ -38,17 +69,22 @@ class Producer { const { globalConfig, topicConfig } = await kafkaJSToRdKafkaConfig(this.#kJSConfig); globalConfig.dr_cb = 'true'; - if (this.#kJSConfig.hasOwnProperty('transactionalId')) { + if (Object.hasOwn(this.#kJSConfig, 'transactionalId')) { globalConfig['transactional.id'] = this.#kJSConfig.transactionalId; } return { globalConfig, topicConfig }; } + /** + * Flattens a list of topics with partitions into a list of topic, partition, offset. + * @param {import("../../types/kafkajs").TopicOffsets[]} topics + * @returns {import("../../types/kafkajs").TopicPartitionOffset} + */ #flattenTopicPartitionOffsets(topics) { return topics.flatMap(topic => { return topic.partitions.map(partition => { - return { partition: partition.partition, offset: partition.offset, topic: topic.topic }; + return { partition: Number(partition.partition), offset: String(partition.offset), topic: String(topic.topic) }; }) }) } @@ -69,14 +105,45 @@ class Producer { this.#readyCb(null); } - async #readyCb(arg) { - if (this.#state !== ProducerState.CONNECTING && this.#state !== ProducerState.INITIALIZED_TRANSACTIONS) { - // I really don't know how to handle this now. + /** + * Processes a delivery report, converting it to the type that the promisified API uses. + * @param {import('../..').LibrdKafkaError} err + * @param {import('../..').DeliveryReport} report + */ + #deliveryCallback(err, report) { + const opaque = report.opaque; + if (!opaque || (typeof opaque.resolve !== 'function' && typeof opaque.reject !== 'function')) { + // not sure how to handle this. + throw new error.KafkaJSError("Internal error: deliveryCallback called without opaque set properly", { code: error.ErrorCodes.ERR__STATE }); + } + + if (err) { + opaque.reject(createKafkaJsErrorFromLibRdKafkaError(err)); return; } - let config = await this.#config(); - if (config.hasOwnProperty('transactional.id') && this.#state !== ProducerState.INITIALIZED_TRANSACTIONS) { + delete report['opaque']; + + const recordMetadata = { + topicName: report.topic, + partition: report.partition, + errorCode: 0, + baseOffset: report.offset, + logAppendTime: '-1', + logStartOffset: '0', + }; + + opaque.resolve(recordMetadata); + } + + async #readyCb() { + if (this.#state !== ProducerState.CONNECTING && this.#state !== ProducerState.INITIALIZED_TRANSACTIONS) { + /* The connectPromiseFunc might not be set, so we throw such an error. It's a state error that we can't recover from. Probably a bug. */ + throw new error.KafkaJSError(`Ready callback called in invalid state ${this.#state}`, { code: error.ErrorCodes.ERR__STATE }); + } + + const config = await this.#config(); + if (Object.hasOwn(config, 'transactional.id') && this.#state !== ProducerState.INITIALIZED_TRANSACTIONS) { this.#state = ProducerState.INITIALIZING_TRANSACTIONS; this.#internalClient.initTransactions(5000 /* default: 5s */, this.#readyTransactions.bind(this)); return; @@ -84,8 +151,8 @@ class Producer { this.#state = ProducerState.CONNECTED; - // Start a loop to poll. - let pollInterval = setInterval(() => { + /* Start a loop to poll. the queues. */ + const pollInterval = setInterval(() => { if (this.#state >= ProducerState.DISCONNECTING) { clearInterval(pollInterval); return; @@ -93,51 +160,39 @@ class Producer { this.#internalClient.poll(); }, 500); - this.#internalClient.on('delivery-report', function (err, report) { - //console.log('got delivery report', report, err); - const opaque = report.opaque; - if (!opaque) { - // not sure how to handle this. - return; - } - if (err) { - opaque.reject('err out'); - return; - } - //console.log('delivery-report: ' + JSON.stringify(report)); - delete report['opaque']; - - const recordMetadata = { - topicName: report.topic, - partition: report.partition, - errorCode: 0, - baseOffset: report.offset, - logAppendTime: null, - logStartOffset: null, - } - - opaque.resolve(recordMetadata); - }); + this.#internalClient.on('delivery-report', this.#deliveryCallback.bind(this)); // Resolve the promise. this.#connectPromiseFunc["resolve"](); } - #errorCb(args) { - console.log('error', args); + /** + * Callback for the event.error event, either fails the initial connect(), or logs the error. + * @param {Error} err + */ + #errorCb(err) { if (this.#state === ProducerState.CONNECTING) { - this.#connectPromiseFunc["reject"](args); + this.#connectPromiseFunc["reject"](err); } else { - // do nothing for now. + /* TODO: we should log the error returned here, depending on the log level. + * Right now, we're just using console.err, but we should allow for a custom + * logger, or at least make a function in _common.js that handles consumer + * and producer. */ + console.error(err); } } + /** + * Set up the client and connect to the bootstrap brokers. + * @returns {Promise} Resolves when connection is complete, rejects on error. + */ async connect() { if (this.#state !== ProducerState.INIT) { - return Promise.reject("Connect has already been called elsewhere."); + throw new error.KafkaJSError("Connect has already been called elsewhere.", { code: error.ErrorCodes.ERR__STATE }); } this.#state = ProducerState.CONNECTING; + const { globalConfig, topicConfig } = await this.#config(); this.#internalClient = new RdKafka.Producer(globalConfig, topicConfig); this.#internalClient.on('ready', this.#readyCb.bind(this)); @@ -146,39 +201,50 @@ class Producer { return new Promise((resolve, reject) => { this.#connectPromiseFunc = { resolve, reject }; - console.log("Connecting...."); this.#internalClient.connect(); - console.log("connect() called"); }); } + /** + * Disconnect from the brokers, clean-up and tear down the client. + * @returns {Promise} Resolves when disconnect is complete, rejects on error. + */ async disconnect() { if (this.#state >= ProducerState.DISCONNECTING) { return; } + this.#state = ProducerState.DISCONNECTING; await new Promise((resolve, reject) => { const cb = (err) => { - err ? reject(err) : resolve(); + if (err) { + reject(createKafkaJsErrorFromLibRdKafkaError(err)); + return; + } this.#state = ProducerState.DISCONNECTED; + resolve(); } - this.#internalClient.disconnect(5000, cb); + this.#internalClient.disconnect(5000 /* default timeout, 5000ms */, cb); }); } + /** + * Start a transaction - can only be used with a transactional producer. + * @returns {Promise} Resolves with the producer when the transaction is started. + */ async transaction() { if (this.#state !== ProducerState.CONNECTED) { - return Promise.reject("Cannot start transaction without awaiting connect()"); + throw new error.KafkaJSError("Cannot start transaction without awaiting connect()", { code: error.ErrorCodes.ERR__STATE }); } if (this.#ongoingTransaction) { - return Promise.reject("Can only start one transaction at a time."); + throw new error.KafkaJSError("Can only start one transaction at a time.", { code: error.ErrorCodes.ERR__STATE }); } return new Promise((resolve, reject) => { this.#internalClient.beginTransaction((err) => { if (err) { - reject(err); + reject(createKafkaJsErrorFromLibRdKafkaError(err)); return; } this.#ongoingTransaction = true; @@ -191,20 +257,24 @@ class Producer { }); } + /** + * Commit the current transaction. + * @returns {Promise} Resolves when the transaction is committed. + */ async commit() { if (this.#state !== ProducerState.CONNECTED) { - return Promise.reject("Cannot commit without awaiting connect()"); + throw new error.KafkaJSError("Cannot commit without awaiting connect()", { code: error.ErrorCodes.ERR__STATE }); } if (!this.#ongoingTransaction) { - return Promise.reject("Cannot commit, no transaction ongoing."); + throw new error.KafkaJSError("Cannot commit, no transaction ongoing.", { code: error.ErrorCodes.ERR__STATE }); } return new Promise((resolve, reject) => { this.#internalClient.commitTransaction(5000 /* default: 5000ms */, err => { if (err) { // TODO: Do we reset ongoingTransaction here? - reject(err); + reject(createKafkaJsErrorFromLibRdKafkaError(err)); return; } this.#ongoingTransaction = false; @@ -213,21 +283,24 @@ class Producer { }); } - + /** + * Abort the current transaction. + * @returns {Promise} Resolves when the transaction is aborted. + */ async abort() { if (this.#state !== ProducerState.CONNECTED) { - return Promise.reject("Cannot abort without awaiting connect()"); + throw new error.KafkaJSError("Cannot abort without awaiting connect()", { code: error.ErrorCodes.ERR__STATE }); } if (!this.#ongoingTransaction) { - return Promise.reject("Cannot abort, no transaction ongoing."); + throw new error.KafkaJSError("Cannot abort, no transaction ongoing.", { code: error.ErrorCodes.ERR__STATE }); } return new Promise((resolve, reject) => { this.#internalClient.abortTransaction(5000 /* default: 5000ms */, err => { if (err) { // TODO: Do we reset ongoingTransaction here? - reject(err); + reject(createKafkaJsErrorFromLibRdKafkaError(err)); return; } this.#ongoingTransaction = false; @@ -236,19 +309,29 @@ class Producer { }); } + /** + * Send offsets for the transaction. + * @param {object} arg - The arguments to sendOffsets + * @param {string} arg.consumerGroupId - The consumer group id to send offsets for. + * @param {Consumer} arg.consumer - The consumer to send offsets for. + * @param {import("../../types/kafkajs").TopicOffsets[]} arg.topics - The topics, partitions and the offsets to send. + * + * @note only one of consumerGroupId or consumer must be set. It is recommended to use `consumer`. + * @returns {Promise} Resolves when the offsets are sent. + */ async sendOffsets(arg) { let { consumerGroupId, topics, consumer } = arg; if ((!consumerGroupId && !consumer) || !Array.isArray(topics) || topics.length === 0) { - return Promise.reject("sendOffsets must have the arguments {consumerGroupId: string or consumer: Consumer, topics: non-empty array"); + throw new error.KafkaJSError("sendOffsets arguments are invalid", { code: error.ErrorCodes.ERR__INVALID_ARG }); } if (this.#state !== ProducerState.CONNECTED) { - return Promise.reject("Cannot sendOffsets without awaiting connect()"); + throw new error.KafkaJSError("Cannot sendOffsets without awaiting connect()", { code: error.ErrorCodes.ERR__STATE }); } if (!this.#ongoingTransaction) { - return Promise.reject("Cannot sendOffsets, no transaction ongoing."); + throw new error.KafkaJSError("Cannot sendOffsets, no transaction ongoing.", { code: error.ErrorCodes.ERR__STATE }); } // If we don't have a consumer, we must create a consumer at this point internally. @@ -273,34 +356,44 @@ class Producer { if (consumerCreated) await consumer.disconnect(); if (err) - reject(err); + reject(createKafkaJsErrorFromLibRdKafkaError(err)); else resolve(); }) }); } + /** + * send(record: ProducerRecord): Promise + + * @param {import('../../types/kafkajs').ProducerRecord} sendOptions - The record to send. The keys `acks`, `timeout`, and `compression` are not used, and should not be set, rather, they should be set in the global config. + * @returns {Promise} Resolves with the record metadata for the messages. + */ async send(sendOptions) { if (this.#state !== ProducerState.CONNECTED) { - return Promise.reject("Cannot send message without awaiting connect()"); + throw new error.KafkaJSError("Cannot send without awaiting connect()", { code: error.ErrorCodes.ERR__STATE }); } if (sendOptions === null || !(sendOptions instanceof Object)) { - return Promise.reject("sendOptions must be set correctly"); + throw new error.KafkaJSError("sendOptions must be set correctly", { code: error.ErrorCodes.ERR__INVALID_ARG }); } // Ignore all properties except topic and messages. // TODO: log a warning instead of ignoring. - if (!sendOptions.hasOwnProperty("topic") || !sendOptions.hasOwnProperty("messages") || !Array.isArray(sendOptions["messages"])) { + if (!Object.hasOwn(sendOptions, "topic") || !Object.hasOwn(sendOptions, "messages") || !Array.isArray(sendOptions["messages"])) { // TODO: add further validations. - return Promise.reject("sendOptions must be of the form {topic: string, messages: Message[]}"); + throw new error.KafkaJSError("sendOptions must be of the form {topic: string, messages: Message[]}", { code: error.ErrorCodes.ERR__INVALID_ARG }); + } + + if (Object.hasOwn(sendOptions, "acks") || Object.hasOwn(sendOptions, "timeout") || Object.hasOwn(sendOptions, "compression")) { + throw new error.KafkaJSError("sendOptions must not contain acks, timeout, or compression", { code: error.ErrorCodes.ERR__INVALID_ARG }); } const msgPromises = []; for (let i = 0; i < sendOptions.messages.length; i++) { const msg = sendOptions.messages[i]; - if (!msg.hasOwnProperty("partition") || msg.partition === null) { + if (!Object.hasOwn(msg, "partition") || msg.partition === null) { msg.partition = -1; } @@ -308,12 +401,23 @@ class Producer { msg.value = Buffer.from(msg.value); } + if (Object.hasOwn(msg, "timestamp") && msg.timestamp) { + msg.timestamp = Number(msg.timestamp); + } else { + msg.timestamp = 0; + } + + msg.headers = convertToRdKafkaHeaders(msg.headers); + msgPromises.push(new Promise((resolve, reject) => { const opaque = { resolve, reject }; - this.#internalClient.produce(sendOptions.topic, msg.partition, msg.value, msg.key, msg.timestamp ?? Date.now(), opaque, msg.headers); + this.#internalClient.produce(sendOptions.topic, msg.partition, msg.value, msg.key, msg.timestamp, opaque, msg.headers); })); } + + /* The delivery report will be handled by the delivery-report event handler, and we can simply wait for it here. */ + const recordMetadataArr = await Promise.all(msgPromises); const topicPartitionRecordMetadata = new Map(); @@ -337,7 +441,7 @@ class Producer { } const ret = []; - for (const [key, value] of topicPartitionRecordMetadata.entries()) { + for (const value of topicPartitionRecordMetadata.values()) { value.baseOffset = value.baseOffset?.toString(); ret.push(value); } diff --git a/types/kafkajs.d.ts b/types/kafkajs.d.ts index 22e04c62..37bbfd9e 100644 --- a/types/kafkajs.d.ts +++ b/types/kafkajs.d.ts @@ -1,5 +1,5 @@ import * as tls from 'tls' -import { ConsumerGlobalConfig, ConsumerTopicConfig, ProducerGlobalConfig, ProducerTopicConfig } from './config' +import { ConsumerGlobalConfig, ConsumerTopicConfig, GlobalConfig, ProducerGlobalConfig, ProducerTopicConfig, TopicConfig } from './config' export type BrokersFunction = () => string[] | Promise @@ -29,6 +29,7 @@ export interface KafkaConfig { reauthenticationThreshold?: number requestTimeout?: number enforceRequestTimeout?: boolean + rdKafka?: Function | { topicConfig?: TopicConfig, globalConfig?: GlobalConfig }; } export interface ProducerConfig { From 9f5c734a2018c392acf19782cce172c6e597bf78 Mon Sep 17 00:00:00 2001 From: Milind L Date: Fri, 1 Dec 2023 12:56:43 +0530 Subject: [PATCH 05/26] Add consumer errors and clean up consumer code --- MIGRATION.md | 78 ++++++++- lib/kafkajs/_common.js | 6 + lib/kafkajs/_consumer.js | 365 +++++++++++++++++++++++++++++---------- lib/kafkajs/_producer.js | 2 +- 4 files changed, 356 insertions(+), 95 deletions(-) diff --git a/MIGRATION.md b/MIGRATION.md index 5794f5ad..76952c61 100644 --- a/MIGRATION.md +++ b/MIGRATION.md @@ -61,9 +61,9 @@ ### Producer -* `sendBatch` is currently unsupported - but will be supported. TODO. However, the actual batching semantics are handled by librdkafka. +* `sendBatch` is not supported (YET). However, the actual batching semantics are handled by librdkafka. * Changes to `send`: - 1. `acks`, `compression` and `timeout` are not set on a per-send basis. Rather, they must be configured in the configuration. + * `acks`, `compression` and `timeout` are not set on a per-send basis. Rather, they must be configured in the configuration. Before: ```javascript const kafka = new Kafka({/* ... */}); @@ -99,8 +99,80 @@ }); ``` - * Error-handling for a failed `send` is stricter. While sending multiple messages, even if one of the messages fails, the method throws an error. + * Error-handling for a failed `send` is stricter. While sending multiple messages, even if one of the messages fails, the method throws an error. ### Consumer + * While passing a list of topics to `subscribe`, the `fromBeginning` property is not supported. Instead, the property `auto.offset.reset` needs to be used. + Before: + ```javascript + const kafka = new Kafka({ /* ... */ }); + const consumer = kafka.consumer({ + groupId: 'test-group', + }); + await consumer.connect(); + await consumer.subscribe({ topics: ["topic"], fromBeginning: true}); + ``` + + After: + ```javascript + const kafka = new Kafka({ /* ... */ }); + const consumer = kafka.consumer({ + groupId: 'test-group', + rdKafka: { + topicConfig: { + 'auto.offset.reset': 'earliest', + }, + } + }); + await consumer.connect(); + await consumer.subscribe({ topics: ["topic"] }); + ``` + + * For auto-commiting using a consumer, the properties on `run` are no longer used. Instead, corresponding rdKafka properties must be set. + * `autoCommit` corresponds to `enable.auto.commit`. + * `autoCommitInterval` corresponds to `auto.commit.interval.ms`. + * `autoCommitThreshold` is no longer supported. + + Before: + ```javascript + const kafka = new Kafka({ /* ... */ }); + const consumer = kafka.consumer({ /* ... */ }); + await consumer.connect(); + await consumer.subscribe({ topics: ["topic"] }); + consumer.run({ + eachMessage: someFunc, + autoCommit: true, + autoCommitThreshold: 5000, + }); + ``` + + After: + ```javascript + const kafka = new Kafka({ /* ... */ }); + const consumer = kafka.consumer({ + /* ... */, + rdKafka: { + globalConfig: { + "enable.auto.commit": "true", + "auto.commit.interval.ms": "5000", + } + }, + }); + await consumer.connect(); + await consumer.subscribe({ topics: ["topic"] }); + consumer.run({ + eachMessage: someFunc, + }); + ``` + + * For the `eachMessage` method while running the consumer: + * The `heartbeat()` no longer needs to be called. Heartbeats are automatically managed by librdkafka. + * The `partitionsConsumedConcurrently` property is not supported (YET). + * The `eachBatch` method is not supported. + * `commitOffsets` does not (YET) support sending metadata for topic partitions being commited. + * `paused()` is not (YET) supported. + * Custom partition assignors are not supported. + + ## node-rdkafka diff --git a/lib/kafkajs/_common.js b/lib/kafkajs/_common.js index 5c8423a3..2878c9fa 100644 --- a/lib/kafkajs/_common.js +++ b/lib/kafkajs/_common.js @@ -125,9 +125,15 @@ function convertToRdKafkaHeaders(kafkaJSHeaders) { return headers; } + +function notImplemented(msg = 'Not implemented') { + throw new error.KafkaJSError(msg, { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); +} + module.exports = { kafkaJSToRdKafkaConfig, topicPartitionOffsetToRdKafka, createKafkaJsErrorFromLibRdKafkaError, convertToRdKafkaHeaders, + notImplemented, }; diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 776d9d4b..ca620b90 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -1,6 +1,13 @@ const LibrdKafkaError = require('../error'); +const error = require('./_error'); const RdKafka = require('../rdkafka'); -const { kafkaJSToRdKafkaConfig, topicPartitionOffsetToRdKafka } = require('./_common'); +const { + kafkaJSToRdKafkaConfig, + topicPartitionOffsetToRdKafka, + createKafkaJsErrorFromLibRdKafkaError, + notImplemented +} = require('./_common'); +const { Buffer } = require('buffer'); const ConsumerState = Object.freeze({ INIT: 0, @@ -11,10 +18,34 @@ const ConsumerState = Object.freeze({ }); class Consumer { - #kJSConfig = null + /** + * kJSConfig is the kafkaJS config object. + * @type {import("../../types/kafkajs").ConsumerConfig|null} + */ + #kJSConfig = null; + + /** + * rdKafkaConfig contains the config objects that will be passed to node-rdkafka. + * @type {{globalConfig: import("../../types/config").ConsumerGlobalConfig, topicConfig: import("../../types/config").ConsumerTopicConfig}|null} + */ #rdKafkaConfig = null; + + /** + * internalClient is the node-rdkafka client used by the API. + * @type {import("../rdkafka").Consumer|null} + */ #internalClient = null; + + /** + * connectPromiseFunc is the set of promise functions used to resolve/reject the connect() promise. + * @type {{resolve: Function, reject: Function}|{}} + */ #connectPromiseFunc = {}; + + /** + * state is the current state of the consumer. + * @type {ConsumerState} + */ #state = ConsumerState.INIT; /** @@ -31,6 +62,56 @@ class Consumer { return this.#rdKafkaConfig; } + /** + * Used as a trampoline to the user's rebalance listener, if any. + * @param {Error} err - error in rebalance + * @param {import("../../types").TopicPartition[]} assignment + */ + #rebalanceCallback(err, assignment) { + // Create the librdkafka error + err = LibrdKafkaError.create(err); + + let call; + switch (err.code) { + // TODO: is this the right way to handle this error? + // We might just be able to throw, because the error is something the user has caused. + case LibrdKafkaError.codes.ERR__ASSIGN_PARTITIONS: + call = (this.#kJSConfig.rebalanceListener.onPartitionsAssigned ? + this.#kJSConfig.rebalanceListener.onPartitionsAssigned(assignment) : + Promise.resolve()).catch(console.error); + break; + case LibrdKafkaError.codes.ERR__REVOKE_PARTITIONS: + call = (this.#kJSConfig.rebalanceListener.onPartitionsRevoked ? + this.#kJSConfig.rebalanceListener.onPartitionsRevoked(assignment) : + Promise.resolve()).catch(console.error); + break; + default: + call = Promise.reject(`Unexpected rebalanceListener error code ${err.code}`).catch((e) => { + console.error(e); + }); + break; + } + + call + .finally(() => { + // Emit the event + this.#internalClient.emit('rebalance', err, assignment); + + try { + if (err.code === LibrdKafkaError.codes.ERR__ASSIGN_PARTITIONS) { + this.#internalClient.assign(assignment); + } else { + this.#internalClient.unassign(); + } + } catch (e) { + // Ignore exceptions if we are not connected + if (this.#internalClient.isConnected()) { + this.#internalClient.emit('rebalance.error', e); + } + } + }); + } + async #finalizedConfig() { const { globalConfig, topicConfig } = await kafkaJSToRdKafkaConfig(this.#kJSConfig); if (this.#kJSConfig.groupId) { @@ -38,56 +119,15 @@ class Consumer { } globalConfig['offset_commit_cb'] = true; if (this.#kJSConfig.rebalanceListener) { - globalConfig['rebalance_cb'] = (err, assignment) => { - // Create the librdkafka error - err = LibrdKafkaError.create(err); - - let call; - switch (err.code) { - case LibrdKafkaError.codes.ERR__ASSIGN_PARTITIONS: - call = (this.#kJSConfig.rebalanceListener.onPartitionsAssigned ? - this.#kJSConfig.rebalanceListener.onPartitionsAssigned(assignment) : - Promise.resolve()).catch(console.error); - break; - case LibrdKafkaError.codes.ERR__REVOKE_PARTITIONS: - call = (this.#kJSConfig.rebalanceListener.onPartitionsRevoked ? - this.#kJSConfig.rebalanceListener.onPartitionsRevoked(assignment) : - Promise.resolve()).catch(console.error); - break; - default: - call = Promise.reject().catch(() => { - console.error(`Unexpected rebalanceListener error code ${err.code}`); - }); - break; - } - - call - .finally(() => { - // Emit the event - this.#internalClient.emit('rebalance', err, assignment); - - try { - if (err.code === LibrdKafkaError.codes.ERR__ASSIGN_PARTITIONS) { - this.#internalClient.assign(assignment); - } else { - this.#internalClient.unassign(); - } - } catch (e) { - // Ignore exceptions if we are not connected - if (this.#internalClient.isConnected()) { - this.#internalClient.emit('rebalance.error', e); - } - } - }); - }; + globalConfig['rebalance_cb'] = this.#rebalanceCallback.bind(this); } return { globalConfig, topicConfig }; } - #readyCb(arg) { + #readyCb() { if (this.#state !== ConsumerState.CONNECTING) { - // I really don't know how to handle this now. - return; + /* The connectPromiseFunc might not be set, so we throw such an error. It's a state error that we can't recover from. Probably a bug. */ + throw new error.KafkaJSError(`Ready callback called in invalid state ${this.#state}`, { code: error.ErrorCodes.ERR__STATE }); } this.#state = ConsumerState.CONNECTED; @@ -95,33 +135,40 @@ class Consumer { this.#connectPromiseFunc['resolve'](); } - #errorCb(args) { - console.log('error', args); + /** + * Callback for the event.error event, either fails the initial connect(), or logs the error. + * @param {Error} err + */ + #errorCb(err) { if (this.#state === ConsumerState.CONNECTING) { - this.#connectPromiseFunc['reject'](args); + this.#connectPromiseFunc['reject'](err); } else { - // do nothing for now. + /* TODO: we should log the error returned here, depending on the log level. + * Right now, we're just using console.err, but we should allow for a custom + * logger, or at least make a function in _common.js that handles consumer + * and producer. */ + console.error(err); } } - #notImplemented() { - throw new Error('Not implemented'); - } - + /** + * Converts a message returned by node-rdkafka into a message that can be used by the eachMessage callback. + * @param {import("../..").Message} message + * @returns {import("../../types/kafkajs").EachMessagePayload} + */ #createPayload(message) { - var key = message.key == null ? null : message.key; + let key = message.key; if (typeof key === 'string') { key = Buffer.from(key); } - let timestamp = message.timestamp ? new Date(message.timestamp).toISOString() - : ''; + let timestamp = message.timestamp ? String(message.timestamp) : ''; - var headers = undefined; + let headers; if (message.headers) { headers = {} for (const [key, value] of Object.entries(message.headers)) { - if (!headers[key]) { + if (!Object.hasOwn(headers, key)) { headers[key] = value; } else if (headers[key].constructor === Array) { headers[key].push(value); @@ -139,39 +186,47 @@ class Consumer { value: message.value, timestamp, attributes: 0, - offset: message.offset, + offset: String(message.offset), size: message.size, headers }, - heartbeat: async () => { }, - pause: () => { } - } + heartbeat: async () => { /* no op */ }, + pause: () => this.pause([{ topic: message.topic, partitions: [message.partition] }]), + }; } + /** + * Consumes a single message from the internal consumer. + * @returns {Promise} a promise that resolves to a single message. + */ async #consumeSingle() { return new Promise((resolve, reject) => { this.#internalClient.consume(1, function (err, messages) { if (err) { - reject(`Consume error code ${err.code}`); + reject(createKafkaJsErrorFromLibRdKafkaError(err)); return; } - const message = messages[0]; resolve(message); }); }); } + /** + * Flattens a list of topics with partitions into a list of topic, partition. + * @param {({topic: string, partitions: number[]}|{topic: string, partition: number})[]} topics + * @returns {import("../../types").TopicPartition[]} a list of (topic, partition). + */ #flattenTopicPartitions(topics) { const ret = []; - for (let topic of topics) { - if (topic.partition !== null) + for (const topic of topics) { + if (typeof topic.partition === 'number') ret.push({ topic: topic.topic, partition: topic.partition }); else { - for (let partition of topic.partitions) { + for (const partition of topic.partitions) { ret.push({ topic: topic.topic, partition }); } } @@ -179,17 +234,25 @@ class Consumer { return ret; } + /** + * @returns {import("../rdkafka").Consumer} the internal node-rdkafka client. + */ _getInternalConsumer() { return this.#internalClient; } + /** + * Set up the client and connect to the bootstrap brokers. + * @returns {Promise} a promise that resolves when the consumer is connected. + */ async connect() { if (this.#state !== ConsumerState.INIT) { - return Promise.reject('Connect has already been called elsewhere.'); + throw new error.KafkaJSError('Connect has already been called elsewhere.', { code: error.ErrorCodes.ERR__STATE }); } - this.#state = ConsumerState.CONNECTING; const { globalConfig, topicConfig } = await this.#config(); + + this.#state = ConsumerState.CONNECTING; this.#internalClient = new RdKafka.KafkaConsumer(globalConfig, topicConfig); this.#internalClient.on('ready', this.#readyCb.bind(this)); this.#internalClient.on('event.error', this.#errorCb.bind(this)); @@ -197,38 +260,79 @@ class Consumer { return new Promise((resolve, reject) => { this.#connectPromiseFunc = { resolve, reject }; - console.log('Connecting....'); this.#internalClient.connect(); - console.log('connect() called'); }); } + /** + * Subscribes the consumer to the given topics. + * @param {import("../../types/kafkajs").ConsumerSubscribeTopics} subscription + */ async subscribe(subscription) { + if (typeof subscription.fromBeginning == 'boolean') { + throw new error.KafkaJSError( + 'fromBeginning is not supported by subscribe(), but must be passed as rdKafka properties to the consumer', + { code: error.ErrorCodes.ERR__INVALID_ARG }); + } + this.#internalClient.subscribe(subscription.topics); } async stop() { - this.#notImplemented(); + notImplemented(); } + /** + * Starts consumer polling. + * @param {import("../../types/kafkajs").ConsumerRunConfig} config + */ async run(config) { if (this.#state !== ConsumerState.CONNECTED) { - throw new Error('Run must be called in state CONNECTED.'); + throw new error.KafkaJSError('Run must be called after a successful connect().', { code: error.ErrorCodes.ERR__STATE }); + } + + if (typeof config.autoCommit == 'boolean' || typeof config.autoCommitInterval == 'number' || typeof config.autoCommitThreshold == 'number') { + throw new error.KafkaJSError( + 'autoCommit related properties are not supported by run(), but must be passed as rdKafka properties to the consumer.', + { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); } while (this.#state === ConsumerState.CONNECTED) { - let m = await this.#consumeSingle(); + const m = await this.#consumeSingle(); if (m) { + /* TODO: add partitionsConsumedConcurrently-based concurrency here. + * If we maintain a map of topic partitions to promises, and a counter, + * we can probably achieve it with the correct guarantees of ordering + * though to maximize performance, we need to consume only from partitions for which + * an eachMessage call is not already going. + * It's risky to consume, and then store the message in something like an + * array/list until it can be processed, because librdkafka marks it as + * 'stored'... but anyway - we can implement something like this. + */ await config.eachMessage( this.#createPayload(m) ) + /* TODO: another check we need to do here is to see how kafkaJS is handling + * commits. Are they commmitting after a message is _processed_? + * In that case we need to turn off librdkafka's auto-commit, and commit + * inside this function. + */ } } } + /** + * Commit offsets for the given topic partitions. If topic partitions are not specified, commits all offsets. + * @param {import("../../types/kafkajs").TopicPartitionOffsetAndMetadata[]?} topicPartitions + * @returns {Promise} a promise that resolves when the offsets have been committed. + */ async commitOffsets(topicPartitions = null) { + if (this.#state !== ConsumerState.CONNECTED) { + throw new error.KafkaJSError('Commit can only be called while connected.', { code: error.ErrorCodes.ERR__STATE }); + } + try { - if (topicPartitions == null) { + if (topicPartitions === null) { this.#internalClient.commitSync(); } else { const topicPartitions = topicPartitions.map( @@ -236,69 +340,148 @@ class Consumer { this.#internalClient.commitSync(topicPartitions); } } catch (e) { - if (!e.code || e.code != LibrdKafkaError.codes.ERR__NO_OFFSET) { - throw e; + if (!e.code || e.code !== error.ErrorCodes.ERR__NO_OFFSET) { + throw createKafkaJsErrorFromLibRdKafkaError(e); } } } + /** + * Seek to the given offset for the topic partition. + * @param {import("../../types/kafkajs").TopicPartitionOffset} topicPartitionOffset + * @returns {Promise} a promise that resolves when the consumer has seeked. + */ seek(topicPartitionOffset) { + if (this.#state !== ConsumerState.CONNECTED) { + throw new error.KafkaJSError('Seek can only be called while connected.', { code: error.ErrorCodes.ERR__STATE }); + } + return new Promise((resolve, reject) => { const rdKafkaTopicPartitionOffset = topicPartitionOffsetToRdKafka(topicPartitionOffset); this.#internalClient.seek(rdKafkaTopicPartitionOffset, 0, (err) => { if (err) { - reject(new Error(`Seek error code ${err.code}`)); + reject(createKafkaJsErrorFromLibRdKafkaError(err)); } else { resolve(); } }); - }).catch(console.error); // Default handler + }); } async describeGroup() { - this.#notImplemented(); + notImplemented(); + } + + /** + * Find the assigned topic partitions for the consumer. + * @returns {import("../../types").TopicPartition[]} the current assignment. + */ + assignment() { + if (this.#state !== ConsumerState.CONNECTED) { + throw new error.KafkaJSError('Assignment can only be called while connected.', { code: error.ErrorCodes.ERR__STATE }); + } + + return this.#flattenTopicPartitions(this.#internalClient.assignments()); + } + + /** + * Fetches all partitions of topic that are assigned to this consumer. + * @param {string} topic + * @returns {number[]} a list of partitions. + */ + #getAllAssignedPartition(topic) { + return this.#internalClient.assignments() + .filter((partition) => partition.topic === topic) + .map((tpo) => tpo.partition); } + /** + * Pauses the given topic partitions. If partitions are not specified, pauses + * all partitions for the given topic. If topic partition(s) are already paused + * this method has no effect. + * @param {{topic: string, partitions?: number[]}[]} topics + */ pause(topics) { + if (this.#state !== ConsumerState.CONNECTED) { + throw new error.KafkaJSError('Pause can only be called while connected.', { code: error.ErrorCodes.ERR__STATE }); + } + + for (let topic of topics) { + if (!topic.partitions) { + topic.partitions = this.#getAllAssignedPartition(topic.topic); + } + } + topics = this.#flattenTopicPartitions(topics); + if (topics.length === 0) { + return; + } + this.#internalClient.pause(topics); } paused() { - this.#notImplemented(); + notImplemented(); } - assignment() { - return this.#flattenTopicPartitions(this.#internalClient.assignments()); - } + /** + * Resumes the given topic partitions. If partitions are not specified, resumes + * all partitions for the given topic. If topic partition(s) are already resumed + * this method has no effect. + * @param {{topic: string, partitions?: number[]}[]} topics + */ resume(topics) { + if (this.#state !== ConsumerState.CONNECTED) { + throw new error.KafkaJSError('Resume can only be called while connected.', { code: error.ErrorCodes.ERR__STATE }); + } + + for (let topic of topics) { + if (!topic.partitions) { + topic.partitions = this.#getAllAssignedPartition(topic.topic); + } + } + topics = this.#flattenTopicPartitions(topics); this.#internalClient.resume(topics); } - on(eventName, listener) { - this.#notImplemented(); + on(/* eventName, listener */) { + notImplemented(); } logger() { - this.#notImplemented(); + notImplemented(); } get events() { - this.#notImplemented(); + notImplemented(); + return null; } + /** + * Disconnects and cleans up the consumer. + * @returns {Promise} a promise that resolves when the consumer has disconnected. + */ async disconnect() { + if (this.#state === ConsumerState.INIT) { + throw new error.KafkaJSError('Disconnect can only be called once consumer is connected.', { code: error.ErrorCodes.ERR__STATE }); + } + if (this.#state >= ConsumerState.DISCONNECTING) { return; } + this.#state = ConsumerState.DISCONNECTING; await new Promise((resolve, reject) => { const cb = (err) => { - err ? reject(err) : resolve(); + if (err) { + reject(createKafkaJsErrorFromLibRdKafkaError(err)); + return; + } this.#state = ConsumerState.DISCONNECTED; + resolve(); } this.#internalClient.disconnect(cb); }); diff --git a/lib/kafkajs/_producer.js b/lib/kafkajs/_producer.js index a6b68588..8181cbf7 100644 --- a/lib/kafkajs/_producer.js +++ b/lib/kafkajs/_producer.js @@ -102,7 +102,7 @@ class Producer { } this.#state = ProducerState.INITIALIZED_TRANSACTIONS; - this.#readyCb(null); + this.#readyCb(); } /** From 6a3f5ce7b9de6ebc7410df29511afb129cb97bf7 Mon Sep 17 00:00:00 2001 From: Milind L Date: Sun, 3 Dec 2023 16:15:02 +0530 Subject: [PATCH 06/26] Exhaustive config compability --- MIGRATION.md | 135 +++++++++++++++++++++++++++++++++--- lib/kafkajs/_common.js | 143 +++++++++++++++++++++++++++++++++++---- lib/kafkajs/_consumer.js | 82 +++++++++++++++++++++- lib/kafkajs/_kafka.js | 35 +++++----- lib/kafkajs/_producer.js | 32 ++++++++- lib/kafkajs/index.js | 4 +- types/kafkajs.d.ts | 6 +- 7 files changed, 384 insertions(+), 53 deletions(-) diff --git a/MIGRATION.md b/MIGRATION.md index 76952c61..08d1c418 100644 --- a/MIGRATION.md +++ b/MIGRATION.md @@ -4,21 +4,70 @@ ### Common -* Configuration changes +#### Configuration changes + ```javascript + const kafka = new Kafka({/* common configuration changes */}); + ``` + There are several changes in the common configuration. Each config property is discussed. + If there needs to be any change, the property is highlighted. -* Error Handling: Some possible subtypes of `KafkaJSError` have been removed, + * An `rdKafka` block can be added to the config. It allows directly setting librdkafka properties. + If you are starting to make the configuration anew, it is best to specify properties using + the `rdKafka` block. [Complete list of properties here](https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md). + + Example: + ```javascript + const kafka = new Kafka({ + rdKafka: { + globalConfig: { /* properties mentioned within the 'global config' section of the list */ } + topicConfig: { /* properties mentioned within the 'topic config' section of the list */ } + }, + /* ... */ + }); + ``` + * **`brokers`** list of strings, representing the bootstrap brokers. + a function is no longer allowed as an argument for this. + * **`ssl`**: boolean, set true if ssl needs to be enabled. + In case additional properties, like CA, Certificate, Key etc. need to be added, use the `rdKafka` block. + * **`sasl`**: omit if the brokers need no authentication, otherwise, an object of the following forms: + - For SASL PLAIN or SASL SCRAM : `{ mechanism: 'plain'|'scram-sha-256'|'scram-sha-512', username: string, password: string }` + - For SASL OAUTHBEARER: not supported yet. + - For AWS IAM or custom mechanisms: not supported with no planned support. + - For GSSAPI/Kerberos: use the `rdKafka` configuration. + * `clientId`: string for identifying this client. + * **`connectionTimeout`** and **`authenticationTimeout`**: + These timeouts (specified in milliseconds) are not enforced individually. Instead, the sum of these values is + enforced. The default value of the sum is 30000. It corresponds to librdkafka's `socket.connection.setup.timeout.ms`. + * **`reauthenticationThreshold`**: no longer checked, librdkafka handles reauthentication on its own. + * **`requestTimeout`**: number of milliseconds for a network request to timeout. The default value has been changed to 60000. It now corresponds to librdkafka's `socket.timeout.ms`. + * **`enforceRequestTimeout`**: if this is set to false, `requestTimeout` is set to 5 minutes. The timeout cannot be disabled completely. + * **`retry`** is partially supported. It must be an object, with the following (optional) properties + - `maxRetryTime`: maximum time to backoff a retry, in milliseconds. Corresponds to librdkafka's `retry.backoff.max.ms`. The default is 1000. + - `initialRetryTime`: minimum time to backoff a retry, in milliseconds. Corresponds to librdkafka's `retry.backoff.ms`. The default is 100. + - `retries`: maximum number of retries, *only* applicable to Produce messages. However, it's recommended to keep this unset. + Librdkafka handles the number of retries, and rather than capping the number of retries, caps the total time spent + while sending the message, controlled by `message.timeout.ms`. + - `factor` and `multiplier` cannot be changed from their defaults of 0.2 and 2. + * **`restartOnFailure`**: this cannot be changed, and will always be true (the consumer recovers from errors on its own). + * `logLevel` is mapped to the syslog(3) levels supported by librdkafka. `LOG_NOTHING` is not YET supported, as some panic situations are still logged. + * **`socketFactory`** is no longer supported. + +#### Error Handling + + Some possible subtypes of `KafkaJSError` have been removed, and additional information has been added into `KafkaJSError`. - Internally, fields have been added denoting if the error is fatal, retriable, or abortable (the latter two only relevant for a - transactional producer). - Some error-specific fields have also been removed. An exhaustive list is at the bottom of this section. + Fields have been added denoting if the error is fatal, retriable, or abortable (the latter two only relevant for a transactional producer). + Some error-specific fields have also been removed. + + An exhaustive list of changes is at the bottom of this section. - For compability, as many error types as possible have been retained, but it is + For compatibility, as many error types as possible have been retained, but it is better to switch to checking the `error.code`. **Action**: Convert any checks based on `instanceof` and `error.name` or to error checks based on `error.code` or `error.type`. - **Example:**: + **Example:** ```javascript try { await producer.send(/* args */); @@ -61,6 +110,35 @@ ### Producer +#### Configuration changes + + ```javascript + const producer = kafka.producer({ /* producer-specific configuration changes. */}); + ``` + + There are several changes in the common configuration. Each config property is discussed. + If there needs to be any change, the property is highlighted. + + * **`createPartitioner`**: this is not supported (YET). For behaviour identical to the Java client (the DefaultPartitioner), + use the `rdKafka` block, and set the property `partitioner` to `murmur2_random`. This is critical + when planning to produce to topics where messages with certain keys have been produced already. + * **`retry`**: See the section for retry above. The producer config `retry` takes precedence over the common config `retry`. + * `metadataMaxAge`: Time in milliseconds after which to refresh metadata for known topics. The default value remains 5min. This + corresponds to the librdkafka property `topic.metadata.refresh.interval.ms` (and not `metadata.max.age.ms`). + * `allowAutoTopicCreation`: determines if a topic should be created if it doesn't exist while producing. True by default. + * `transactionTimeout`: The maximum amount of time in milliseconds that the transaction coordinator will wait for a transaction + status update from the producer before proactively aborting the ongoing transaction. The default value remains 60000. + Only applicable when `transactionalId` is set to true. + * `idempotent`: if set to true, ensures that messages are delivered exactly once and in order. False by default. + In case this is set to true, certain constraints must be respected for other properties, `maxInFlightRequests <= 5`, `retry.retries >= 0`. + * **`maxInFlightRequests`**: Maximum number of in-flight requests *per broker connection*. If not set, a very high limit is used. + * `transactionalId`: if set, turns this into a transactional producer with this identifier. This also automatically sets `idempotent` to true. + * An `rdKafka` block can be added to the config. It allows directly setting librdkafka properties. + If you are starting to make the configuration anew, it is best to specify properties using + the `rdKafka` block. [Complete list of properties here](https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md). + +#### Semantic and Per-Method Changes + * `sendBatch` is not supported (YET). However, the actual batching semantics are handled by librdkafka. * Changes to `send`: * `acks`, `compression` and `timeout` are not set on a per-send basis. Rather, they must be configured in the configuration. @@ -103,6 +181,44 @@ ### Consumer +#### Configuration changes + + ```javascript + const consumer = kafka.consumer({ /* producer-specific configuration changes. */}); + ``` + There are several changes in the common configuration. Each config property is discussed. + If there needs to be any change, the property is highlighted. The change could be a change in + the default values, some added/missing features, or a change in semantics. + + * **`partitionAssigners`**: The **default value** of this is changed to `[PartitionAssigners.range,PartitionAssigners.roundRobin]`. Support for range, roundRobin and cooperativeSticky + partition assignors is provided. The cooperative assignor cannot be used along with the other two, and there + is no support for custom assignors. An alias for these properties is also made available, `partitionAssignors` and `PartitionAssignors` to maintain + parlance with the Java client's terminology. + * **`sessionTimeout`**: If no heartbeats are received by the broker for a group member within the session timeout, the broker will remove the consumer from + the group and trigger a rebalance. The **default value** is changed to 45000. + * **`rebalanceTimeout`**: The maximum allowed time for each member to join the group once a rebalance has begun. The **default value** is changed to 300000. + Note, before changing: setting this value *also* changes the max poll interval. Message processing in `eachMessage` must not take more than this time. + * `heartbeatInterval`: The expected time in milliseconds between heartbeats to the consumer coordinator. The default value remains 3000. + * `metadataMaxAge`: Time in milliseconds after which to refresh metadata for known topics. The default value remains 5min. This + corresponds to the librdkafka property `topic.metadata.refresh.interval.ms` (and not `metadata.max.age.ms`). + * **`allowAutoTopicCreation`**: determines if a topic should be created if it doesn't exist while producing. The **default value** is changed to false. + * **`maxBytesPerPartition`**: determines how many bytes can be fetched in one request from a single partition. The default value remains 1048576. + There is a slight change in semantics, this size grows dynamically if a single message larger than this is encountered, + and the client does not get stuck. + * `minBytes`: Minimum number of bytes the broker responds with (or wait until `maxWaitTimeInMs`). The default remains 1. + * **`maxBytes`**: Maximum number of bytes the broker responds with. The **default value** is changed to 52428800 (50MB). + * **`maxWaitTimeInMs`**: Maximum time in milliseconds the broker waits for the `minBytes` to be fulfilled. The **default value** is changed to 500. + * **`retry`**: See the section for retry above. The consumer config `retry` takes precedence over the common config `retry`. + * `readUncommitted`: if true, consumer will read transactional messages which have not been committed. The default value remains false. + * **`maxInFlightRequests`**: Maximum number of in-flight requests *per broker connection*. If not set, a very high limit is used. + * `rackId`: Can be set to an arbitrary string which will be used for fetch-from-follower if set up on the cluster. + * An `rdKafka` block can be added to the config. It allows directly setting librdkafka properties. + If you are starting to make the configuration anew, it is best to specify properties using + the `rdKafka` block. [Complete list of properties here](https://github.com/confluentinc/librdkafka/blob/master/CONFIGURATION.md). + +#### Semantic and Per-Method Changes + + * While passing a list of topics to `subscribe`, the `fromBeginning` property is not supported. Instead, the property `auto.offset.reset` needs to be used. Before: ```javascript @@ -129,7 +245,7 @@ await consumer.subscribe({ topics: ["topic"] }); ``` - * For auto-commiting using a consumer, the properties on `run` are no longer used. Instead, corresponding rdKafka properties must be set. + * For auto-committing using a consumer, the properties on `run` are no longer used. Instead, corresponding rdKafka properties must be set. * `autoCommit` corresponds to `enable.auto.commit`. * `autoCommitInterval` corresponds to `auto.commit.interval.ms`. * `autoCommitThreshold` is no longer supported. @@ -170,9 +286,8 @@ * The `heartbeat()` no longer needs to be called. Heartbeats are automatically managed by librdkafka. * The `partitionsConsumedConcurrently` property is not supported (YET). * The `eachBatch` method is not supported. - * `commitOffsets` does not (YET) support sending metadata for topic partitions being commited. + * `commitOffsets` does not (YET) support sending metadata for topic partitions being committed. * `paused()` is not (YET) supported. * Custom partition assignors are not supported. - ## node-rdkafka diff --git a/lib/kafkajs/_common.js b/lib/kafkajs/_common.js index 2878c9fa..a0fa3392 100644 --- a/lib/kafkajs/_common.js +++ b/lib/kafkajs/_common.js @@ -1,39 +1,151 @@ const error = require("./_error"); +const process = require("process"); + +const logLevel = Object.freeze({ + NOTHING: 0, + ERROR: 1, + WARN: 2, + INFO: 3, + DEBUG: 4, +}); /** - * @function kafkaJSToRdKafkaConfig() + * Converts the common configuration from KafkaJS to a format that can be used by node-rdkafka. * @param {object} config * @returns {{globalConfig: import("../../types/config").ConsumerGlobalConfig|import("../../types/config").ProducerTopicConfig, topicConfig: import("../../types/config").ConsumerTopicConfig|import("../../types/config").ProducerTopicConfig}} + * @throws {error.KafkaJSError} if the configuration is invalid. + * The error code will be ERR__INVALID_ARG in case of invalid arguments or features that are not supported. + * The error code will be ERR__NOT_IMPLEMENTED in case of features that are not yet implemented. */ async function kafkaJSToRdKafkaConfig(config) { - const globalConfig = { - "allow.auto.create.topics": "false", - }; + const globalConfig = {}; const topicConfig = {}; + + if (!Array.isArray(config["brokers"])) { + throw new error.KafkaJSError("brokers must be an list of strings", { + code: error.ErrorCodes.ERR__INVALID_ARG, + }); + } globalConfig["bootstrap.servers"] = config["brokers"].join(","); + if (Object.hasOwn(config, "clientId")) { + globalConfig["client.id"] = config.clientId; + } + let withSASL = false; - if (config.sasl) { + if (Object.hasOwn(config, "sasl")) { const sasl = config.sasl; - if ( - sasl.mechanism === "plain" && - typeof sasl.username === "string" && - typeof sasl.password === "string" - ) { - globalConfig["sasl.mechanism"] = "PLAIN"; - globalConfig["sasl.username"] = sasl.username; - globalConfig["sasl.password"] = sasl.password; - withSASL = true; + const mechanism = sasl.mechanism.toUpperCase(); + + if (mechanism === 'OAUTHBEARER') { + throw new error.KafkaJSError("OAUTHBEARER is not supported", { + code: error.ErrorCodes.ERR__NOT_IMPLEMENTED, + }); + } + + /* The mechanism must be PLAIN or SCRAM. */ + + if (typeof sasl.username !== "string" || typeof sasl.password !== "string") { + throw new error.KafkaJSError("username and password must be present and be strings", { + code: error.ErrorCodes.ERR__INVALID_ARG, + }); } + + globalConfig["sasl.mechanism"] = mechanism; + globalConfig["sasl.username"] = sasl.username; + globalConfig["sasl.password"] = sasl.password; + withSASL = true; } - if (config.ssl === true && withSASL) { + if (Object.hasOwn(config, "ssl") && withSASL) { globalConfig["security.protocol"] = "sasl_ssl"; } else if (withSASL) { globalConfig["security.protocol"] = "sasl_plaintext"; } + if (Object.hasOwn(config, "requestTimeout")) { + globalConfig["socket.timeout.ms"] = config.requestTimeout; + } + + if (Object.hasOwn(config, "enforceRequestTimeout")) { + globalConfig["socket.timeout.ms"] = 300000; + } + + const connectionTimeout = config.connectionTimeout ?? 0; + const authenticationTimeout = config.authenticationTimeout ?? 0; + let totalConnectionTimeout = Number(connectionTimeout) + Number(authenticationTimeout); + + /* The minimum value for socket.connection.setup.timeout.ms is 1000. */ + if (totalConnectionTimeout) { + totalConnectionTimeout = Math.max(totalConnectionTimeout, 1000); + globalConfig["socket.connection.setup.timeout.ms"] = totalConnectionTimeout; + } + + if (Object.hasOwn(config, "retry")) { + const { maxRetryTime, initialRetryTime, factor, multiplier, retries } = config.retry; + + if (maxRetryTime) { + globalConfig["retry.backoff.max.ms"] = maxRetryTime; + } + + if (initialRetryTime) { + globalConfig["retry.backoff.ms"] = initialRetryTime; + } + + if (retries) { + globalConfig["retries"] = retries; + } + + if (factor || multiplier) { + throw new error.KafkaJSError("retry.factor and retry.multiplier are not supported", { + code: error.ErrorCodes.ERR__INVALID_ARG, + }); + } + } + + if (Object.hasOwn(config, "restartOnFailure") && !config.restartOnFailure) { + throw new error.KafkaJSError("restartOnFailure cannot be false, it must be true or unset", { + code: error.ErrorCodes.ERR__INVALID_ARG, + }); + } + + if (Object.hasOwn(config, "socketFactory")) { + throw new error.KafkaJSError("socketFactory is not supported", { + code: error.ErrorCodes.ERR__INVALID_ARG, + }); + } + + if (Object.hasOwn(config, "logLevel")) { + let setLevel = config.logLevel; + + if (process.env.KAFKAJS_LOG_LEVEL) { + setLevel = logLevel[process.env.KAFKAJS_LOG_LEVEL.toUpperCase()]; + } + + switch (setLevel) { + case logLevel.NOTHING: + globalConfig["log_level"] = 0; /* LOG_EMERG - we don't have a true log nothing yet */ + break; + case logLevel.ERROR: + globalConfig["log_level"] = 3 /* LOG_ERR */; + break; + case logLevel.WARN: + globalConfig["log_level"] = 4 /* LOG_WARNING */; + break; + case logLevel.INFO: + globalConfig["log_level"] = 6 /* LOG_INFO */; + break; + case logLevel.DEBUG: + globalConfig["debug"] = "all" /* this will set librdkafka log_level to 7 */; + break; + default: + throw new error.KafkaJSError("Invalid logLevel", { + code: error.ErrorCodes.ERR__INVALID_ARG, + }); + } + } + if (config.rdKafka) { if (config.rdKafka.constructor === Function) { await config.rdKafka(globalConfig, topicConfig); @@ -136,4 +248,5 @@ module.exports = { createKafkaJsErrorFromLibRdKafkaError, convertToRdKafkaHeaders, notImplemented, + logLevel, }; diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index ca620b90..df94503a 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -17,6 +17,12 @@ const ConsumerState = Object.freeze({ DISCONNECTED: 4, }); +const PartitionAssigners = Object.freeze({ + roundRobin: 'roundrobin', + range: 'range', + cooperativeSticky: 'cooperative-sticky', +}); + class Consumer { /** * kJSConfig is the kafkaJS config object. @@ -113,10 +119,77 @@ class Consumer { } async #finalizedConfig() { + /* This sets the common configuration options for the client. */ const { globalConfig, topicConfig } = await kafkaJSToRdKafkaConfig(this.#kJSConfig); - if (this.#kJSConfig.groupId) { + + /* Consumer specific configuration */ + + if (Object.hasOwn(this.#kJSConfig, 'groupId')) { globalConfig['group.id'] = this.#kJSConfig.groupId; } + + if (Object.hasOwn(this.#kJSConfig, 'partitionAssigners')) { + this.#kJSConfig.partitionAssignors = this.#kJSConfig.partitionAssigners; + } + + if (Object.hasOwn(this.#kJSConfig, 'partitionAssignors')) { + this.#kJSConfig.partitionAssignors.forEach(assignor => { + if (typeof assignor !== 'string') + throw new error.KafkaJSError('partitionAssignors must be a list of strings from within `PartitionAssignors`', { code: error.ErrorCodes.ERR__INVALID_ARG }); + }); + + globalConfig['partition.assignment.strategy'] = this.#kJSConfig.partitionAssignors.join(','); + } + + if (Object.hasOwn(this.#kJSConfig, 'sessionTimeout')) { + globalConfig['session.timeout.ms'] = this.#kJSConfig.sessionTimeout; + } + + if (Object.hasOwn(this.#kJSConfig, 'rebalanceTimeout')) { + /* In librdkafka, we use the max poll interval as the rebalance timeout as well. */ + globalConfig['max.poll.interval.ms'] = this.#kJSConfig.rebalanceTimeout; + } + + if (Object.hasOwn(this.#kJSConfig, 'heartbeatInterval')) { + globalConfig['heartbeat.interval.ms'] = this.#kJSConfig.heartbeatInterval; + } + + if (Object.hasOwn(this.#kJSConfig, 'metadataMaxAge')) { + globalConfig['topic.metadata.refresh.interval.ms'] = this.#kJSConfig.metadataMaxAge; + } + + if (Object.hasOwn(this.#kJSConfig, 'allowAutoTopicCreation')) { + globalConfig['allow.auto.create.topics'] = this.#kJSConfig.allowAutoTopicCreation; + } + + if (Object.hasOwn(this.#kJSConfig, 'maxBytesPerPartition')) { + globalConfig['max.partition.fetch.bytes'] = this.#kJSConfig.maxBytesPerPartition; + } + + if (Object.hasOwn(this.#kJSConfig, 'maxWaitTimeInMs')) { + globalConfig['fetch.wait.max.ms'] = this.#kJSConfig.maxWaitTimeInMs; + } + + if (Object.hasOwn(this.#kJSConfig, 'minBytes')) { + globalConfig['fetch.min.bytes'] = this.#kJSConfig.minBytes; + } + + if (Object.hasOwn(this.#kJSConfig, 'maxBytes')) { + globalConfig['fetch.message.max.bytes'] = this.#kJSConfig.maxBytes; + } + + if (Object.hasOwn(this.#kJSConfig, 'readUncommitted')) { + globalConfig['isolation.level'] = this.#kJSConfig.readUncommitted ? 'read_uncommitted' : 'read_committed'; + } + + if (Object.hasOwn(this.#kJSConfig, 'maxInFlightRequests')) { + globalConfig['max.in.flight'] = this.#kJSConfig.maxInFlightRequests; + } + + if (Object.hasOwn(this.#kJSConfig, 'rackId')) { + globalConfig['client.rack'] = this.#kJSConfig.rackId; + } + globalConfig['offset_commit_cb'] = true; if (this.#kJSConfig.rebalanceListener) { globalConfig['rebalance_cb'] = this.#rebalanceCallback.bind(this); @@ -260,7 +333,10 @@ class Consumer { return new Promise((resolve, reject) => { this.#connectPromiseFunc = { resolve, reject }; - this.#internalClient.connect(); + this.#internalClient.connect(null, (err) => { + if (err) + reject(createKafkaJsErrorFromLibRdKafkaError(err)); + }); }); } @@ -488,4 +564,4 @@ class Consumer { } } -module.exports = { Consumer } +module.exports = { Consumer, PartitionAssigners, } diff --git a/lib/kafkajs/_kafka.js b/lib/kafkajs/_kafka.js index 58e888bd..81ca34b8 100644 --- a/lib/kafkajs/_kafka.js +++ b/lib/kafkajs/_kafka.js @@ -1,6 +1,7 @@ const { Producer } = require('./_producer'); -const { Consumer } = require('./_consumer'); +const { Consumer, PartitionAssigners } = require('./_consumer'); const error = require('./_error'); +const { logLevel } = require('./_common'); class Kafka { #commonClientConfig = {}; @@ -10,30 +11,28 @@ class Kafka { * @param {import("../../types/kafkajs").KafkaConfig} config */ constructor(config) { - this.#commonClientConfig = config; + this.#commonClientConfig = config ?? {}; } /** * Merge the producer/consumer specific configuration with the common configuration. * @param {import("../../types/kafkajs").ProducerConfig|import("../../types/kafkajs").ConsumerConfig} config - * @returns + * @returns {(import("../../types/kafkajs").ProducerConfig & import("../../types/kafkajs").KafkaConfig) | (import("../../types/kafkajs").ConsumerConfig & import("../../types/kafkajs").KafkaConfig)} */ #mergeConfiguration(config) { - let baseConfig = Object.assign({}, this.#commonClientConfig); - config = Object.assign({}, config); - - // TODO: there's some confusion around this, as we currently allow - // rdKafka to be a function, but here, we don't seem to treat it as such. - // Correct this, so that only objects are allowed for `rdKafka`. - let rdKafka = baseConfig.rdKafka; - Object.assign(baseConfig, config); - if (typeof rdKafka === 'object' && typeof config.rdKafka === 'object') { - baseConfig.rdKafka = { - ...rdKafka, - ...config.rdKafka, - }; + config = config ?? {}; + const mergedConfig = Object.assign({}, this.#commonClientConfig); + + mergedConfig.rdKafka = mergedConfig.rdKafka ?? {}; + + if (typeof config.rdKafka === 'object') { + Object.assign(mergedConfig.rdKafka, config.rdKafka); + delete config.rdKafka; } - return baseConfig; + + Object.assign(mergedConfig, config); + + return mergedConfig; } /** @@ -55,4 +54,4 @@ class Kafka { } } -module.exports = { Kafka, ...error }; +module.exports = { Kafka, ...error, logLevel, PartitionAssigners, PartitionAssignors: PartitionAssigners }; diff --git a/lib/kafkajs/_producer.js b/lib/kafkajs/_producer.js index 8181cbf7..04239d5b 100644 --- a/lib/kafkajs/_producer.js +++ b/lib/kafkajs/_producer.js @@ -66,13 +66,40 @@ class Producer { } async #finalizedConfig() { + /* This sets the common configuration options for the client. */ const { globalConfig, topicConfig } = await kafkaJSToRdKafkaConfig(this.#kJSConfig); + + /* Create producer specific configuration. */ globalConfig.dr_cb = 'true'; + if (Object.hasOwn(this.#kJSConfig, 'metadataMaxAge')) { + globalConfig['topic.metadata.refresh.interval.ms'] = this.#kJSConfig.metadataMaxAge; + } + + if (Object.hasOwn(this.#kJSConfig, 'allowAutoTopicCreation')) { + globalConfig['allow.auto.create.topics'] = this.#kJSConfig.allowAutoTopicCreation; + } + + if (Object.hasOwn(this.#kJSConfig, 'transactionTimeout')) { + globalConfig['transaction.timeout.ms'] = this.#kJSConfig.transactionTimeout; + } + + if (Object.hasOwn(this.#kJSConfig, 'idempotent')) { + globalConfig['enable.idempotence'] = this.#kJSConfig.idempotent; + } + + if (Object.hasOwn(this.#kJSConfig, 'maxInFlightRequests')) { + globalConfig['max.in.flight'] = this.#kJSConfig.maxInFlightRequests; + } + if (Object.hasOwn(this.#kJSConfig, 'transactionalId')) { globalConfig['transactional.id'] = this.#kJSConfig.transactionalId; } + if (Object.hasOwn(this.#kJSConfig, 'createPartitioner')) { + throw new error.KafkaJSError("createPartitioner is not supported", { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); + } + return { globalConfig, topicConfig }; } @@ -201,7 +228,10 @@ class Producer { return new Promise((resolve, reject) => { this.#connectPromiseFunc = { resolve, reject }; - this.#internalClient.connect(); + this.#internalClient.connect(null, (err) => { + if (err) + reject(createKafkaJsErrorFromLibRdKafkaError(err)); + }); }); } diff --git a/lib/kafkajs/index.js b/lib/kafkajs/index.js index beac115f..5181979b 100644 --- a/lib/kafkajs/index.js +++ b/lib/kafkajs/index.js @@ -1,3 +1 @@ -const { Kafka } = require("./_kafka"); - -module.exports = { Kafka }; +module.exports = require("./_kafka"); diff --git a/types/kafkajs.d.ts b/types/kafkajs.d.ts index 37bbfd9e..1c15557e 100644 --- a/types/kafkajs.d.ts +++ b/types/kafkajs.d.ts @@ -29,7 +29,7 @@ export interface KafkaConfig { reauthenticationThreshold?: number requestTimeout?: number enforceRequestTimeout?: boolean - rdKafka?: Function | { topicConfig?: TopicConfig, globalConfig?: GlobalConfig }; + rdKafka?: { topicConfig?: TopicConfig, globalConfig?: GlobalConfig }; } export interface ProducerConfig { @@ -39,7 +39,7 @@ export interface ProducerConfig { transactionalId?: string transactionTimeout?: number maxInFlightRequests?: number - rdKafka?: Function | { topicConfig?: ProducerTopicConfig, globalConfig?: ProducerGlobalConfig } + rdKafka?: { topicConfig?: ProducerTopicConfig, globalConfig?: ProducerGlobalConfig } } export interface IHeaders { @@ -127,7 +127,7 @@ export interface ConsumerConfig { maxInFlightRequests?: number readUncommitted?: boolean rackId?: string - rdKafka?: Function | { topicConfig?: ConsumerTopicConfig, globalConfig?: ConsumerGlobalConfig } + rdKafka?: { topicConfig?: ConsumerTopicConfig, globalConfig?: ConsumerGlobalConfig } } export type ConsumerEvents = { From 7837ac4cc1eec7e308dabaa62bc02f40ae55fe4b Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 4 Dec 2023 18:19:57 +0530 Subject: [PATCH 07/26] Add experimental admin client and example --- MIGRATION.md | 8 ++ examples/kafkajs/admin.js | 38 ++++++++ lib/kafkajs/_admin.js | 195 ++++++++++++++++++++++++++++++++++++++ lib/kafkajs/_common.js | 2 +- lib/kafkajs/_kafka.js | 5 + types/kafkajs.d.ts | 21 ++++ 6 files changed, 268 insertions(+), 1 deletion(-) create mode 100644 examples/kafkajs/admin.js create mode 100644 lib/kafkajs/_admin.js diff --git a/MIGRATION.md b/MIGRATION.md index 08d1c418..84c9b482 100644 --- a/MIGRATION.md +++ b/MIGRATION.md @@ -290,4 +290,12 @@ * `paused()` is not (YET) supported. * Custom partition assignors are not supported. +### Admin Client + + * The admin-client is currently experimental, and only has support for a limited subset of methods. The API is subject to change. + The methods supported are: + * The `createTopics` method does not yet support the `validateOnly` or `waitForLeaders` properties, and the per-topic configuration + does not support `replicaAssignment`. + * The `deleteTopics` method is fully supported. + ## node-rdkafka diff --git a/examples/kafkajs/admin.js b/examples/kafkajs/admin.js new file mode 100644 index 00000000..83c20acf --- /dev/null +++ b/examples/kafkajs/admin.js @@ -0,0 +1,38 @@ +const { Kafka } = require('../..').KafkaJS +//const { Kafka } = require('kafkajs') + +async function adminStart() { + const kafka = new Kafka({ + brokers: [''], + }); + + const admin = kafka.admin(); + await admin.connect(); + + await admin.createTopics({ + topics: [ + { + topic: 'test-topic', + numPartitions: 3, + replicationFactor: 1, + } + ] + }).then(() => { + console.log("Topic created successfully"); + }).catch((err) => { + console.log("Topic creation failed", err); + }); + + await admin.deleteTopics({ + topics: ['test-topic'], + timeout: 5600, + }).then(() => { + console.log("Topic deleted successfully"); + }).catch((err) => { + console.log("Topic deletion failed", err); + }); + + await admin.disconnect(); +} + +adminStart(); diff --git a/lib/kafkajs/_admin.js b/lib/kafkajs/_admin.js new file mode 100644 index 00000000..5e39825d --- /dev/null +++ b/lib/kafkajs/_admin.js @@ -0,0 +1,195 @@ +const RdKafka = require('../rdkafka'); +const { kafkaJSToRdKafkaConfig, createKafkaJsErrorFromLibRdKafkaError } = require('./_common'); +const error = require('./_error'); + +/** + * NOTE: The Admin client is currently in an experimental state with many + * features missing or incomplete, and the API is subject to change. + */ + +const AdminState = Object.freeze({ + INIT: 0, + CONNECTING: 1, + CONNECTED: 4, + DISCONNECTING: 5, + DISCONNECTED: 6, +}); + +class Admin { + /** + * kJSConfig is the merged kafkaJS config object. + * @type {import("../../types/kafkajs").AdminConfig & import("../../types/kafkajs").KafkaConfig} + */ + #kJSConfig = null; + + /** + * rdKafkaConfig contains the config objects that will be passed to node-rdkafka. + * @type {{globalConfig: import("../../types/config").GlobalConfig}|null} + */ + #rdKafkaConfig = null; + + /** + * internalClient is the node-rdkafka client used by the API. + * @type {import("../rdkafka").AdminClient|null} + */ + #internalClient = null; + /** + * state is the current state of the admin client. + * @type {AdminState} + */ + #state = AdminState.INIT; + + /** + * @constructor + * @param {import("../../types/kafkajs").ProducerConfig} kJSConfig + */ + constructor(kJSConfig) { + this.#kJSConfig = kJSConfig; + } + + async #config() { + if (!this.#rdKafkaConfig) + this.#rdKafkaConfig = await this.#finalizedConfig(); + return this.#rdKafkaConfig; + } + + async #finalizedConfig() { + /* This sets the common configuration options for the client. */ + const { globalConfig } = await kafkaJSToRdKafkaConfig(this.#kJSConfig); + + return { globalConfig }; + } + + /** + * Set up the client and connect to the bootstrap brokers. + * @returns {Promise} Resolves when connection is complete, rejects on error. + */ + async connect() { + if (this.#state !== AdminState.INIT) { + throw new error.KafkaJSError("Connect has already been called elsewhere.", { code: error.ErrorCodes.ERR__STATE }); + } + + this.#state = AdminState.CONNECTING; + + const { globalConfig } = await this.#config(); + + return new Promise((resolve, reject) => { + try { + /* AdminClient creation is a synchronous operation for node-rdkafka */ + this.#internalClient = RdKafka.AdminClient.create(globalConfig); + this.#state = AdminState.CONNECTED; + resolve(); + } catch (err) { + reject(createKafkaJsErrorFromLibRdKafkaError(err)); + } + }); + } + + /** + * Disconnect from the brokers, clean-up and tear down the client. + * @returns {Promise} Resolves when disconnect is complete, rejects on error. + */ + async disconnect() { + if (this.#state >= AdminState.DISCONNECTING) { + return; + } + + this.#state = AdminState.DISCONNECTING; + return new Promise((resolve, reject) => { + try { + /* AdminClient disconnect for node-rdkakfa is synchronous. */ + this.#internalClient.disconnect(); + this.#state = AdminState.DISCONNECTED; + resolve(); + } catch (err) { + reject(createKafkaJsErrorFromLibRdKafkaError(err)); + } + }); + } + + + /** + * Converts a topic configuration object from kafkaJS to a format suitable for node-rdkafka. + * @param {import("../../types/kafkajs").ITopicConfig} topic + * @returns {import("../../index").NewTopic} + */ + #topicConfigToRdKafka(topic) { + let topicConfig = { topic: topic.topic }; + topicConfig.topic = topic.topic; + topicConfig.num_partitions = topic.numPartitions ?? -1; + topicConfig.replication_factor = topic.replicationFactor ?? -1; + + if (Object.hasOwn(topic, "replicaAssignment")) { + throw new error.KafkaJSError("replicaAssignment is not yet implemented.", { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); + } + + topicConfig.config = {}; + topic.configEntries = topic.configEntries ?? []; + for (const configEntry of topic.configEntries) { + topicConfig.config[configEntry.name] = configEntry.value; + } + + return topicConfig; + } + + /** + * Create topics with the given configuration. + * @param {{ validateOnly?: boolean, waitForLeaders?: boolean, timeout?: number, topics: import("../../types/kafkajs").ITopicConfig[] }} options + * @returns {Promise} Resolves when the topics are created, rejects on error. + */ + async createTopics(options) { + if (this.#state !== AdminState.CONNECTED) { + throw new error.KafkaJSError("Admin client is not connected.", { code: error.ErrorCodes.ERR__STATE }); + } + + if (Object.hasOwn(options, "validateOnly")) { + throw new error.KafkaJSError("validateOnly is not yet implemented.", { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); + } + + if (Object.hasOwn(options, "waitForLeaders")) { + throw new error.KafkaJSError("waitForLeaders is not yet implemented.", { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); + } + + /* Convert each topic to a format suitable for node-rdkafka, and dispatch the call. */ + const ret = + options.topics + .map(this.#topicConfigToRdKafka) + .map(topicConfig => new Promise((resolve, reject) => { + this.#internalClient.createTopic(topicConfig, options.timeout ?? 5000, (err) => { + if (err) { + reject(createKafkaJsErrorFromLibRdKafkaError(err)); + } else { + resolve(); + } + }); + })); + + return Promise.all(ret); + } + + /** + * Deletes given topics. + * @param {{topics: string[], timeout?: number}} options + * @returns {Promise} Resolves when the topics are deleted, rejects on error. + */ + async deleteTopics(options) { + if (this.#state !== AdminState.CONNECTED) { + throw new error.KafkaJSError("Admin client is not connected.", { code: error.ErrorCodes.ERR__STATE }); + } + + return Promise.all( + options.topics.map(topic => new Promise((resolve, reject) => { + this.#internalClient.deleteTopic(topic, options.timeout ?? 5000, err => { + if (err) { + reject(createKafkaJsErrorFromLibRdKafkaError(err)); + } else { + resolve(); + } + }); + })) + ); + } + +} + +module.exports = { Admin } diff --git a/lib/kafkajs/_common.js b/lib/kafkajs/_common.js index a0fa3392..7bdb59f4 100644 --- a/lib/kafkajs/_common.js +++ b/lib/kafkajs/_common.js @@ -201,7 +201,7 @@ function createKafkaJsErrorFromLibRdKafkaError(librdKafkaError) { } else if (properties.code === error.ErrorCodes.ERR__NOT_IMPLEMENTED) { err = new error.KafkaJSNotImplemented(librdKafkaError, properties); } else if (properties.code === error.ErrorCodes.ERR__TIMED_OUT) { - err = new error.KafkaJSTimedOut(librdKafkaError, properties); + err = new error.KafkaJSTimeout(librdKafkaError, properties); } else if (properties.code === error.ErrorCodes.ERR__ALL_BROKERS_DOWN) { err = new error.KafkaJSNoBrokerAvailableError(librdKafkaError, properties); } else if (properties.code === error.ErrorCodes.ERR__TRANSPORT) { diff --git a/lib/kafkajs/_kafka.js b/lib/kafkajs/_kafka.js index 81ca34b8..cc0f2ab2 100644 --- a/lib/kafkajs/_kafka.js +++ b/lib/kafkajs/_kafka.js @@ -1,5 +1,6 @@ const { Producer } = require('./_producer'); const { Consumer, PartitionAssigners } = require('./_consumer'); +const { Admin } = require('./_admin'); const error = require('./_error'); const { logLevel } = require('./_common'); @@ -52,6 +53,10 @@ class Kafka { consumer(config) { return new Consumer(this.#mergeConfiguration(config)); } + + admin(config) { + return new Admin(this.#mergeConfiguration(config)); + } } module.exports = { Kafka, ...error, logLevel, PartitionAssigners, PartitionAssignors: PartitionAssigners }; diff --git a/types/kafkajs.d.ts b/types/kafkajs.d.ts index 1c15557e..906ef267 100644 --- a/types/kafkajs.d.ts +++ b/types/kafkajs.d.ts @@ -149,6 +149,27 @@ export type ConsumerEvents = { REQUEST_QUEUE_SIZE: 'consumer.network.request_queue_size' } +export interface AdminConfig { + retry?: RetryOptions +} + +export interface ITopicConfig { + topic: string + numPartitions?: number + replicationFactor?: number + replicaAssignment?: ReplicaAssignment[] + configEntries?: IResourceConfigEntry[] +} + +export interface ReplicaAssignment { + partition: number + replicas: Array +} + +export interface IResourceConfigEntry { + name: string + value: string +} export enum logLevel { NOTHING = 0, From 8d2daeb8f92b110ed9ab63b7e2fc6b09d02de9c0 Mon Sep 17 00:00:00 2001 From: Milind L Date: Sat, 9 Dec 2023 11:14:46 +0530 Subject: [PATCH 08/26] Allow topic config properties to be set with the global config [c++] --- index.d.ts | 4 ++-- lib/client.js | 1 - src/kafka-consumer.cc | 20 ++++++++++---------- src/producer.cc | 25 +++++++++++++------------ 4 files changed, 25 insertions(+), 25 deletions(-) diff --git a/index.d.ts b/index.d.ts index bfbbdcbe..e361e11c 100644 --- a/index.d.ts +++ b/index.d.ts @@ -203,7 +203,7 @@ export abstract class Client extends EventEmitter { } export class KafkaConsumer extends Client { - constructor(conf: ConsumerGlobalConfig, topicConf: ConsumerTopicConfig); + constructor(conf: ConsumerGlobalConfig | ConsumerTopicConfig, topicConf?: ConsumerTopicConfig); assign(assignments: Assignment[]): this; @@ -256,7 +256,7 @@ export class KafkaConsumer extends Client { } export class Producer extends Client { - constructor(conf: ProducerGlobalConfig, topicConf?: ProducerTopicConfig); + constructor(conf: ProducerGlobalConfig | ProducerTopicConfig, topicConf?: ProducerTopicConfig); flush(timeout?: NumberNullUndefined, cb?: (err: LibrdKafkaError) => void): this; diff --git a/lib/client.js b/lib/client.js index 9ba38b69..89464845 100644 --- a/lib/client.js +++ b/lib/client.js @@ -45,7 +45,6 @@ function Client(globalConf, SubClientType, topicConf) { // the producer and consumer main wrappers var no_event_cb = globalConf.event_cb === false; - topicConf = topicConf || {}; // delete this because librdkafka will complain since this particular // key is a real conf value diff --git a/src/kafka-consumer.cc b/src/kafka-consumer.cc index eccab3e9..5592d7fd 100644 --- a/src/kafka-consumer.cc +++ b/src/kafka-consumer.cc @@ -31,7 +31,8 @@ KafkaConsumer::KafkaConsumer(Conf* gconfig, Conf* tconfig): Connection(gconfig, tconfig) { std::string errstr; - m_gconfig->set("default_topic_conf", m_tconfig, errstr); + if (m_tconfig) + m_gconfig->set("default_topic_conf", m_tconfig, errstr); m_consume_loop = nullptr; } @@ -553,10 +554,6 @@ void KafkaConsumer::New(const Nan::FunctionCallbackInfo& info) { return Nan::ThrowError("Global configuration data must be specified"); } - if (!info[1]->IsObject()) { - return Nan::ThrowError("Topic configuration must be specified"); - } - std::string errstr; Conf* gconfig = @@ -567,13 +564,16 @@ void KafkaConsumer::New(const Nan::FunctionCallbackInfo& info) { return Nan::ThrowError(errstr.c_str()); } - Conf* tconfig = - Conf::create(RdKafka::Conf::CONF_TOPIC, + // If tconfig isn't set, then just let us pick properties from gconf. + Conf* tconfig = nullptr; + if (info[1]->IsObject()) { + tconfig = Conf::create(RdKafka::Conf::CONF_TOPIC, (info[1]->ToObject(Nan::GetCurrentContext())).ToLocalChecked(), errstr); - if (!tconfig) { - delete gconfig; - return Nan::ThrowError(errstr.c_str()); + if (!tconfig) { + delete gconfig; + return Nan::ThrowError(errstr.c_str()); + } } KafkaConsumer* consumer = new KafkaConsumer(gconfig, tconfig); diff --git a/src/producer.cc b/src/producer.cc index f5a32b1b..f4e9f56d 100644 --- a/src/producer.cc +++ b/src/producer.cc @@ -35,7 +35,9 @@ Producer::Producer(Conf* gconfig, Conf* tconfig): m_partitioner_cb() { std::string errstr; - m_gconfig->set("default_topic_conf", m_tconfig, errstr); + if (m_tconfig) + m_gconfig->set("default_topic_conf", m_tconfig, errstr); + m_gconfig->set("dr_cb", &m_dr_cb, errstr); } @@ -110,10 +112,6 @@ void Producer::New(const Nan::FunctionCallbackInfo& info) { return Nan::ThrowError("Global configuration data must be specified"); } - if (!info[1]->IsObject()) { - return Nan::ThrowError("Topic configuration must be specified"); - } - std::string errstr; Conf* gconfig = @@ -124,14 +122,17 @@ void Producer::New(const Nan::FunctionCallbackInfo& info) { return Nan::ThrowError(errstr.c_str()); } - Conf* tconfig = - Conf::create(RdKafka::Conf::CONF_TOPIC, - (info[1]->ToObject(Nan::GetCurrentContext())).ToLocalChecked(), errstr); + // If tconfig isn't set, then just let us pick properties from gconf. + Conf* tconfig = nullptr; + if (info[1]->IsObject()) { + tconfig = Conf::create(RdKafka::Conf::CONF_TOPIC, + (info[1]->ToObject(Nan::GetCurrentContext())).ToLocalChecked(), errstr); - if (!tconfig) { - // No longer need this since we aren't instantiating anything - delete gconfig; - return Nan::ThrowError(errstr.c_str()); + if (!tconfig) { + // No longer need this since we aren't instantiating anything + delete gconfig; + return Nan::ThrowError(errstr.c_str()); + } } Producer* producer = new Producer(gconfig, tconfig); From d1c30f40810a13598d519a4158e4c581283a31cc Mon Sep 17 00:00:00 2001 From: Milind L Date: Sat, 9 Dec 2023 17:13:03 +0530 Subject: [PATCH 09/26] Add first test for promisified API: consumer. Adds test helpers and jest. --- MIGRATION.md | 8 +- Makefile | 6 +- lib/kafkajs/_admin.js | 6 + lib/kafkajs/_consumer.js | 54 +- lib/kafkajs/_producer.js | 47 + package-lock.json | 4768 +++++++++++++++---- package.json | 1 + test/promisified/consumer/subscribe.spec.js | 157 + test/promisified/testhelpers.js | 61 + types/kafkajs.d.ts | 14 + 10 files changed, 4068 insertions(+), 1054 deletions(-) create mode 100644 test/promisified/consumer/subscribe.spec.js create mode 100644 test/promisified/testhelpers.js diff --git a/MIGRATION.md b/MIGRATION.md index 84c9b482..e3e70c79 100644 --- a/MIGRATION.md +++ b/MIGRATION.md @@ -139,7 +139,6 @@ #### Semantic and Per-Method Changes -* `sendBatch` is not supported (YET). However, the actual batching semantics are handled by librdkafka. * Changes to `send`: * `acks`, `compression` and `timeout` are not set on a per-send basis. Rather, they must be configured in the configuration. Before: @@ -178,6 +177,7 @@ ``` * Error-handling for a failed `send` is stricter. While sending multiple messages, even if one of the messages fails, the method throws an error. +* `sendBatch` is supported. However, the actual batching semantics are handled by librdkafka, and it just acts as a wrapper around `send` (See `send` for changes). ### Consumer @@ -219,7 +219,10 @@ #### Semantic and Per-Method Changes - * While passing a list of topics to `subscribe`, the `fromBeginning` property is not supported. Instead, the property `auto.offset.reset` needs to be used. + * Changes to subscribe: + * Regex flags are ignored while passing a topic subscription (like 'i' or 'g'). + * Subscribe must be called after `connect`. + * While passing a list of topics to `subscribe`, the `fromBeginning` property is not supported. Instead, the property `auto.offset.reset` needs to be used. Before: ```javascript const kafka = new Kafka({ /* ... */ }); @@ -229,7 +232,6 @@ await consumer.connect(); await consumer.subscribe({ topics: ["topic"], fromBeginning: true}); ``` - After: ```javascript const kafka = new Kafka({ /* ... */ }); diff --git a/Makefile b/Makefile index 0b11d2e8..666d4832 100644 --- a/Makefile +++ b/Makefile @@ -13,7 +13,8 @@ endif NODE ?= node CPPLINT ?= cpplint.py BUILDTYPE ?= Release -TESTS = "test/**/*.js" +TESTS = $(ls test/producer/*.js test/*.js test/tools/*.js) +PROMISIFIED_TESTS = "test/promisified" E2E_TESTS = $(wildcard e2e/*.spec.js) TEST_REPORTER = TEST_OUTPUT = @@ -24,7 +25,7 @@ CONFIG_OUTPUTS = \ CPPLINT_FILES = $(wildcard src/*.cc src/*.h) CPPLINT_FILTER = -legal/copyright -JSLINT_FILES = lib/*.js test/*.js e2e/*.js +JSLINT_FILES = lib/*.js test/*.js e2e/*.js lib/kafkajs/*.js PACKAGE = $(shell node -pe 'require("./package.json").name.split("/")[1]') VERSION = $(shell node -pe 'require("./package.json").version') @@ -58,6 +59,7 @@ $(CONFIG_OUTPUTS): node_modules/.dirstamp binding.gyp test: node_modules/.dirstamp @./node_modules/.bin/mocha --ui exports $(TEST_REPORTER) $(TESTS) $(TEST_OUTPUT) + @./node_modules/.bin/jest --ci --runInBand $(PROMISIFIED_TESTS) check: node_modules/.dirstamp @$(NODE) util/test-compile.js diff --git a/lib/kafkajs/_admin.js b/lib/kafkajs/_admin.js index 5e39825d..5e86ba75 100644 --- a/lib/kafkajs/_admin.js +++ b/lib/kafkajs/_admin.js @@ -90,6 +90,12 @@ class Admin { * @returns {Promise} Resolves when disconnect is complete, rejects on error. */ async disconnect() { + /* Not yet connected - no error. */ + if (this.#state == AdminState.INIT) { + return; + } + + /* Already disconnecting, or disconnected. */ if (this.#state >= AdminState.DISCONNECTING) { return; } diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index df94503a..410a8e4f 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -342,16 +342,48 @@ class Consumer { /** * Subscribes the consumer to the given topics. - * @param {import("../../types/kafkajs").ConsumerSubscribeTopics} subscription + * @param {import("../../types/kafkajs").ConsumerSubscribeTopics | import("../../types/kafkajs").ConsumerSubscribeTopic} subscription */ async subscribe(subscription) { - if (typeof subscription.fromBeginning == 'boolean') { + if (this.#state !== ConsumerState.CONNECTED) { + throw new error.KafkaJSError('Subscribe can only be called while connected.', { code: error.ErrorCodes.ERR__STATE }); + } + + if (typeof subscription.fromBeginning === 'boolean') { throw new error.KafkaJSError( - 'fromBeginning is not supported by subscribe(), but must be passed as rdKafka properties to the consumer', - { code: error.ErrorCodes.ERR__INVALID_ARG }); + 'fromBeginning is not supported by subscribe(), but must be passed as an rdKafka property to the consumer.', + { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); } - this.#internalClient.subscribe(subscription.topics); + if (!Object.hasOwn(subscription, 'topics') && !Object.hasOwn(subscription, 'topic')) { + throw new error.KafkaJSError('Either topics or topic must be specified.', { code: error.ErrorCodes.ERR__INVALID_ARG }); + } + + let topics = []; + if (subscription.topic) { + topics.push(subscription.topic); + } else if (Array.isArray(subscription.topics)) { + topics = subscription.topics; + } else { + throw new error.KafkaJSError('topics must be an object of the type ConsumerSubscribeTopics.', { code: error.ErrorCodes.ERR__INVALID_ARG }); + } + + topics = topics.map(topic => { + if (typeof topic === 'string') { + return topic; + } else if (topic instanceof RegExp) { + // Flags are not supported, and librdkafka only considers a regex match if the first character of the regex is ^. + const regexSource = topic.source; + if (regexSource.charAt(0) !== '^') + return '^' + regexSource; + else + return regexSource; + } else { + throw new error.KafkaJSError('Invalid topic ' + topic + ' (' + typeof topic + '), the topic name has to be a String or a RegExp', { code: error.ErrorCodes.ERR__INVALID_ARG }); + } + }); + + this.#internalClient.subscribe(topics); } async stop() { @@ -541,10 +573,18 @@ class Consumer { * @returns {Promise} a promise that resolves when the consumer has disconnected. */ async disconnect() { - if (this.#state === ConsumerState.INIT) { - throw new error.KafkaJSError('Disconnect can only be called once consumer is connected.', { code: error.ErrorCodes.ERR__STATE }); + /* Not yet connected - no error. */ + if (this.#state == ConsumerState.INIT) { + return; } + /* TODO: We should handle a case where we are connecting, we should + * await the connection and then schedule a disconnect. */ + + /* Already disconnecting, or disconnected. */ + if (this.#state >= ConsumerState.DISCONNECTING) { + return; + } if (this.#state >= ConsumerState.DISCONNECTING) { return; } diff --git a/lib/kafkajs/_producer.js b/lib/kafkajs/_producer.js index 04239d5b..cf5ac62d 100644 --- a/lib/kafkajs/_producer.js +++ b/lib/kafkajs/_producer.js @@ -240,6 +240,15 @@ class Producer { * @returns {Promise} Resolves when disconnect is complete, rejects on error. */ async disconnect() { + /* Not yet connected - no error. */ + if (this.#state == ProducerState.INIT) { + return; + } + + /* TODO: We should handle a case where we are connecting, we should + * await the connection and then schedule a disconnect. */ + + /* Already disconnecting, or disconnected. */ if (this.#state >= ProducerState.DISCONNECTING) { return; } @@ -477,6 +486,44 @@ class Producer { } return ret; } + + /** + * sendBatch(batch: ProducerBatch): Promise + * @param {import('../../types/kafkajs').ProducerBatch} sendOptions - The record to send. The keys `acks`, `timeout`, and `compression` are not used, and should not be set, rather, they should be set in the global config. + * @returns {Promise} Resolves with the record metadata for the messages. + */ + async sendBatch(sendOptions) { + if (this.#state !== ProducerState.CONNECTED) { + throw new error.KafkaJSError("Cannot send without awaiting connect()", { code: error.ErrorCodes.ERR__STATE }); + } + + if (sendOptions === null || !(sendOptions instanceof Object)) { + throw new error.KafkaJSError("sendOptions must be set correctly", { code: error.ErrorCodes.ERR__INVALID_ARG }); + } + + // Ignore all properties except topic and messages. + if (Object.hasOwn(sendOptions, "acks") || Object.hasOwn(sendOptions, "timeout") || Object.hasOwn(sendOptions, "compression")) { + throw new error.KafkaJSError("sendOptions must not contain acks, timeout, or compression", { code: error.ErrorCodes.ERR__INVALID_ARG }); + } + + if (sendOptions.topicMessages !== null && !Array.isArray(sendOptions.topicMessages)) { + throw new error.KafkaJSError("sendOptions.topicMessages must be an array if set", { code: error.ErrorCodes.ERR__INVALID_ARG }); + } + + if (!sendOptions.topicMessages || sendOptions.topicMessages.length === 0) { + return Promise.resolve([]); + } + + // Internally, we just use send() because the batching is handled by librdkafka. + const sentPromises = []; + + for (const topicMessage of sendOptions.topicMessages) { + sentPromises.push(this.send(topicMessage)); + } + + const records = await Promise.all(sentPromises); + return records.flat(); + } } module.exports = { Producer } diff --git a/package-lock.json b/package-lock.json index 90fc9bdb..53717a2d 100644 --- a/package-lock.json +++ b/package-lock.json @@ -17,6 +17,7 @@ "devDependencies": { "@types/node": "^20.4.5", "bluebird": "^3.5.3", + "jest": "^29.7.0", "jsdoc": "^3.4.0", "jshint": "^2.10.1", "mocha": "^10.2.0", @@ -28,712 +29,2045 @@ "node": ">=14.0.0" } }, - "node_modules/@babel/parser": { - "version": "7.23.0", - "resolved": "https://registry.npmjs.org/@babel/parser/-/parser-7.23.0.tgz", - "integrity": "sha512-vvPKKdMemU85V9WE/l5wZEmImpCtLqbnTvqDS2U1fJ96KrxoW7KrXhNsNCblQlg8Ck4b85yxdTyelsMUgFUXiw==", + "node_modules/@ampproject/remapping": { + "version": "2.2.1", + "resolved": "https://registry.npmjs.org/@ampproject/remapping/-/remapping-2.2.1.tgz", + "integrity": "sha512-lFMjJTrFL3j7L9yBxwYfCq2k6qqwHyzuUl/XBnif78PWTJYyL/dfowQHWE3sp6U6ZzqWiiIZnpTMO96zhkjwtg==", "dev": true, - "bin": { - "parser": "bin/babel-parser.js" + "dependencies": { + "@jridgewell/gen-mapping": "^0.3.0", + "@jridgewell/trace-mapping": "^0.3.9" }, "engines": { "node": ">=6.0.0" } }, - "node_modules/@isaacs/cliui": { - "version": "8.0.2", - "resolved": "https://registry.npmjs.org/@isaacs/cliui/-/cliui-8.0.2.tgz", - "integrity": "sha512-O8jcjabXaleOG9DQ0+ARXWZBTfnP4WNAqzuiJK7ll44AmxGKv/J2M4TPjxjY3znBCfvBXFzucm1twdyFybFqEA==", + "node_modules/@babel/code-frame": { + "version": "7.23.5", + "resolved": "https://registry.npmjs.org/@babel/code-frame/-/code-frame-7.23.5.tgz", + "integrity": "sha512-CgH3s1a96LipHCmSUmYFPwY7MNx8C3avkq7i4Wl3cfa662ldtUe4VM1TPXX70pfmrlWTb6jLqTYrZyT2ZTJBgA==", "dev": true, "dependencies": { - "string-width": "^5.1.2", - "string-width-cjs": "npm:string-width@^4.2.0", - "strip-ansi": "^7.0.1", - "strip-ansi-cjs": "npm:strip-ansi@^6.0.1", - "wrap-ansi": "^8.1.0", - "wrap-ansi-cjs": "npm:wrap-ansi@^7.0.0" + "@babel/highlight": "^7.23.4", + "chalk": "^2.4.2" }, "engines": { - "node": ">=12" + "node": ">=6.9.0" } }, - "node_modules/@isaacs/cliui/node_modules/ansi-regex": { - "version": "6.0.1", - "resolved": "https://registry.npmjs.org/ansi-regex/-/ansi-regex-6.0.1.tgz", - "integrity": "sha512-n5M855fKb2SsfMIiFFoVrABHJC8QtHwVx+mHWP3QcEqBHYienj5dHSgjbxtC0WEZXYt4wcD6zrQElDPhFuZgfA==", + "node_modules/@babel/code-frame/node_modules/ansi-styles": { + "version": "3.2.1", + "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-3.2.1.tgz", + "integrity": "sha512-VT0ZI6kZRdTh8YyJw3SMbYm/u+NqfsAxEpWO0Pf9sq8/e94WxxOpPKx9FR1FlyCtOVDNOQ+8ntlqFxiRc+r5qA==", "dev": true, - "engines": { - "node": ">=12" + "dependencies": { + "color-convert": "^1.9.0" }, - "funding": { - "url": "https://github.com/chalk/ansi-regex?sponsor=1" + "engines": { + "node": ">=4" } }, - "node_modules/@isaacs/cliui/node_modules/ansi-styles": { - "version": "6.2.1", - "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-6.2.1.tgz", - "integrity": "sha512-bN798gFfQX+viw3R7yrGWRqnrN2oRkEkUjjl4JNn4E8GxxbjtG3FbrEIIY3l8/hrwUwIeCZvi4QuOTP4MErVug==", + "node_modules/@babel/code-frame/node_modules/chalk": { + "version": "2.4.2", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.4.2.tgz", + "integrity": "sha512-Mti+f9lpJNcwF4tWV8/OrTTtF1gZi+f8FqlyAdouralcFWFQWF2+NgCHShjkCb+IFBLq9buZwE1xckQU4peSuQ==", "dev": true, - "engines": { - "node": ">=12" + "dependencies": { + "ansi-styles": "^3.2.1", + "escape-string-regexp": "^1.0.5", + "supports-color": "^5.3.0" }, - "funding": { - "url": "https://github.com/chalk/ansi-styles?sponsor=1" + "engines": { + "node": ">=4" + } + }, + "node_modules/@babel/code-frame/node_modules/color-convert": { + "version": "1.9.3", + "resolved": "https://registry.npmjs.org/color-convert/-/color-convert-1.9.3.tgz", + "integrity": "sha512-QfAUtd+vFdAtFQcC8CCyYt1fYWxSqAiK2cSD6zDB8N3cpsEBAvRxp9zOGg6G/SHHJYAT88/az/IuDGALsNVbGg==", + "dev": true, + "dependencies": { + "color-name": "1.1.3" } }, - "node_modules/@isaacs/cliui/node_modules/emoji-regex": { - "version": "9.2.2", - "resolved": "https://registry.npmjs.org/emoji-regex/-/emoji-regex-9.2.2.tgz", - "integrity": "sha512-L18DaJsXSUk2+42pv8mLs5jJT2hqFkFE4j21wOmgbUqsZ2hL72NsUU785g9RXgo3s0ZNgVl42TiHp3ZtOv/Vyg==", + "node_modules/@babel/code-frame/node_modules/color-name": { + "version": "1.1.3", + "resolved": "https://registry.npmjs.org/color-name/-/color-name-1.1.3.tgz", + "integrity": "sha512-72fSenhMw2HZMTVHeCA9KCmpEIbzWiQsjN+BHcBbS9vr1mtt+vJjPdksIBNUmKAW8TFUDPJK5SUU3QhE9NEXDw==", "dev": true }, - "node_modules/@isaacs/cliui/node_modules/string-width": { - "version": "5.1.2", - "resolved": "https://registry.npmjs.org/string-width/-/string-width-5.1.2.tgz", - "integrity": "sha512-HnLOCR3vjcY8beoNLtcjZ5/nxn2afmME6lhrDrebokqMap+XbeW8n9TXpPDOqdGK5qcI3oT0GKTW6wC7EMiVqA==", + "node_modules/@babel/code-frame/node_modules/escape-string-regexp": { + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/escape-string-regexp/-/escape-string-regexp-1.0.5.tgz", + "integrity": "sha512-vbRorB5FUQWvla16U8R/qgaFIya2qGzwDrNmCZuYKrbdSUMG6I1ZCGQRefkRVhuOkIGVne7BQ35DSfo1qvJqFg==", "dev": true, - "dependencies": { - "eastasianwidth": "^0.2.0", - "emoji-regex": "^9.2.2", - "strip-ansi": "^7.0.1" - }, "engines": { - "node": ">=12" - }, - "funding": { - "url": "https://github.com/sponsors/sindresorhus" + "node": ">=0.8.0" + } + }, + "node_modules/@babel/code-frame/node_modules/has-flag": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/has-flag/-/has-flag-3.0.0.tgz", + "integrity": "sha512-sKJf1+ceQBr4SMkvQnBDNDtf4TXpVhVGateu0t918bl30FnbE2m4vNLX+VWe/dpjlb+HugGYzW7uQXH98HPEYw==", + "dev": true, + "engines": { + "node": ">=4" } }, - "node_modules/@isaacs/cliui/node_modules/strip-ansi": { - "version": "7.1.0", - "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-7.1.0.tgz", - "integrity": "sha512-iq6eVVI64nQQTRYq2KtEg2d2uU7LElhTJwsH4YzIHZshxlgZms/wIc4VoDQTlG/IvVIrBKG06CrZnp0qv7hkcQ==", + "node_modules/@babel/code-frame/node_modules/supports-color": { + "version": "5.5.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.5.0.tgz", + "integrity": "sha512-QjVjwdXIt408MIiAqCX4oUKsgU2EqAGzs2Ppkm4aQYbjm+ZEWEcW4SfFNTr4uMNZma0ey4f5lgLrkB0aX0QMow==", "dev": true, "dependencies": { - "ansi-regex": "^6.0.1" + "has-flag": "^3.0.0" }, "engines": { - "node": ">=12" - }, - "funding": { - "url": "https://github.com/chalk/strip-ansi?sponsor=1" + "node": ">=4" } }, - "node_modules/@isaacs/cliui/node_modules/wrap-ansi": { - "version": "8.1.0", - "resolved": "https://registry.npmjs.org/wrap-ansi/-/wrap-ansi-8.1.0.tgz", - "integrity": "sha512-si7QWI6zUMq56bESFvagtmzMdGOtoxfR+Sez11Mobfc7tm+VkUckk9bW2UeffTGVUbOksxmSw0AA2gs8g71NCQ==", + "node_modules/@babel/compat-data": { + "version": "7.23.5", + "resolved": "https://registry.npmjs.org/@babel/compat-data/-/compat-data-7.23.5.tgz", + "integrity": "sha512-uU27kfDRlhfKl+w1U6vp16IuvSLtjAxdArVXPa9BvLkrr7CYIsxH5adpHObeAGY/41+syctUWOZ140a2Rvkgjw==", + "dev": true, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/core": { + "version": "7.23.5", + "resolved": "https://registry.npmjs.org/@babel/core/-/core-7.23.5.tgz", + "integrity": "sha512-Cwc2XjUrG4ilcfOw4wBAK+enbdgwAcAJCfGUItPBKR7Mjw4aEfAFYrLxeRp4jWgtNIKn3n2AlBOfwwafl+42/g==", "dev": true, "dependencies": { - "ansi-styles": "^6.1.0", - "string-width": "^5.0.1", - "strip-ansi": "^7.0.1" + "@ampproject/remapping": "^2.2.0", + "@babel/code-frame": "^7.23.5", + "@babel/generator": "^7.23.5", + "@babel/helper-compilation-targets": "^7.22.15", + "@babel/helper-module-transforms": "^7.23.3", + "@babel/helpers": "^7.23.5", + "@babel/parser": "^7.23.5", + "@babel/template": "^7.22.15", + "@babel/traverse": "^7.23.5", + "@babel/types": "^7.23.5", + "convert-source-map": "^2.0.0", + "debug": "^4.1.0", + "gensync": "^1.0.0-beta.2", + "json5": "^2.2.3", + "semver": "^6.3.1" }, "engines": { - "node": ">=12" + "node": ">=6.9.0" }, "funding": { - "url": "https://github.com/chalk/wrap-ansi?sponsor=1" + "type": "opencollective", + "url": "https://opencollective.com/babel" } }, - "node_modules/@mapbox/node-pre-gyp": { - "version": "1.0.11", - "resolved": "https://registry.npmjs.org/@mapbox/node-pre-gyp/-/node-pre-gyp-1.0.11.tgz", - "integrity": "sha512-Yhlar6v9WQgUp/He7BdgzOz8lqMQ8sU+jkCq7Wx8Myc5YFJLbEe7lgui/V7G1qB1DJykHSGwreceSaD60Y0PUQ==", - "dependencies": { - "detect-libc": "^2.0.0", - "https-proxy-agent": "^5.0.0", - "make-dir": "^3.1.0", - "node-fetch": "^2.6.7", - "nopt": "^5.0.0", - "npmlog": "^5.0.1", - "rimraf": "^3.0.2", - "semver": "^7.3.5", - "tar": "^6.1.11" - }, + "node_modules/@babel/core/node_modules/semver": { + "version": "6.3.1", + "resolved": "https://registry.npmjs.org/semver/-/semver-6.3.1.tgz", + "integrity": "sha512-BR7VvDCVHO+q2xBEWskxS6DJE1qRnb7DxzUrogb71CWoSficBxYsiAGd+Kl0mmq/MprG9yArRkyrQxTO6XjMzA==", + "dev": true, "bin": { - "node-pre-gyp": "bin/node-pre-gyp" + "semver": "bin/semver.js" } }, - "node_modules/@npmcli/fs": { - "version": "3.1.0", - "resolved": "https://registry.npmjs.org/@npmcli/fs/-/fs-3.1.0.tgz", - "integrity": "sha512-7kZUAaLscfgbwBQRbvdMYaZOWyMEcPTH/tJjnyAWJ/dvvs9Ef+CERx/qJb9GExJpl1qipaDGn7KqHnFGGixd0w==", + "node_modules/@babel/generator": { + "version": "7.23.5", + "resolved": "https://registry.npmjs.org/@babel/generator/-/generator-7.23.5.tgz", + "integrity": "sha512-BPssCHrBD+0YrxviOa3QzpqwhNIXKEtOa2jQrm4FlmkC2apYgRnQcmPWiGZDlGxiNtltnUFolMe8497Esry+jA==", "dev": true, "dependencies": { - "semver": "^7.3.5" + "@babel/types": "^7.23.5", + "@jridgewell/gen-mapping": "^0.3.2", + "@jridgewell/trace-mapping": "^0.3.17", + "jsesc": "^2.5.1" }, "engines": { - "node": "^14.17.0 || ^16.13.0 || >=18.0.0" + "node": ">=6.9.0" } }, - "node_modules/@pkgjs/parseargs": { - "version": "0.11.0", - "resolved": "https://registry.npmjs.org/@pkgjs/parseargs/-/parseargs-0.11.0.tgz", - "integrity": "sha512-+1VkjdD0QBLPodGrJUeqarH8VAIvQODIbwh9XpP5Syisf7YoQgsJKPNFoqqLQlu+VQ/tVSshMR6loPMn8U+dPg==", + "node_modules/@babel/helper-compilation-targets": { + "version": "7.22.15", + "resolved": "https://registry.npmjs.org/@babel/helper-compilation-targets/-/helper-compilation-targets-7.22.15.tgz", + "integrity": "sha512-y6EEzULok0Qvz8yyLkCvVX+02ic+By2UdOhylwUOvOn9dvYc9mKICJuuU1n1XBI02YWsNsnrY1kc6DVbjcXbtw==", "dev": true, - "optional": true, + "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" + }, "engines": { - "node": ">=14" + "node": ">=6.9.0" } }, - "node_modules/@tootallnate/once": { - "version": "2.0.0", - "resolved": "https://registry.npmjs.org/@tootallnate/once/-/once-2.0.0.tgz", - "integrity": "sha512-XCuKFP5PS55gnMVu3dty8KPatLqUoy/ZYzDzAGCQ8JNFCkLXzmI7vNHCR+XpbZaMWQK/vQubr7PkYq8g470J/A==", + "node_modules/@babel/helper-compilation-targets/node_modules/semver": { + "version": "6.3.1", + "resolved": "https://registry.npmjs.org/semver/-/semver-6.3.1.tgz", + "integrity": "sha512-BR7VvDCVHO+q2xBEWskxS6DJE1qRnb7DxzUrogb71CWoSficBxYsiAGd+Kl0mmq/MprG9yArRkyrQxTO6XjMzA==", "dev": true, - "engines": { - "node": ">= 10" + "bin": { + "semver": "bin/semver.js" } }, - "node_modules/@types/linkify-it": { - "version": "3.0.4", - "resolved": "https://registry.npmjs.org/@types/linkify-it/-/linkify-it-3.0.4.tgz", - "integrity": "sha512-hPpIeeHb/2UuCw06kSNAOVWgehBLXEo0/fUs0mw3W2qhqX89PI2yvok83MnuctYGCPrabGIoi0fFso4DQ+sNUQ==", - "dev": true + "node_modules/@babel/helper-environment-visitor": { + "version": "7.22.20", + "resolved": "https://registry.npmjs.org/@babel/helper-environment-visitor/-/helper-environment-visitor-7.22.20.tgz", + "integrity": "sha512-zfedSIzFhat/gFhWfHtgWvlec0nqB9YEIVrpuwjruLlXfUSnA8cJB0miHKwqDnQ7d32aKo2xt88/xZptwxbfhA==", + "dev": true, + "engines": { + "node": ">=6.9.0" + } }, - "node_modules/@types/markdown-it": { - "version": "12.2.3", - "resolved": "https://registry.npmjs.org/@types/markdown-it/-/markdown-it-12.2.3.tgz", - "integrity": "sha512-GKMHFfv3458yYy+v/N8gjufHO6MSZKCOXpZc5GXIWWy8uldwfmPn98vp81gZ5f9SVw8YYBctgfJ22a2d7AOMeQ==", + "node_modules/@babel/helper-function-name": { + "version": "7.23.0", + "resolved": "https://registry.npmjs.org/@babel/helper-function-name/-/helper-function-name-7.23.0.tgz", + "integrity": "sha512-OErEqsrxjZTJciZ4Oo+eoZqeW9UIiOcuYKRJA4ZAgV9myA+pOXhhmpfNCKjEH/auVfEYVFJ6y1Tc4r0eIApqiw==", "dev": true, "dependencies": { - "@types/linkify-it": "*", - "@types/mdurl": "*" + "@babel/template": "^7.22.15", + "@babel/types": "^7.23.0" + }, + "engines": { + "node": ">=6.9.0" } }, - "node_modules/@types/mdurl": { - "version": "1.0.4", - "resolved": "https://registry.npmjs.org/@types/mdurl/-/mdurl-1.0.4.tgz", - "integrity": "sha512-ARVxjAEX5TARFRzpDRVC6cEk0hUIXCCwaMhz8y7S1/PxU6zZS1UMjyobz7q4w/D/R552r4++EhwmXK1N2rAy0A==", - "dev": true - }, - "node_modules/@types/node": { - "version": "20.8.10", - "resolved": "https://registry.npmjs.org/@types/node/-/node-20.8.10.tgz", - "integrity": "sha512-TlgT8JntpcbmKUFzjhsyhGfP2fsiz1Mv56im6enJ905xG1DAYesxJaeSbGqQmAw8OWPdhyJGhGSQGKRNJ45u9w==", + "node_modules/@babel/helper-hoist-variables": { + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-hoist-variables/-/helper-hoist-variables-7.22.5.tgz", + "integrity": "sha512-wGjk9QZVzvknA6yKIUURb8zY3grXCcOZt+/7Wcy8O2uctxhplmUPkOdlgoNhmdVee2c92JXbf1xpMtVNbfoxRw==", "dev": true, "dependencies": { - "undici-types": "~5.26.4" + "@babel/types": "^7.22.5" + }, + "engines": { + "node": ">=6.9.0" } }, - "node_modules/abbrev": { - "version": "1.1.1", - "resolved": "https://registry.npmjs.org/abbrev/-/abbrev-1.1.1.tgz", - "integrity": "sha512-nne9/IiQ/hzIhY6pdDnbBtz7DjPTKrY00P/zvPSm5pOFkl6xuGrGnXn/VtTNNfNtAfZ9/1RtehkszU9qcTii0Q==" + "node_modules/@babel/helper-module-imports": { + "version": "7.22.15", + "resolved": "https://registry.npmjs.org/@babel/helper-module-imports/-/helper-module-imports-7.22.15.tgz", + "integrity": "sha512-0pYVBnDKZO2fnSPCrgM/6WMc7eS20Fbok+0r88fp+YtWVLZrp4CkafFGIp+W0VKw4a22sgebPT99y+FDNMdP4w==", + "dev": true, + "dependencies": { + "@babel/types": "^7.22.15" + }, + "engines": { + "node": ">=6.9.0" + } }, - "node_modules/agent-base": { - "version": "6.0.2", - "resolved": "https://registry.npmjs.org/agent-base/-/agent-base-6.0.2.tgz", - "integrity": "sha512-RZNwNclF7+MS/8bDg70amg32dyeZGZxiDuQmZxKLAlQjr3jGyLx+4Kkk58UO7D2QdgFIQCovuSuZESne6RG6XQ==", + "node_modules/@babel/helper-module-transforms": { + "version": "7.23.3", + "resolved": "https://registry.npmjs.org/@babel/helper-module-transforms/-/helper-module-transforms-7.23.3.tgz", + "integrity": "sha512-7bBs4ED9OmswdfDzpz4MpWgSrV7FXlc3zIagvLFjS5H+Mk7Snr21vQ6QwrsoCGMfNC4e4LQPdoULEt4ykz0SRQ==", + "dev": true, "dependencies": { - "debug": "4" + "@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" }, "engines": { - "node": ">= 6.0.0" + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0" } }, - "node_modules/agentkeepalive": { - "version": "4.5.0", - "resolved": "https://registry.npmjs.org/agentkeepalive/-/agentkeepalive-4.5.0.tgz", - "integrity": "sha512-5GG/5IbQQpC9FpkRGsSvZI5QYeSCzlJHdpBQntCsuTOxhKD8lqKhrleg2Yi7yvMIf82Ycmmqln9U8V9qwEiJew==", + "node_modules/@babel/helper-plugin-utils": { + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-plugin-utils/-/helper-plugin-utils-7.22.5.tgz", + "integrity": "sha512-uLls06UVKgFG9QD4OeFYLEGteMIAa5kpTPcFL28yuCIIzsf6ZyKZMllKVOCZFhiZ5ptnwX4mtKdWCBE/uT4amg==", + "dev": true, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/helper-simple-access": { + "version": "7.22.5", + "resolved": "https://registry.npmjs.org/@babel/helper-simple-access/-/helper-simple-access-7.22.5.tgz", + "integrity": "sha512-n0H99E/K+Bika3++WNL17POvo4rKWZ7lZEp1Q+fStVbUi8nxPQEBOlTmCOxW/0JsS56SKKQ+ojAe2pHKJHN35w==", "dev": true, "dependencies": { - "humanize-ms": "^1.2.1" + "@babel/types": "^7.22.5" }, "engines": { - "node": ">= 8.0.0" + "node": ">=6.9.0" } }, - "node_modules/aggregate-error": { - "version": "3.1.0", - "resolved": "https://registry.npmjs.org/aggregate-error/-/aggregate-error-3.1.0.tgz", - "integrity": "sha512-4I7Td01quW/RpocfNayFdFVk1qSuoh0E7JrbRJ16nH01HhKFQ88INq9Sd+nd72zqRySlr9BmDA8xlEJ6vJMrYA==", + "node_modules/@babel/helper-split-export-declaration": { + "version": "7.22.6", + "resolved": "https://registry.npmjs.org/@babel/helper-split-export-declaration/-/helper-split-export-declaration-7.22.6.tgz", + "integrity": "sha512-AsUnxuLhRYsisFiaJwvp1QF+I3KjD5FOxut14q/GzovUe6orHLesW2C7d754kRm53h5gqrz6sFl6sxc4BVtE/g==", "dev": true, "dependencies": { - "clean-stack": "^2.0.0", - "indent-string": "^4.0.0" + "@babel/types": "^7.22.5" }, "engines": { - "node": ">=8" + "node": ">=6.9.0" } }, - "node_modules/ansi-colors": { - "version": "4.1.1", - "resolved": "https://registry.npmjs.org/ansi-colors/-/ansi-colors-4.1.1.tgz", - "integrity": "sha512-JoX0apGbHaUJBNl6yF+p6JAFYZ666/hhCGKN5t9QFjbJQKUU/g8MNbFDbvfrgKXvI1QpZplPOnwIo99lX/AAmA==", + "node_modules/@babel/helper-string-parser": { + "version": "7.23.4", + "resolved": "https://registry.npmjs.org/@babel/helper-string-parser/-/helper-string-parser-7.23.4.tgz", + "integrity": "sha512-803gmbQdqwdf4olxrX4AJyFBV/RTr3rSmOj0rKwesmzlfhYNDEs+/iOcznzpNWlJlIlTJC2QfPFcHB6DlzdVLQ==", "dev": true, "engines": { - "node": ">=6" + "node": ">=6.9.0" } }, - "node_modules/ansi-regex": { - "version": "5.0.1", - "resolved": "https://registry.npmjs.org/ansi-regex/-/ansi-regex-5.0.1.tgz", - "integrity": "sha512-quJQXlTSUGL2LH9SUXo8VwsY4soanhgo6LNSm84E1LBcE8s3O0wpdiRzyR9z/ZZJMlMWv37qOOb9pdJlMUEKFQ==", + "node_modules/@babel/helper-validator-identifier": { + "version": "7.22.20", + "resolved": "https://registry.npmjs.org/@babel/helper-validator-identifier/-/helper-validator-identifier-7.22.20.tgz", + "integrity": "sha512-Y4OZ+ytlatR8AI+8KZfKuL5urKp7qey08ha31L8b3BwewJAoJamTzyvxPR/5D+KkdJCGPq/+8TukHBlY10FX9A==", + "dev": true, "engines": { - "node": ">=8" + "node": ">=6.9.0" } }, - "node_modules/ansi-styles": { - "version": "4.3.0", - "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-4.3.0.tgz", - "integrity": "sha512-zbB9rCJAT1rbjiVDb2hqKFHNYLxgtk8NURxZ3IZwD3F6NtxbXZQCnnSi1Lkx+IDohdPlFp222wVALIheZJQSEg==", + "node_modules/@babel/helper-validator-option": { + "version": "7.23.5", + "resolved": "https://registry.npmjs.org/@babel/helper-validator-option/-/helper-validator-option-7.23.5.tgz", + "integrity": "sha512-85ttAOMLsr53VgXkTbkx8oA6YTfT4q7/HzXSLEYmjcSTJPMPQtvq1BD79Byep5xMUYbGRzEpDsjUf3dyp54IKw==", + "dev": true, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/@babel/helpers": { + "version": "7.23.5", + "resolved": "https://registry.npmjs.org/@babel/helpers/-/helpers-7.23.5.tgz", + "integrity": "sha512-oO7us8FzTEsG3U6ag9MfdF1iA/7Z6dz+MtFhifZk8C8o453rGJFFWUP1t+ULM9TUIAzC9uxXEiXjOiVMyd7QPg==", "dev": true, "dependencies": { - "color-convert": "^2.0.1" + "@babel/template": "^7.22.15", + "@babel/traverse": "^7.23.5", + "@babel/types": "^7.23.5" }, "engines": { - "node": ">=8" - }, - "funding": { - "url": "https://github.com/chalk/ansi-styles?sponsor=1" + "node": ">=6.9.0" } }, - "node_modules/anymatch": { - "version": "3.1.3", - "resolved": "https://registry.npmjs.org/anymatch/-/anymatch-3.1.3.tgz", - "integrity": "sha512-KMReFUr0B4t+D+OBkjR3KYqvocp2XaSzO55UcB6mgQMd3KbcE+mWTyvVV7D/zsdEbNnV6acZUutkiHQXvTr1Rw==", + "node_modules/@babel/highlight": { + "version": "7.23.4", + "resolved": "https://registry.npmjs.org/@babel/highlight/-/highlight-7.23.4.tgz", + "integrity": "sha512-acGdbYSfp2WheJoJm/EBBBLh/ID8KDc64ISZ9DYtBmC8/Q204PZJLHyzeB5qMzJ5trcOkybd78M4x2KWsUq++A==", "dev": true, "dependencies": { - "normalize-path": "^3.0.0", - "picomatch": "^2.0.4" + "@babel/helper-validator-identifier": "^7.22.20", + "chalk": "^2.4.2", + "js-tokens": "^4.0.0" }, "engines": { - "node": ">= 8" + "node": ">=6.9.0" } }, - "node_modules/aproba": { - "version": "2.0.0", - "resolved": "https://registry.npmjs.org/aproba/-/aproba-2.0.0.tgz", - "integrity": "sha512-lYe4Gx7QT+MKGbDsA+Z+he/Wtef0BiwDOlK/XkBrdfsh9J/jPPXbX0tE9x9cl27Tmu5gg3QUbUrQYa/y+KOHPQ==" - }, - "node_modules/are-we-there-yet": { - "version": "2.0.0", - "resolved": "https://registry.npmjs.org/are-we-there-yet/-/are-we-there-yet-2.0.0.tgz", - "integrity": "sha512-Ci/qENmwHnsYo9xKIcUJN5LeDKdJ6R1Z1j9V/J5wyq8nh/mYPEpIKJbBZXtZjG04HiK7zV/p6Vs9952MrMeUIw==", + "node_modules/@babel/highlight/node_modules/ansi-styles": { + "version": "3.2.1", + "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-3.2.1.tgz", + "integrity": "sha512-VT0ZI6kZRdTh8YyJw3SMbYm/u+NqfsAxEpWO0Pf9sq8/e94WxxOpPKx9FR1FlyCtOVDNOQ+8ntlqFxiRc+r5qA==", + "dev": true, "dependencies": { - "delegates": "^1.0.0", - "readable-stream": "^3.6.0" + "color-convert": "^1.9.0" }, "engines": { - "node": ">=10" + "node": ">=4" } }, - "node_modules/are-we-there-yet/node_modules/readable-stream": { - "version": "3.6.2", - "resolved": "https://registry.npmjs.org/readable-stream/-/readable-stream-3.6.2.tgz", - "integrity": "sha512-9u/sniCrY3D5WdsERHzHE4G2YCXqoG5FTHUiCC4SIbr6XcLZBY05ya9EKjYek9O5xOAwjGq+1JdGBAS7Q9ScoA==", + "node_modules/@babel/highlight/node_modules/chalk": { + "version": "2.4.2", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.4.2.tgz", + "integrity": "sha512-Mti+f9lpJNcwF4tWV8/OrTTtF1gZi+f8FqlyAdouralcFWFQWF2+NgCHShjkCb+IFBLq9buZwE1xckQU4peSuQ==", + "dev": true, "dependencies": { - "inherits": "^2.0.3", - "string_decoder": "^1.1.1", - "util-deprecate": "^1.0.1" + "ansi-styles": "^3.2.1", + "escape-string-regexp": "^1.0.5", + "supports-color": "^5.3.0" }, "engines": { - "node": ">= 6" + "node": ">=4" } }, - "node_modules/are-we-there-yet/node_modules/string_decoder": { - "version": "1.3.0", - "resolved": "https://registry.npmjs.org/string_decoder/-/string_decoder-1.3.0.tgz", - "integrity": "sha512-hkRX8U1WjJFd8LsDJ2yQ/wWWxaopEsABU1XfkM8A+j0+85JAGppt16cr1Whg6KIbb4okU6Mql6BOj+uup/wKeA==", + "node_modules/@babel/highlight/node_modules/color-convert": { + "version": "1.9.3", + "resolved": "https://registry.npmjs.org/color-convert/-/color-convert-1.9.3.tgz", + "integrity": "sha512-QfAUtd+vFdAtFQcC8CCyYt1fYWxSqAiK2cSD6zDB8N3cpsEBAvRxp9zOGg6G/SHHJYAT88/az/IuDGALsNVbGg==", + "dev": true, "dependencies": { - "safe-buffer": "~5.2.0" + "color-name": "1.1.3" } }, - "node_modules/argparse": { - "version": "2.0.1", - "resolved": "https://registry.npmjs.org/argparse/-/argparse-2.0.1.tgz", - "integrity": "sha512-8+9WqebbFzpX9OR+Wa6O29asIogeRMzcGtAINdpMHHyAg10f05aSFVBbcEqGf/PXw1EjAZ+q2/bEBg3DvurK3Q==", + "node_modules/@babel/highlight/node_modules/color-name": { + "version": "1.1.3", + "resolved": "https://registry.npmjs.org/color-name/-/color-name-1.1.3.tgz", + "integrity": "sha512-72fSenhMw2HZMTVHeCA9KCmpEIbzWiQsjN+BHcBbS9vr1mtt+vJjPdksIBNUmKAW8TFUDPJK5SUU3QhE9NEXDw==", "dev": true }, - "node_modules/balanced-match": { - "version": "1.0.2", - "resolved": "https://registry.npmjs.org/balanced-match/-/balanced-match-1.0.2.tgz", - "integrity": "sha512-3oSeUO0TMV67hN1AmbXsK4yaqU7tjiHlbxRDZOpH0KW9+CeX4bRAaX0Anxt0tx2MrpRpWwQaPwIlISEJhYU5Pw==" - }, - "node_modules/binary-extensions": { - "version": "2.2.0", - "resolved": "https://registry.npmjs.org/binary-extensions/-/binary-extensions-2.2.0.tgz", - "integrity": "sha512-jDctJ/IVQbZoJykoeHbhXpOlNBqGNcwXJKJog42E5HDPUwQTSdjCHdihjj0DlnheQ7blbT6dHOafNAiS8ooQKA==", + "node_modules/@babel/highlight/node_modules/escape-string-regexp": { + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/escape-string-regexp/-/escape-string-regexp-1.0.5.tgz", + "integrity": "sha512-vbRorB5FUQWvla16U8R/qgaFIya2qGzwDrNmCZuYKrbdSUMG6I1ZCGQRefkRVhuOkIGVne7BQ35DSfo1qvJqFg==", "dev": true, "engines": { - "node": ">=8" - } - }, - "node_modules/bindings": { - "version": "1.5.0", - "resolved": "https://registry.npmjs.org/bindings/-/bindings-1.5.0.tgz", - "integrity": "sha512-p2q/t/mhvuOj/UeLlV6566GD/guowlr0hHxClI0W9m7MWYkL1F0hLo+0Aexs9HSPCtR1SXQ0TD3MMKrXZajbiQ==", - "dependencies": { - "file-uri-to-path": "1.0.0" + "node": ">=0.8.0" } }, - "node_modules/bluebird": { - "version": "3.7.2", - "resolved": "https://registry.npmjs.org/bluebird/-/bluebird-3.7.2.tgz", - "integrity": "sha512-XpNj6GDQzdfW+r2Wnn7xiSAd7TM3jzkxGXBGTtWKuSXv1xUV+azxAm8jdWZN06QTQk+2N2XB9jRDkvbmQmcRtg==", - "dev": true - }, - "node_modules/brace-expansion": { - "version": "1.1.11", - "resolved": "https://registry.npmjs.org/brace-expansion/-/brace-expansion-1.1.11.tgz", - "integrity": "sha512-iCuPHDFgrHX7H2vEI/5xpz07zSHB00TpugqhmYtVmMO6518mCuRMoOYFldEBl0g187ufozdaHgWKcYFb61qGiA==", - "dependencies": { - "balanced-match": "^1.0.0", - "concat-map": "0.0.1" + "node_modules/@babel/highlight/node_modules/has-flag": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/has-flag/-/has-flag-3.0.0.tgz", + "integrity": "sha512-sKJf1+ceQBr4SMkvQnBDNDtf4TXpVhVGateu0t918bl30FnbE2m4vNLX+VWe/dpjlb+HugGYzW7uQXH98HPEYw==", + "dev": true, + "engines": { + "node": ">=4" } }, - "node_modules/braces": { - "version": "3.0.2", - "resolved": "https://registry.npmjs.org/braces/-/braces-3.0.2.tgz", - "integrity": "sha512-b8um+L1RzM3WDSzvhm6gIz1yfTbBt6YTlcEKAvsmqCZZFw46z626lVj9j1yEPW33H5H+lBQpZMP1k8l+78Ha0A==", + "node_modules/@babel/highlight/node_modules/supports-color": { + "version": "5.5.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.5.0.tgz", + "integrity": "sha512-QjVjwdXIt408MIiAqCX4oUKsgU2EqAGzs2Ppkm4aQYbjm+ZEWEcW4SfFNTr4uMNZma0ey4f5lgLrkB0aX0QMow==", "dev": true, "dependencies": { - "fill-range": "^7.0.1" + "has-flag": "^3.0.0" }, "engines": { - "node": ">=8" + "node": ">=4" } }, - "node_modules/browser-stdout": { - "version": "1.3.1", - "resolved": "https://registry.npmjs.org/browser-stdout/-/browser-stdout-1.3.1.tgz", - "integrity": "sha512-qhAVI1+Av2X7qelOfAIYwXONood6XlZE/fXaBSmW/T5SzLAmCgzi+eiWE7fUvbHaeNBQH13UftjpXxsfLkMpgw==", - "dev": true + "node_modules/@babel/parser": { + "version": "7.23.5", + "resolved": "https://registry.npmjs.org/@babel/parser/-/parser-7.23.5.tgz", + "integrity": "sha512-hOOqoiNXrmGdFbhgCzu6GiURxUgM27Xwd/aPuu8RfHEZPBzL1Z54okAHAQjXfcQNwvrlkAmAp4SlRTZ45vlthQ==", + "dev": true, + "bin": { + "parser": "bin/babel-parser.js" + }, + "engines": { + "node": ">=6.0.0" + } }, - "node_modules/cacache": { - "version": "17.1.4", - "resolved": "https://registry.npmjs.org/cacache/-/cacache-17.1.4.tgz", - "integrity": "sha512-/aJwG2l3ZMJ1xNAnqbMpA40of9dj/pIH3QfiuQSqjfPJF747VR0J/bHn+/KdNnHKc6XQcWt/AfRSBft82W1d2A==", + "node_modules/@babel/plugin-syntax-async-generators": { + "version": "7.8.4", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-async-generators/-/plugin-syntax-async-generators-7.8.4.tgz", + "integrity": "sha512-tycmZxkGfZaxhMRbXlPXuVFpdWlXpir2W4AMhSJgRKzk/eDlIXOhb2LHWoLpDF7TEHylV5zNhykX6KAgHJmTNw==", "dev": true, "dependencies": { - "@npmcli/fs": "^3.1.0", - "fs-minipass": "^3.0.0", - "glob": "^10.2.2", - "lru-cache": "^7.7.1", - "minipass": "^7.0.3", - "minipass-collect": "^1.0.2", - "minipass-flush": "^1.0.5", - "minipass-pipeline": "^1.2.4", - "p-map": "^4.0.0", - "ssri": "^10.0.0", - "tar": "^6.1.11", - "unique-filename": "^3.0.0" + "@babel/helper-plugin-utils": "^7.8.0" }, - "engines": { - "node": "^14.17.0 || ^16.13.0 || >=18.0.0" + "peerDependencies": { + "@babel/core": "^7.0.0-0" } }, - "node_modules/cacache/node_modules/brace-expansion": { - "version": "2.0.1", - "resolved": "https://registry.npmjs.org/brace-expansion/-/brace-expansion-2.0.1.tgz", - "integrity": "sha512-XnAIvQ8eM+kC6aULx6wuQiwVsnzsi9d3WxzV3FpWTGA19F621kwdbsAcFKXgKUHZWsy+mY6iL1sHTxWEFCytDA==", + "node_modules/@babel/plugin-syntax-bigint": { + "version": "7.8.3", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-bigint/-/plugin-syntax-bigint-7.8.3.tgz", + "integrity": "sha512-wnTnFlG+YxQm3vDxpGE57Pj0srRU4sHE/mDkt1qv2YJJSeUAec2ma4WLUnUPeKjyrfntVwe/N6dCXpU+zL3Npg==", "dev": true, "dependencies": { - "balanced-match": "^1.0.0" + "@babel/helper-plugin-utils": "^7.8.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" } }, - "node_modules/cacache/node_modules/glob": { - "version": "10.3.10", - "resolved": "https://registry.npmjs.org/glob/-/glob-10.3.10.tgz", - "integrity": "sha512-fa46+tv1Ak0UPK1TOy/pZrIybNNt4HCv7SDzwyfiOZkvZLEbjsZkJBPtDHVshZjbecAoAGSC20MjLDG/qr679g==", + "node_modules/@babel/plugin-syntax-class-properties": { + "version": "7.12.13", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-class-properties/-/plugin-syntax-class-properties-7.12.13.tgz", + "integrity": "sha512-fm4idjKla0YahUNgFNLCB0qySdsoPiZP3iQE3rky0mBUtMZ23yDJ9SJdg6dXTSDnulOVqiF3Hgr9nbXvXTQZYA==", "dev": true, "dependencies": { - "foreground-child": "^3.1.0", - "jackspeak": "^2.3.5", - "minimatch": "^9.0.1", - "minipass": "^5.0.0 || ^6.0.2 || ^7.0.0", - "path-scurry": "^1.10.1" + "@babel/helper-plugin-utils": "^7.12.13" }, - "bin": { - "glob": "dist/esm/bin.mjs" + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-syntax-import-meta": { + "version": "7.10.4", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-import-meta/-/plugin-syntax-import-meta-7.10.4.tgz", + "integrity": "sha512-Yqfm+XDx0+Prh3VSeEQCPU81yC+JWZ2pDPFSS4ZdpfZhp4MkFMaDC1UqseovEKwSUpnIL7+vK+Clp7bfh0iD7g==", + "dev": true, + "dependencies": { + "@babel/helper-plugin-utils": "^7.10.4" }, - "engines": { - "node": ">=16 || 14 >=14.17" + "peerDependencies": { + "@babel/core": "^7.0.0-0" + } + }, + "node_modules/@babel/plugin-syntax-json-strings": { + "version": "7.8.3", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-json-strings/-/plugin-syntax-json-strings-7.8.3.tgz", + "integrity": "sha512-lY6kdGpWHvjoe2vk4WrAapEuBR69EMxZl+RoGRhrFGNYVK8mOPAW8VfbT/ZgrFbXlDNiiaxQnAtgVCZ6jv30EA==", + "dev": true, + "dependencies": { + "@babel/helper-plugin-utils": "^7.8.0" }, - "funding": { - "url": "https://github.com/sponsors/isaacs" + "peerDependencies": { + "@babel/core": "^7.0.0-0" } }, - "node_modules/cacache/node_modules/minimatch": { - "version": "9.0.3", - "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-9.0.3.tgz", - "integrity": "sha512-RHiac9mvaRw0x3AYRgDC1CxAP7HTcNrrECeA8YYJeWnpo+2Q5CegtZjaotWTWxDG3UeGA1coE05iH1mPjT/2mg==", + "node_modules/@babel/plugin-syntax-jsx": { + "version": "7.23.3", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-jsx/-/plugin-syntax-jsx-7.23.3.tgz", + "integrity": "sha512-EB2MELswq55OHUoRZLGg/zC7QWUKfNLpE57m/S2yr1uEneIgsTgrSzXP3NXEsMkVn76OlaVVnzN+ugObuYGwhg==", "dev": true, "dependencies": { - "brace-expansion": "^2.0.1" + "@babel/helper-plugin-utils": "^7.22.5" }, "engines": { - "node": ">=16 || 14 >=14.17" + "node": ">=6.9.0" }, - "funding": { - "url": "https://github.com/sponsors/isaacs" + "peerDependencies": { + "@babel/core": "^7.0.0-0" } }, - "node_modules/cacache/node_modules/minipass": { - "version": "7.0.4", - "resolved": "https://registry.npmjs.org/minipass/-/minipass-7.0.4.tgz", - "integrity": "sha512-jYofLM5Dam9279rdkWzqHozUo4ybjdZmCsDHePy5V/PbBcVMiSZR97gmAy45aqi8CK1lG2ECd356FU86avfwUQ==", + "node_modules/@babel/plugin-syntax-logical-assignment-operators": { + "version": "7.10.4", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-logical-assignment-operators/-/plugin-syntax-logical-assignment-operators-7.10.4.tgz", + "integrity": "sha512-d8waShlpFDinQ5MtvGU9xDAOzKH47+FFoney2baFIoMr952hKOLp1HR7VszoZvOsV/4+RRszNY7D17ba0te0ig==", "dev": true, - "engines": { - "node": ">=16 || 14 >=14.17" + "dependencies": { + "@babel/helper-plugin-utils": "^7.10.4" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" } }, - "node_modules/camelcase": { - "version": "6.3.0", - "resolved": "https://registry.npmjs.org/camelcase/-/camelcase-6.3.0.tgz", - "integrity": "sha512-Gmy6FhYlCY7uOElZUSbxo2UCDH8owEk996gkbrpsgGtrJLM3J7jGxl9Ic7Qwwj4ivOE5AWZWRMecDdF7hqGjFA==", + "node_modules/@babel/plugin-syntax-nullish-coalescing-operator": { + "version": "7.8.3", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-nullish-coalescing-operator/-/plugin-syntax-nullish-coalescing-operator-7.8.3.tgz", + "integrity": "sha512-aSff4zPII1u2QD7y+F8oDsz19ew4IGEJg9SVW+bqwpwtfFleiQDMdzA/R+UlWDzfnHFCxxleFT0PMIrR36XLNQ==", "dev": true, - "engines": { - "node": ">=10" + "dependencies": { + "@babel/helper-plugin-utils": "^7.8.0" }, - "funding": { - "url": "https://github.com/sponsors/sindresorhus" + "peerDependencies": { + "@babel/core": "^7.0.0-0" } }, - "node_modules/catharsis": { - "version": "0.9.0", - "resolved": "https://registry.npmjs.org/catharsis/-/catharsis-0.9.0.tgz", - "integrity": "sha512-prMTQVpcns/tzFgFVkVp6ak6RykZyWb3gu8ckUpd6YkTlacOd3DXGJjIpD4Q6zJirizvaiAjSSHlOsA+6sNh2A==", + "node_modules/@babel/plugin-syntax-numeric-separator": { + "version": "7.10.4", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-numeric-separator/-/plugin-syntax-numeric-separator-7.10.4.tgz", + "integrity": "sha512-9H6YdfkcK/uOnY/K7/aA2xpzaAgkQn37yzWUMRK7OaPOqOpGS1+n0H5hxT9AUw9EsSjPW8SVyMJwYRtWs3X3ug==", "dev": true, "dependencies": { - "lodash": "^4.17.15" + "@babel/helper-plugin-utils": "^7.10.4" }, - "engines": { - "node": ">= 10" + "peerDependencies": { + "@babel/core": "^7.0.0-0" } }, - "node_modules/chalk": { - "version": "4.1.2", - "resolved": "https://registry.npmjs.org/chalk/-/chalk-4.1.2.tgz", - "integrity": "sha512-oKnbhFyRIXpUuez8iBMmyEa4nbj4IOQyuhc/wy9kY7/WVPcwIO9VA668Pu8RkO7+0G76SLROeyw9CpQ061i4mA==", + "node_modules/@babel/plugin-syntax-object-rest-spread": { + "version": "7.8.3", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-object-rest-spread/-/plugin-syntax-object-rest-spread-7.8.3.tgz", + "integrity": "sha512-XoqMijGZb9y3y2XskN+P1wUGiVwWZ5JmoDRwx5+3GmEplNyVM2s2Dg8ILFQm8rWM48orGy5YpI5Bl8U1y7ydlA==", "dev": true, "dependencies": { - "ansi-styles": "^4.1.0", - "supports-color": "^7.1.0" - }, - "engines": { - "node": ">=10" + "@babel/helper-plugin-utils": "^7.8.0" }, - "funding": { - "url": "https://github.com/chalk/chalk?sponsor=1" + "peerDependencies": { + "@babel/core": "^7.0.0-0" } }, - "node_modules/chalk/node_modules/supports-color": { - "version": "7.2.0", - "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-7.2.0.tgz", - "integrity": "sha512-qpCAvRl9stuOHveKsn7HncJRvv501qIacKzQlO/+Lwxc9+0q2wLyv4Dfvt80/DPn2pqOBsJdDiogXGR9+OvwRw==", + "node_modules/@babel/plugin-syntax-optional-catch-binding": { + "version": "7.8.3", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-optional-catch-binding/-/plugin-syntax-optional-catch-binding-7.8.3.tgz", + "integrity": "sha512-6VPD0Pc1lpTqw0aKoeRTMiB+kWhAoT24PA+ksWSBrFtl5SIRVpZlwN3NNPQjehA2E/91FV3RjLWoVTglWcSV3Q==", "dev": true, "dependencies": { - "has-flag": "^4.0.0" + "@babel/helper-plugin-utils": "^7.8.0" }, - "engines": { - "node": ">=8" + "peerDependencies": { + "@babel/core": "^7.0.0-0" } }, - "node_modules/chokidar": { - "version": "3.5.3", - "resolved": "https://registry.npmjs.org/chokidar/-/chokidar-3.5.3.tgz", - "integrity": "sha512-Dr3sfKRP6oTcjf2JmUmFJfeVMvXBdegxB0iVQ5eb2V10uFJUCAS8OByZdVAyVb8xXNz3GjjTgj9kLWsZTqE6kw==", + "node_modules/@babel/plugin-syntax-optional-chaining": { + "version": "7.8.3", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-optional-chaining/-/plugin-syntax-optional-chaining-7.8.3.tgz", + "integrity": "sha512-KoK9ErH1MBlCPxV0VANkXW2/dw4vlbGDrFgz8bmUsBGYkFRcbRwMh6cIJubdPrkxRwuGdtCk0v/wPTKbQgBjkg==", "dev": true, - "funding": [ - { - "type": "individual", - "url": "https://paulmillr.com/funding/" - } - ], "dependencies": { - "anymatch": "~3.1.2", - "braces": "~3.0.2", - "glob-parent": "~5.1.2", - "is-binary-path": "~2.1.0", - "is-glob": "~4.0.1", - "normalize-path": "~3.0.0", - "readdirp": "~3.6.0" - }, - "engines": { - "node": ">= 8.10.0" + "@babel/helper-plugin-utils": "^7.8.0" }, - "optionalDependencies": { - "fsevents": "~2.3.2" + "peerDependencies": { + "@babel/core": "^7.0.0-0" } }, - "node_modules/chownr": { - "version": "2.0.0", - "resolved": "https://registry.npmjs.org/chownr/-/chownr-2.0.0.tgz", - "integrity": "sha512-bIomtDF5KGpdogkLd9VspvFzk9KfpyyGlS8YFVZl7TGPBHL5snIOnxeshwVgPteQ9b4Eydl+pVbIyE1DcvCWgQ==", + "node_modules/@babel/plugin-syntax-top-level-await": { + "version": "7.14.5", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-top-level-await/-/plugin-syntax-top-level-await-7.14.5.tgz", + "integrity": "sha512-hx++upLv5U1rgYfwe1xBQUhRmU41NEvpUvrp8jkrSCdvGSnM5/qdRMtylJ6PG5OFkBaHkbTAKTnd3/YyESRHFw==", + "dev": true, + "dependencies": { + "@babel/helper-plugin-utils": "^7.14.5" + }, "engines": { - "node": ">=10" + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" } }, - "node_modules/clean-stack": { - "version": "2.2.0", - "resolved": "https://registry.npmjs.org/clean-stack/-/clean-stack-2.2.0.tgz", - "integrity": "sha512-4diC9HaTE+KRAMWhDhrGOECgWZxoevMc5TlkObMqNSsVU62PYzXZ/SMTjzyGAFF1YusgxGcSWTEXBhp0CPwQ1A==", + "node_modules/@babel/plugin-syntax-typescript": { + "version": "7.23.3", + "resolved": "https://registry.npmjs.org/@babel/plugin-syntax-typescript/-/plugin-syntax-typescript-7.23.3.tgz", + "integrity": "sha512-9EiNjVJOMwCO+43TqoTrgQ8jMwcAd0sWyXi9RPfIsLTj4R2MADDDQXELhffaUx/uJv2AYcxBgPwH6j4TIA4ytQ==", "dev": true, + "dependencies": { + "@babel/helper-plugin-utils": "^7.22.5" + }, "engines": { - "node": ">=6" + "node": ">=6.9.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0-0" } }, - "node_modules/cli": { - "version": "1.0.1", - "resolved": "https://registry.npmjs.org/cli/-/cli-1.0.1.tgz", - "integrity": "sha512-41U72MB56TfUMGndAKK8vJ78eooOD4Z5NOL4xEfjc0c23s+6EYKXlXsmACBVclLP1yOfWCgEganVzddVrSNoTg==", + "node_modules/@babel/template": { + "version": "7.22.15", + "resolved": "https://registry.npmjs.org/@babel/template/-/template-7.22.15.tgz", + "integrity": "sha512-QPErUVm4uyJa60rkI73qneDacvdvzxshT3kksGqlGWYdOTIUOwJ7RDUL8sGqslY1uXWSL6xMFKEXDS3ox2uF0w==", "dev": true, "dependencies": { - "exit": "0.1.2", - "glob": "^7.1.1" + "@babel/code-frame": "^7.22.13", + "@babel/parser": "^7.22.15", + "@babel/types": "^7.22.15" }, "engines": { - "node": ">=0.2.5" + "node": ">=6.9.0" } }, - "node_modules/cliui": { - "version": "7.0.4", - "resolved": "https://registry.npmjs.org/cliui/-/cliui-7.0.4.tgz", - "integrity": "sha512-OcRE68cOsVMXp1Yvonl/fzkQOyjLSu/8bhPDfQt0e0/Eb283TKP20Fs2MqoPsr9SwA595rRCA+QMzYc9nBP+JQ==", + "node_modules/@babel/traverse": { + "version": "7.23.5", + "resolved": "https://registry.npmjs.org/@babel/traverse/-/traverse-7.23.5.tgz", + "integrity": "sha512-czx7Xy5a6sapWWRx61m1Ke1Ra4vczu1mCTtJam5zRTBOonfdJ+S/B6HYmGYu3fJtr8GGET3si6IhgWVBhJ/m8w==", "dev": true, "dependencies": { - "string-width": "^4.2.0", - "strip-ansi": "^6.0.0", - "wrap-ansi": "^7.0.0" + "@babel/code-frame": "^7.23.5", + "@babel/generator": "^7.23.5", + "@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.5", + "@babel/types": "^7.23.5", + "debug": "^4.1.0", + "globals": "^11.1.0" + }, + "engines": { + "node": ">=6.9.0" } }, - "node_modules/color-convert": { - "version": "2.0.1", - "resolved": "https://registry.npmjs.org/color-convert/-/color-convert-2.0.1.tgz", - "integrity": "sha512-RRECPsj7iu/xb5oKYcsFHSppFNnsj/52OVTRKb4zP5onXwVF3zVmmToNcOfGC+CRDpfK/U584fMg38ZHCaElKQ==", + "node_modules/@babel/types": { + "version": "7.23.5", + "resolved": "https://registry.npmjs.org/@babel/types/-/types-7.23.5.tgz", + "integrity": "sha512-ON5kSOJwVO6xXVRTvOI0eOnWe7VdUcIpsovGo9U/Br4Ie4UVFQTboO2cYnDhAGU6Fp+UxSiT+pMft0SMHfuq6w==", "dev": true, "dependencies": { - "color-name": "~1.1.4" + "@babel/helper-string-parser": "^7.23.4", + "@babel/helper-validator-identifier": "^7.22.20", + "to-fast-properties": "^2.0.0" }, "engines": { - "node": ">=7.0.0" + "node": ">=6.9.0" } }, - "node_modules/color-name": { - "version": "1.1.4", - "resolved": "https://registry.npmjs.org/color-name/-/color-name-1.1.4.tgz", - "integrity": "sha512-dOy+3AuW3a2wNbZHIuMZpTcgjGuLU/uBL/ubcZF9OXbDo8ff4O8yVp5Bf0efS8uEoYo5q4Fx7dY9OgQGXgAsQA==", + "node_modules/@bcoe/v8-coverage": { + "version": "0.2.3", + "resolved": "https://registry.npmjs.org/@bcoe/v8-coverage/-/v8-coverage-0.2.3.tgz", + "integrity": "sha512-0hYQ8SB4Db5zvZB4axdMHGwEaQjkZzFjQiN9LVYvIFB2nSUHW9tYpxWriPrWDASIxiaXax83REcLxuSdnGPZtw==", "dev": true }, - "node_modules/color-support": { + "node_modules/@gar/promisify": { "version": "1.1.3", - "resolved": "https://registry.npmjs.org/color-support/-/color-support-1.1.3.tgz", - "integrity": "sha512-qiBjkpbMLO/HL68y+lh4q0/O1MZFj2RX6X/KmMa3+gJD3z+WwI1ZzDHysvqHGS3mP6mznPckpXmw1nI9cJjyRg==", - "bin": { - "color-support": "bin.js" - } - }, - "node_modules/concat-map": { - "version": "0.0.1", - "resolved": "https://registry.npmjs.org/concat-map/-/concat-map-0.0.1.tgz", - "integrity": "sha512-/Srv4dswyQNBfohGpz9o6Yb3Gz3SrUDqBH5rTuhGR7ahtlbYKnVxw2bCFMRljaA7EXHaXZ8wsHdodFvbkhKmqg==" + "resolved": "https://registry.npmjs.org/@gar/promisify/-/promisify-1.1.3.tgz", + "integrity": "sha512-k2Ty1JcVojjJFwrg/ThKi2ujJ7XNLYaFGNB/bWT9wGR+oSMJHMa5w+CUq6p/pVrKeNNgA7pCqEcjSnHVoqJQFw==", + "dev": true }, - "node_modules/console-browserify": { + "node_modules/@istanbuljs/load-nyc-config": { "version": "1.1.0", - "resolved": "https://registry.npmjs.org/console-browserify/-/console-browserify-1.1.0.tgz", - "integrity": "sha512-duS7VP5pvfsNLDvL1O4VOEbw37AI3A4ZUQYemvDlnpGrNu9tprR7BYWpDYwC0Xia0Zxz5ZupdiIrUp0GH1aXfg==", + "resolved": "https://registry.npmjs.org/@istanbuljs/load-nyc-config/-/load-nyc-config-1.1.0.tgz", + "integrity": "sha512-VjeHSlIzpv/NyD3N0YuHfXOPDIixcA1q2ZV98wsMqcYlPmv2n3Yb2lYP9XMElnaFVXg5A7YLTeLu6V84uQDjmQ==", "dev": true, "dependencies": { - "date-now": "^0.1.4" + "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" + }, + "engines": { + "node": ">=8" } }, - "node_modules/console-control-strings": { - "version": "1.1.0", - "resolved": "https://registry.npmjs.org/console-control-strings/-/console-control-strings-1.1.0.tgz", - "integrity": "sha512-ty/fTekppD2fIwRvnZAVdeOiGd1c7YXEixbgJTNzqcxJWKQnjJ/V1bNEEE6hygpM3WjwHFUVK6HTjWSzV4a8sQ==" - }, - "node_modules/core-util-is": { - "version": "1.0.3", - "resolved": "https://registry.npmjs.org/core-util-is/-/core-util-is-1.0.3.tgz", - "integrity": "sha512-ZQBvi1DcpJ4GDqanjucZ2Hj3wEO5pZDS89BWbkcrvdxksJorwUDDZamX9ldFkp9aw2lmBDLgkObEA4DWNJ9FYQ==", - "dev": true - }, - "node_modules/cross-spawn": { - "version": "7.0.3", - "resolved": "https://registry.npmjs.org/cross-spawn/-/cross-spawn-7.0.3.tgz", - "integrity": "sha512-iRDPJKUPVEND7dHPO8rkbOnPpyDygcDFtWjpeWNCgy8WP2rXcxXL8TskReQl6OrB2G7+UJrags1q15Fudc7G6w==", + "node_modules/@istanbuljs/schema": { + "version": "0.1.3", + "resolved": "https://registry.npmjs.org/@istanbuljs/schema/-/schema-0.1.3.tgz", + "integrity": "sha512-ZXRY4jNvVgSVQ8DL3LTcakaAtXwTVUxE81hslsyD2AtoXW/wVob10HkOJ1X/pAlcI7D+2YoZKg5do8G/w6RYgA==", "dev": true, - "dependencies": { - "path-key": "^3.1.0", - "shebang-command": "^2.0.0", - "which": "^2.0.1" - }, "engines": { - "node": ">= 8" + "node": ">=8" } }, - "node_modules/date-now": { - "version": "0.1.4", - "resolved": "https://registry.npmjs.org/date-now/-/date-now-0.1.4.tgz", - "integrity": "sha512-AsElvov3LoNB7tf5k37H2jYSB+ZZPMT5sG2QjJCcdlV5chIv6htBUBUui2IKRjgtKAKtCBN7Zbwa+MtwLjSeNw==", - "dev": true - }, - "node_modules/debug": { - "version": "4.3.4", - "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", - "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", - "dependencies": { - "ms": "2.1.2" + "node_modules/@jest/console": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/@jest/console/-/console-29.7.0.tgz", + "integrity": "sha512-5Ni4CU7XHQi32IJ398EEP4RrB8eV09sXP2ROqD4bksHrnTree52PsxvX8tpL8LvTZ3pFzXyPbNQReSN41CAhOg==", + "dev": true, + "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" + }, + "engines": { + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + } + }, + "node_modules/@jest/core": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/@jest/core/-/core-29.7.0.tgz", + "integrity": "sha512-n7aeXWKMnGtDA48y8TLWJPJmLmmZ642Ceo78cYWEpiD7FzDgmNDV/GCVRorPABdXLJZ/9wzzgZAlHjXjxDHGsg==", + "dev": true, + "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" }, "engines": { - "node": ">=6.0" + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + }, + "peerDependencies": { + "node-notifier": "^8.0.1 || ^9.0.0 || ^10.0.0" }, "peerDependenciesMeta": { - "supports-color": { + "node-notifier": { "optional": true } } }, - "node_modules/debug/node_modules/ms": { - "version": "2.1.2", - "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.2.tgz", - "integrity": "sha512-sGkPx+VjMtmA6MX27oA4FBFELFCZZ4S4XqeGOXCv68tT+jb3vk/RyaKWP0PTKyWtmLSM0b+adUTEvbs1PEaH2w==" - }, - "node_modules/decamelize": { - "version": "4.0.0", - "resolved": "https://registry.npmjs.org/decamelize/-/decamelize-4.0.0.tgz", - "integrity": "sha512-9iE1PgSik9HeIIw2JO94IidnE3eBoQrFJ3w7sFuzSX4DpmZ3v5sZpUiV5Swcf6mQEF+Y0ru8Neo+p+nyh2J+hQ==", + "node_modules/@jest/environment": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/@jest/environment/-/environment-29.7.0.tgz", + "integrity": "sha512-aQIfHDq33ExsN4jP1NWGXhxgQ/wixs60gDiKO+XVMd8Mn0NWPWgc34ZQDTb2jKaUWQ7MuwoitXAsN2XVXNMpAw==", "dev": true, - "engines": { - "node": ">=10" + "dependencies": { + "@jest/fake-timers": "^29.7.0", + "@jest/types": "^29.6.3", + "@types/node": "*", + "jest-mock": "^29.7.0" }, - "funding": { - "url": "https://github.com/sponsors/sindresorhus" + "engines": { + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" } }, - "node_modules/delegates": { - "version": "1.0.0", - "resolved": "https://registry.npmjs.org/delegates/-/delegates-1.0.0.tgz", - "integrity": "sha512-bd2L678uiWATM6m5Z1VzNCErI3jiGzt6HGY8OVICs40JQq/HALfbyNJmp0UDakEY4pMMaN0Ly5om/B1VI/+xfQ==" - }, - "node_modules/detect-libc": { - "version": "2.0.2", - "resolved": "https://registry.npmjs.org/detect-libc/-/detect-libc-2.0.2.tgz", - "integrity": "sha512-UX6sGumvvqSaXgdKGUsgZWqcUyIXZ/vZTrlRT/iobiKhGL0zL4d3osHj3uqllWJK+i+sixDS/3COVEOFbupFyw==", + "node_modules/@jest/expect": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/@jest/expect/-/expect-29.7.0.tgz", + "integrity": "sha512-8uMeAMycttpva3P1lBHB8VciS9V0XAr3GymPpipdyQXbBcuhkLQOSe8E/p92RyAdToS6ZD1tFkX+CkhoECE0dQ==", + "dev": true, + "dependencies": { + "expect": "^29.7.0", + "jest-snapshot": "^29.7.0" + }, "engines": { - "node": ">=8" + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" } }, - "node_modules/diff": { - "version": "5.0.0", - "resolved": "https://registry.npmjs.org/diff/-/diff-5.0.0.tgz", - "integrity": "sha512-/VTCrvm5Z0JGty/BWHljh+BAiw3IK+2j87NGMu8Nwc/f48WoDAC395uomO9ZD117ZOBaHmkX1oyLvkVM/aIT3w==", + "node_modules/@jest/expect-utils": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/@jest/expect-utils/-/expect-utils-29.7.0.tgz", + "integrity": "sha512-GlsNBWiFQFCVi9QVSx7f5AgMeLxe9YCCs5PuP2O2LdjDAA8Jh9eX7lA1Jq/xdXw3Wb3hyvlFNfZIfcRetSzYcA==", "dev": true, + "dependencies": { + "jest-get-type": "^29.6.3" + }, "engines": { - "node": ">=0.3.1" + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" } }, - "node_modules/dom-serializer": { - "version": "0.2.2", - "resolved": "https://registry.npmjs.org/dom-serializer/-/dom-serializer-0.2.2.tgz", - "integrity": "sha512-2/xPb3ORsQ42nHYiSunXkDjPLBaEj/xTwUO4B7XCZQTRk7EBtTOPaygh10YAAh2OI1Qrp6NWfpAhzswj0ydt9g==", + "node_modules/@jest/fake-timers": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/@jest/fake-timers/-/fake-timers-29.7.0.tgz", + "integrity": "sha512-q4DH1Ha4TTFPdxLsqDXK1d3+ioSL7yL5oCMJZgDYm6i+6CygW5E5xVr/D1HdsGxjt1ZWSfUAs9OxSB/BNelWrQ==", "dev": true, "dependencies": { - "domelementtype": "^2.0.1", - "entities": "^2.0.0" + "@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" + }, + "engines": { + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + } + }, + "node_modules/@jest/globals": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/@jest/globals/-/globals-29.7.0.tgz", + "integrity": "sha512-mpiz3dutLbkW2MNFubUGUEVLkTGiqW6yLVTA+JbP6fI6J5iL9Y0Nlg8k95pcF8ctKwCS7WVxteBs29hhfAotzQ==", + "dev": true, + "dependencies": { + "@jest/environment": "^29.7.0", + "@jest/expect": "^29.7.0", + "@jest/types": "^29.6.3", + "jest-mock": "^29.7.0" + }, + "engines": { + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + } + }, + "node_modules/@jest/reporters": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/@jest/reporters/-/reporters-29.7.0.tgz", + "integrity": "sha512-DApq0KJbJOEzAFYjHADNNxAE3KbhxQB1y5Kplb5Waqw6zVbuWatSnMjE5gs8FUgEPmNsnZA3NCWl9NG0ia04Pg==", + "dev": true, + "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" + }, + "engines": { + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + }, + "peerDependencies": { + "node-notifier": "^8.0.1 || ^9.0.0 || ^10.0.0" + }, + "peerDependenciesMeta": { + "node-notifier": { + "optional": true + } + } + }, + "node_modules/@jest/schemas": { + "version": "29.6.3", + "resolved": "https://registry.npmjs.org/@jest/schemas/-/schemas-29.6.3.tgz", + "integrity": "sha512-mo5j5X+jIZmJQveBKeS/clAueipV7KgiX1vMgCxam1RNYiqE1w62n0/tJJnHtjW8ZHcQco5gY85jA3mi0L+nSA==", + "dev": true, + "dependencies": { + "@sinclair/typebox": "^0.27.8" + }, + "engines": { + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + } + }, + "node_modules/@jest/source-map": { + "version": "29.6.3", + "resolved": "https://registry.npmjs.org/@jest/source-map/-/source-map-29.6.3.tgz", + "integrity": "sha512-MHjT95QuipcPrpLM+8JMSzFx6eHp5Bm+4XeFDJlwsvVBjmKNiIAvasGK2fxz2WbGRlnvqehFbh07MMa7n3YJnw==", + "dev": true, + "dependencies": { + "@jridgewell/trace-mapping": "^0.3.18", + "callsites": "^3.0.0", + "graceful-fs": "^4.2.9" + }, + "engines": { + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + } + }, + "node_modules/@jest/test-result": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/@jest/test-result/-/test-result-29.7.0.tgz", + "integrity": "sha512-Fdx+tv6x1zlkJPcWXmMDAG2HBnaR9XPSd5aDWQVsfrZmLVT3lU1cwyxLgRmXR9yrq4NBoEm9BMsfgFzTQAbJYA==", + "dev": true, + "dependencies": { + "@jest/console": "^29.7.0", + "@jest/types": "^29.6.3", + "@types/istanbul-lib-coverage": "^2.0.0", + "collect-v8-coverage": "^1.0.0" + }, + "engines": { + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + } + }, + "node_modules/@jest/test-sequencer": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/@jest/test-sequencer/-/test-sequencer-29.7.0.tgz", + "integrity": "sha512-GQwJ5WZVrKnOJuiYiAF52UNUJXgTZx1NHjFSEB0qEMmSZKAkdMoIzw/Cj6x6NF4AvV23AUqDpFzQkN/eYCYTxw==", + "dev": true, + "dependencies": { + "@jest/test-result": "^29.7.0", + "graceful-fs": "^4.2.9", + "jest-haste-map": "^29.7.0", + "slash": "^3.0.0" + }, + "engines": { + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + } + }, + "node_modules/@jest/transform": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/@jest/transform/-/transform-29.7.0.tgz", + "integrity": "sha512-ok/BTPFzFKVMwO5eOHRrvnBVHdRy9IrsrW1GpMaQ9MCnilNLXQKmAX8s1YXDFaai9xJpac2ySzV0YeRRECr2Vw==", + "dev": true, + "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" + }, + "engines": { + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + } + }, + "node_modules/@jest/types": { + "version": "29.6.3", + "resolved": "https://registry.npmjs.org/@jest/types/-/types-29.6.3.tgz", + "integrity": "sha512-u3UPsIilWKOM3F9CXtrG8LEJmNxwoCQC/XVj4IKYXvvpx7QIi/Kg1LI5uDmDpKlac62NUtX7eLjRh+jVZcLOzw==", + "dev": true, + "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" + }, + "engines": { + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + } + }, + "node_modules/@jridgewell/gen-mapping": { + "version": "0.3.3", + "resolved": "https://registry.npmjs.org/@jridgewell/gen-mapping/-/gen-mapping-0.3.3.tgz", + "integrity": "sha512-HLhSWOLRi875zjjMG/r+Nv0oCW8umGb0BgEhyX3dDX3egwZtB8PqLnjz3yedt8R5StBrzcg4aBpnh8UA9D1BoQ==", + "dev": true, + "dependencies": { + "@jridgewell/set-array": "^1.0.1", + "@jridgewell/sourcemap-codec": "^1.4.10", + "@jridgewell/trace-mapping": "^0.3.9" + }, + "engines": { + "node": ">=6.0.0" + } + }, + "node_modules/@jridgewell/resolve-uri": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/@jridgewell/resolve-uri/-/resolve-uri-3.1.1.tgz", + "integrity": "sha512-dSYZh7HhCDtCKm4QakX0xFpsRDqjjtZf/kjI/v3T3Nwt5r8/qz/M19F9ySyOqU94SXBmeG9ttTul+YnR4LOxFA==", + "dev": true, + "engines": { + "node": ">=6.0.0" + } + }, + "node_modules/@jridgewell/set-array": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/@jridgewell/set-array/-/set-array-1.1.2.tgz", + "integrity": "sha512-xnkseuNADM0gt2bs+BvhO0p78Mk762YnZdsuzFV018NoG1Sj1SCQvpSqa7XUaTam5vAGasABV9qXASMKnFMwMw==", + "dev": true, + "engines": { + "node": ">=6.0.0" + } + }, + "node_modules/@jridgewell/sourcemap-codec": { + "version": "1.4.15", + "resolved": "https://registry.npmjs.org/@jridgewell/sourcemap-codec/-/sourcemap-codec-1.4.15.tgz", + "integrity": "sha512-eF2rxCRulEKXHTRiDrDy6erMYWqNw4LPdQ8UQA4huuxaQsVeRPFl2oM8oDGxMFhJUWZf9McpLtJasDDZb/Bpeg==", + "dev": true + }, + "node_modules/@jridgewell/trace-mapping": { + "version": "0.3.20", + "resolved": "https://registry.npmjs.org/@jridgewell/trace-mapping/-/trace-mapping-0.3.20.tgz", + "integrity": "sha512-R8LcPeWZol2zR8mmH3JeKQ6QRCFb7XgUhV9ZlGhHLGyg4wpPiPZNQOOWhFZhxKw8u//yTbNGI42Bx/3paXEQ+Q==", + "dev": true, + "dependencies": { + "@jridgewell/resolve-uri": "^3.1.0", + "@jridgewell/sourcemap-codec": "^1.4.14" + } + }, + "node_modules/@mapbox/node-pre-gyp": { + "version": "1.0.11", + "resolved": "https://registry.npmjs.org/@mapbox/node-pre-gyp/-/node-pre-gyp-1.0.11.tgz", + "integrity": "sha512-Yhlar6v9WQgUp/He7BdgzOz8lqMQ8sU+jkCq7Wx8Myc5YFJLbEe7lgui/V7G1qB1DJykHSGwreceSaD60Y0PUQ==", + "dependencies": { + "detect-libc": "^2.0.0", + "https-proxy-agent": "^5.0.0", + "make-dir": "^3.1.0", + "node-fetch": "^2.6.7", + "nopt": "^5.0.0", + "npmlog": "^5.0.1", + "rimraf": "^3.0.2", + "semver": "^7.3.5", + "tar": "^6.1.11" + }, + "bin": { + "node-pre-gyp": "bin/node-pre-gyp" + } + }, + "node_modules/@npmcli/fs": { + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/@npmcli/fs/-/fs-2.1.2.tgz", + "integrity": "sha512-yOJKRvohFOaLqipNtwYB9WugyZKhC/DZC4VYPmpaCzDBrA8YpK3qHZ8/HGscMnE4GqbkLNuVcCnxkeQEdGt6LQ==", + "dev": true, + "dependencies": { + "@gar/promisify": "^1.1.3", + "semver": "^7.3.5" + }, + "engines": { + "node": "^12.13.0 || ^14.15.0 || >=16.0.0" + } + }, + "node_modules/@npmcli/move-file": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/@npmcli/move-file/-/move-file-2.0.1.tgz", + "integrity": "sha512-mJd2Z5TjYWq/ttPLLGqArdtnC74J6bOzg4rMDnN+p1xTacZ2yPRCk2y0oSWQtygLR9YVQXgOcONrwtnk3JupxQ==", + "deprecated": "This functionality has been moved to @npmcli/fs", + "dev": true, + "dependencies": { + "mkdirp": "^1.0.4", + "rimraf": "^3.0.2" + }, + "engines": { + "node": "^12.13.0 || ^14.15.0 || >=16.0.0" + } + }, + "node_modules/@sinclair/typebox": { + "version": "0.27.8", + "resolved": "https://registry.npmjs.org/@sinclair/typebox/-/typebox-0.27.8.tgz", + "integrity": "sha512-+Fj43pSMwJs4KRrH/938Uf+uAELIgVBmQzg/q1YG10djyfA3TnrU8N8XzqCh/okZdszqBQTZf96idMfE5lnwTA==", + "dev": true + }, + "node_modules/@sinonjs/commons": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/@sinonjs/commons/-/commons-3.0.0.tgz", + "integrity": "sha512-jXBtWAF4vmdNmZgD5FoKsVLv3rPgDnLgPbU84LIJ3otV44vJlDRokVng5v8NFJdCf/da9legHcKaRuZs4L7faA==", + "dev": true, + "dependencies": { + "type-detect": "4.0.8" + } + }, + "node_modules/@sinonjs/fake-timers": { + "version": "10.3.0", + "resolved": "https://registry.npmjs.org/@sinonjs/fake-timers/-/fake-timers-10.3.0.tgz", + "integrity": "sha512-V4BG07kuYSUkTCSBHG8G8TNhM+F19jXFWnQtzj+we8DrkpSBCee9Z3Ms8yiGer/dlmhe35/Xdgyo3/0rQKg7YA==", + "dev": true, + "dependencies": { + "@sinonjs/commons": "^3.0.0" + } + }, + "node_modules/@tootallnate/once": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/@tootallnate/once/-/once-2.0.0.tgz", + "integrity": "sha512-XCuKFP5PS55gnMVu3dty8KPatLqUoy/ZYzDzAGCQ8JNFCkLXzmI7vNHCR+XpbZaMWQK/vQubr7PkYq8g470J/A==", + "dev": true, + "engines": { + "node": ">= 10" + } + }, + "node_modules/@types/babel__core": { + "version": "7.20.5", + "resolved": "https://registry.npmjs.org/@types/babel__core/-/babel__core-7.20.5.tgz", + "integrity": "sha512-qoQprZvz5wQFJwMDqeseRXWv3rqMvhgpbXFfVyWhbx9X47POIA6i/+dXefEmZKoAgOaTdaIgNSMqMIU61yRyzA==", + "dev": true, + "dependencies": { + "@babel/parser": "^7.20.7", + "@babel/types": "^7.20.7", + "@types/babel__generator": "*", + "@types/babel__template": "*", + "@types/babel__traverse": "*" + } + }, + "node_modules/@types/babel__generator": { + "version": "7.6.7", + "resolved": "https://registry.npmjs.org/@types/babel__generator/-/babel__generator-7.6.7.tgz", + "integrity": "sha512-6Sfsq+EaaLrw4RmdFWE9Onp63TOUue71AWb4Gpa6JxzgTYtimbM086WnYTy2U67AofR++QKCo08ZP6pwx8YFHQ==", + "dev": true, + "dependencies": { + "@babel/types": "^7.0.0" + } + }, + "node_modules/@types/babel__template": { + "version": "7.4.4", + "resolved": "https://registry.npmjs.org/@types/babel__template/-/babel__template-7.4.4.tgz", + "integrity": "sha512-h/NUaSyG5EyxBIp8YRxo4RMe2/qQgvyowRwVMzhYhBCONbW8PUsg4lkFMrhgZhUe5z3L3MiLDuvyJ/CaPa2A8A==", + "dev": true, + "dependencies": { + "@babel/parser": "^7.1.0", + "@babel/types": "^7.0.0" + } + }, + "node_modules/@types/babel__traverse": { + "version": "7.20.4", + "resolved": "https://registry.npmjs.org/@types/babel__traverse/-/babel__traverse-7.20.4.tgz", + "integrity": "sha512-mSM/iKUk5fDDrEV/e83qY+Cr3I1+Q3qqTuEn++HAWYjEa1+NxZr6CNrcJGf2ZTnq4HoFGC3zaTPZTobCzCFukA==", + "dev": true, + "dependencies": { + "@babel/types": "^7.20.7" + } + }, + "node_modules/@types/graceful-fs": { + "version": "4.1.9", + "resolved": "https://registry.npmjs.org/@types/graceful-fs/-/graceful-fs-4.1.9.tgz", + "integrity": "sha512-olP3sd1qOEe5dXTSaFvQG+02VdRXcdytWLAZsAq1PecU8uqQAhkrnbli7DagjtXKW/Bl7YJbUsa8MPcuc8LHEQ==", + "dev": true, + "dependencies": { + "@types/node": "*" + } + }, + "node_modules/@types/istanbul-lib-coverage": { + "version": "2.0.6", + "resolved": "https://registry.npmjs.org/@types/istanbul-lib-coverage/-/istanbul-lib-coverage-2.0.6.tgz", + "integrity": "sha512-2QF/t/auWm0lsy8XtKVPG19v3sSOQlJe/YHZgfjb/KBBHOGSV+J2q/S671rcq9uTBrLAXmZpqJiaQbMT+zNU1w==", + "dev": true + }, + "node_modules/@types/istanbul-lib-report": { + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/@types/istanbul-lib-report/-/istanbul-lib-report-3.0.3.tgz", + "integrity": "sha512-NQn7AHQnk/RSLOxrBbGyJM/aVQ+pjj5HCgasFxc0K/KhoATfQ/47AyUl15I2yBUpihjmas+a+VJBOqecrFH+uA==", + "dev": true, + "dependencies": { + "@types/istanbul-lib-coverage": "*" + } + }, + "node_modules/@types/istanbul-reports": { + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/@types/istanbul-reports/-/istanbul-reports-3.0.4.tgz", + "integrity": "sha512-pk2B1NWalF9toCRu6gjBzR69syFjP4Od8WRAX+0mmf9lAjCRicLOWc+ZrxZHx/0XRjotgkF9t6iaMJ+aXcOdZQ==", + "dev": true, + "dependencies": { + "@types/istanbul-lib-report": "*" + } + }, + "node_modules/@types/linkify-it": { + "version": "3.0.5", + "resolved": "https://registry.npmjs.org/@types/linkify-it/-/linkify-it-3.0.5.tgz", + "integrity": "sha512-yg6E+u0/+Zjva+buc3EIb+29XEg4wltq7cSmd4Uc2EE/1nUVmxyzpX6gUXD0V8jIrG0r7YeOGVIbYRkxeooCtw==", + "dev": true + }, + "node_modules/@types/markdown-it": { + "version": "12.2.3", + "resolved": "https://registry.npmjs.org/@types/markdown-it/-/markdown-it-12.2.3.tgz", + "integrity": "sha512-GKMHFfv3458yYy+v/N8gjufHO6MSZKCOXpZc5GXIWWy8uldwfmPn98vp81gZ5f9SVw8YYBctgfJ22a2d7AOMeQ==", + "dev": true, + "dependencies": { + "@types/linkify-it": "*", + "@types/mdurl": "*" + } + }, + "node_modules/@types/mdurl": { + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/@types/mdurl/-/mdurl-1.0.5.tgz", + "integrity": "sha512-6L6VymKTzYSrEf4Nev4Xa1LCHKrlTlYCBMTlQKFuddo1CvQcE52I0mwfOJayueUC7MJuXOeHTcIU683lzd0cUA==", + "dev": true + }, + "node_modules/@types/node": { + "version": "20.10.4", + "resolved": "https://registry.npmjs.org/@types/node/-/node-20.10.4.tgz", + "integrity": "sha512-D08YG6rr8X90YB56tSIuBaddy/UXAA9RKJoFvrsnogAum/0pmjkgi4+2nx96A330FmioegBWmEYQ+syqCFaveg==", + "dev": true, + "dependencies": { + "undici-types": "~5.26.4" + } + }, + "node_modules/@types/stack-utils": { + "version": "2.0.3", + "resolved": "https://registry.npmjs.org/@types/stack-utils/-/stack-utils-2.0.3.tgz", + "integrity": "sha512-9aEbYZ3TbYMznPdcdr3SmIrLXwC/AKZXQeCf9Pgao5CKb8CyHuEX5jzWPTkvregvhRJHcpRO6BFoGW9ycaOkYw==", + "dev": true + }, + "node_modules/@types/yargs": { + "version": "17.0.32", + "resolved": "https://registry.npmjs.org/@types/yargs/-/yargs-17.0.32.tgz", + "integrity": "sha512-xQ67Yc/laOG5uMfX/093MRlGGCIBzZMarVa+gfNKJxWAIgykYpVGkBdbqEzGDDfCrVUj6Hiff4mTZ5BA6TmAog==", + "dev": true, + "dependencies": { + "@types/yargs-parser": "*" + } + }, + "node_modules/@types/yargs-parser": { + "version": "21.0.3", + "resolved": "https://registry.npmjs.org/@types/yargs-parser/-/yargs-parser-21.0.3.tgz", + "integrity": "sha512-I4q9QU9MQv4oEOz4tAHJtNz1cwuLxn2F3xcc2iV5WdqLPpUnj30aUuxt1mAxYTG+oe8CZMV/+6rU4S4gRDzqtQ==", + "dev": true + }, + "node_modules/abbrev": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/abbrev/-/abbrev-1.1.1.tgz", + "integrity": "sha512-nne9/IiQ/hzIhY6pdDnbBtz7DjPTKrY00P/zvPSm5pOFkl6xuGrGnXn/VtTNNfNtAfZ9/1RtehkszU9qcTii0Q==" + }, + "node_modules/agent-base": { + "version": "6.0.2", + "resolved": "https://registry.npmjs.org/agent-base/-/agent-base-6.0.2.tgz", + "integrity": "sha512-RZNwNclF7+MS/8bDg70amg32dyeZGZxiDuQmZxKLAlQjr3jGyLx+4Kkk58UO7D2QdgFIQCovuSuZESne6RG6XQ==", + "dependencies": { + "debug": "4" + }, + "engines": { + "node": ">= 6.0.0" + } + }, + "node_modules/agentkeepalive": { + "version": "4.5.0", + "resolved": "https://registry.npmjs.org/agentkeepalive/-/agentkeepalive-4.5.0.tgz", + "integrity": "sha512-5GG/5IbQQpC9FpkRGsSvZI5QYeSCzlJHdpBQntCsuTOxhKD8lqKhrleg2Yi7yvMIf82Ycmmqln9U8V9qwEiJew==", + "dev": true, + "dependencies": { + "humanize-ms": "^1.2.1" + }, + "engines": { + "node": ">= 8.0.0" + } + }, + "node_modules/aggregate-error": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/aggregate-error/-/aggregate-error-3.1.0.tgz", + "integrity": "sha512-4I7Td01quW/RpocfNayFdFVk1qSuoh0E7JrbRJ16nH01HhKFQ88INq9Sd+nd72zqRySlr9BmDA8xlEJ6vJMrYA==", + "dev": true, + "dependencies": { + "clean-stack": "^2.0.0", + "indent-string": "^4.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/ansi-colors": { + "version": "4.1.1", + "resolved": "https://registry.npmjs.org/ansi-colors/-/ansi-colors-4.1.1.tgz", + "integrity": "sha512-JoX0apGbHaUJBNl6yF+p6JAFYZ666/hhCGKN5t9QFjbJQKUU/g8MNbFDbvfrgKXvI1QpZplPOnwIo99lX/AAmA==", + "dev": true, + "engines": { + "node": ">=6" + } + }, + "node_modules/ansi-escapes": { + "version": "4.3.2", + "resolved": "https://registry.npmjs.org/ansi-escapes/-/ansi-escapes-4.3.2.tgz", + "integrity": "sha512-gKXj5ALrKWQLsYG9jlTRmR/xKluxHV+Z9QEwNIgCfM1/uwPMCuzVVnh5mwTd+OuBZcwSIMbqssNWRm1lE51QaQ==", + "dev": true, + "dependencies": { + "type-fest": "^0.21.3" + }, + "engines": { + "node": ">=8" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/ansi-regex": { + "version": "5.0.1", + "resolved": "https://registry.npmjs.org/ansi-regex/-/ansi-regex-5.0.1.tgz", + "integrity": "sha512-quJQXlTSUGL2LH9SUXo8VwsY4soanhgo6LNSm84E1LBcE8s3O0wpdiRzyR9z/ZZJMlMWv37qOOb9pdJlMUEKFQ==", + "engines": { + "node": ">=8" + } + }, + "node_modules/ansi-styles": { + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-4.3.0.tgz", + "integrity": "sha512-zbB9rCJAT1rbjiVDb2hqKFHNYLxgtk8NURxZ3IZwD3F6NtxbXZQCnnSi1Lkx+IDohdPlFp222wVALIheZJQSEg==", + "dev": true, + "dependencies": { + "color-convert": "^2.0.1" + }, + "engines": { + "node": ">=8" + }, + "funding": { + "url": "https://github.com/chalk/ansi-styles?sponsor=1" + } + }, + "node_modules/anymatch": { + "version": "3.1.3", + "resolved": "https://registry.npmjs.org/anymatch/-/anymatch-3.1.3.tgz", + "integrity": "sha512-KMReFUr0B4t+D+OBkjR3KYqvocp2XaSzO55UcB6mgQMd3KbcE+mWTyvVV7D/zsdEbNnV6acZUutkiHQXvTr1Rw==", + "dev": true, + "dependencies": { + "normalize-path": "^3.0.0", + "picomatch": "^2.0.4" + }, + "engines": { + "node": ">= 8" + } + }, + "node_modules/aproba": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/aproba/-/aproba-2.0.0.tgz", + "integrity": "sha512-lYe4Gx7QT+MKGbDsA+Z+he/Wtef0BiwDOlK/XkBrdfsh9J/jPPXbX0tE9x9cl27Tmu5gg3QUbUrQYa/y+KOHPQ==" + }, + "node_modules/are-we-there-yet": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/are-we-there-yet/-/are-we-there-yet-2.0.0.tgz", + "integrity": "sha512-Ci/qENmwHnsYo9xKIcUJN5LeDKdJ6R1Z1j9V/J5wyq8nh/mYPEpIKJbBZXtZjG04HiK7zV/p6Vs9952MrMeUIw==", + "dependencies": { + "delegates": "^1.0.0", + "readable-stream": "^3.6.0" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/are-we-there-yet/node_modules/readable-stream": { + "version": "3.6.2", + "resolved": "https://registry.npmjs.org/readable-stream/-/readable-stream-3.6.2.tgz", + "integrity": "sha512-9u/sniCrY3D5WdsERHzHE4G2YCXqoG5FTHUiCC4SIbr6XcLZBY05ya9EKjYek9O5xOAwjGq+1JdGBAS7Q9ScoA==", + "dependencies": { + "inherits": "^2.0.3", + "string_decoder": "^1.1.1", + "util-deprecate": "^1.0.1" + }, + "engines": { + "node": ">= 6" + } + }, + "node_modules/are-we-there-yet/node_modules/string_decoder": { + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/string_decoder/-/string_decoder-1.3.0.tgz", + "integrity": "sha512-hkRX8U1WjJFd8LsDJ2yQ/wWWxaopEsABU1XfkM8A+j0+85JAGppt16cr1Whg6KIbb4okU6Mql6BOj+uup/wKeA==", + "dependencies": { + "safe-buffer": "~5.2.0" + } + }, + "node_modules/argparse": { + "version": "1.0.10", + "resolved": "https://registry.npmjs.org/argparse/-/argparse-1.0.10.tgz", + "integrity": "sha512-o5Roy6tNG4SL/FOkCAN6RzjiakZS25RLYFrcMttJqbdd8BWrnA+fGz57iN5Pb06pvBGvl5gQ0B48dJlslXvoTg==", + "dev": true, + "dependencies": { + "sprintf-js": "~1.0.2" + } + }, + "node_modules/babel-jest": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/babel-jest/-/babel-jest-29.7.0.tgz", + "integrity": "sha512-BrvGY3xZSwEcCzKvKsCi2GgHqDqsYkOP4/by5xCgIwGXQxIEh+8ew3gmrE1y7XRR6LHZIj6yLYnUi/mm2KXKBg==", + "dev": true, + "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" + }, + "engines": { + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + }, + "peerDependencies": { + "@babel/core": "^7.8.0" + } + }, + "node_modules/babel-plugin-istanbul": { + "version": "6.1.1", + "resolved": "https://registry.npmjs.org/babel-plugin-istanbul/-/babel-plugin-istanbul-6.1.1.tgz", + "integrity": "sha512-Y1IQok9821cC9onCx5otgFfRm7Lm+I+wwxOx738M/WLPZ9Q42m4IG5W0FNX8WLL2gYMZo3JkuXIH2DOpWM+qwA==", + "dev": true, + "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" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/babel-plugin-istanbul/node_modules/istanbul-lib-instrument": { + "version": "5.2.1", + "resolved": "https://registry.npmjs.org/istanbul-lib-instrument/-/istanbul-lib-instrument-5.2.1.tgz", + "integrity": "sha512-pzqtp31nLv/XFOzXGuvhCb8qhjmTVo5vjVk19XE4CRlSWz0KoeJ3bw9XsA7nOp9YBf4qHjwBxkDzKcME/J29Yg==", + "dev": true, + "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" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/babel-plugin-istanbul/node_modules/semver": { + "version": "6.3.1", + "resolved": "https://registry.npmjs.org/semver/-/semver-6.3.1.tgz", + "integrity": "sha512-BR7VvDCVHO+q2xBEWskxS6DJE1qRnb7DxzUrogb71CWoSficBxYsiAGd+Kl0mmq/MprG9yArRkyrQxTO6XjMzA==", + "dev": true, + "bin": { + "semver": "bin/semver.js" + } + }, + "node_modules/babel-plugin-jest-hoist": { + "version": "29.6.3", + "resolved": "https://registry.npmjs.org/babel-plugin-jest-hoist/-/babel-plugin-jest-hoist-29.6.3.tgz", + "integrity": "sha512-ESAc/RJvGTFEzRwOTT4+lNDk/GNHMkKbNzsvT0qKRfDyyYTskxB5rnU2njIDYVxXCBHHEI1c0YwHob3WaYujOg==", + "dev": true, + "dependencies": { + "@babel/template": "^7.3.3", + "@babel/types": "^7.3.3", + "@types/babel__core": "^7.1.14", + "@types/babel__traverse": "^7.0.6" + }, + "engines": { + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + } + }, + "node_modules/babel-preset-current-node-syntax": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/babel-preset-current-node-syntax/-/babel-preset-current-node-syntax-1.0.1.tgz", + "integrity": "sha512-M7LQ0bxarkxQoN+vz5aJPsLBn77n8QgTFmo8WK0/44auK2xlCXrYcUxHFxgU7qW5Yzw/CjmLRK2uJzaCd7LvqQ==", + "dev": true, + "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" + }, + "peerDependencies": { + "@babel/core": "^7.0.0" + } + }, + "node_modules/babel-preset-jest": { + "version": "29.6.3", + "resolved": "https://registry.npmjs.org/babel-preset-jest/-/babel-preset-jest-29.6.3.tgz", + "integrity": "sha512-0B3bhxR6snWXJZtR/RliHTDPRgn1sNHOR0yVtq/IiQFyuOVjFS+wuio/R4gSNkyYmKmJB4wGZv2NZanmKmTnNA==", + "dev": true, + "dependencies": { + "babel-plugin-jest-hoist": "^29.6.3", + "babel-preset-current-node-syntax": "^1.0.0" + }, + "engines": { + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + }, + "peerDependencies": { + "@babel/core": "^7.0.0" + } + }, + "node_modules/balanced-match": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/balanced-match/-/balanced-match-1.0.2.tgz", + "integrity": "sha512-3oSeUO0TMV67hN1AmbXsK4yaqU7tjiHlbxRDZOpH0KW9+CeX4bRAaX0Anxt0tx2MrpRpWwQaPwIlISEJhYU5Pw==" + }, + "node_modules/binary-extensions": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/binary-extensions/-/binary-extensions-2.2.0.tgz", + "integrity": "sha512-jDctJ/IVQbZoJykoeHbhXpOlNBqGNcwXJKJog42E5HDPUwQTSdjCHdihjj0DlnheQ7blbT6dHOafNAiS8ooQKA==", + "dev": true, + "engines": { + "node": ">=8" + } + }, + "node_modules/bindings": { + "version": "1.5.0", + "resolved": "https://registry.npmjs.org/bindings/-/bindings-1.5.0.tgz", + "integrity": "sha512-p2q/t/mhvuOj/UeLlV6566GD/guowlr0hHxClI0W9m7MWYkL1F0hLo+0Aexs9HSPCtR1SXQ0TD3MMKrXZajbiQ==", + "dependencies": { + "file-uri-to-path": "1.0.0" + } + }, + "node_modules/bluebird": { + "version": "3.7.2", + "resolved": "https://registry.npmjs.org/bluebird/-/bluebird-3.7.2.tgz", + "integrity": "sha512-XpNj6GDQzdfW+r2Wnn7xiSAd7TM3jzkxGXBGTtWKuSXv1xUV+azxAm8jdWZN06QTQk+2N2XB9jRDkvbmQmcRtg==", + "dev": true + }, + "node_modules/brace-expansion": { + "version": "1.1.11", + "resolved": "https://registry.npmjs.org/brace-expansion/-/brace-expansion-1.1.11.tgz", + "integrity": "sha512-iCuPHDFgrHX7H2vEI/5xpz07zSHB00TpugqhmYtVmMO6518mCuRMoOYFldEBl0g187ufozdaHgWKcYFb61qGiA==", + "dependencies": { + "balanced-match": "^1.0.0", + "concat-map": "0.0.1" + } + }, + "node_modules/braces": { + "version": "3.0.2", + "resolved": "https://registry.npmjs.org/braces/-/braces-3.0.2.tgz", + "integrity": "sha512-b8um+L1RzM3WDSzvhm6gIz1yfTbBt6YTlcEKAvsmqCZZFw46z626lVj9j1yEPW33H5H+lBQpZMP1k8l+78Ha0A==", + "dev": true, + "dependencies": { + "fill-range": "^7.0.1" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/browser-stdout": { + "version": "1.3.1", + "resolved": "https://registry.npmjs.org/browser-stdout/-/browser-stdout-1.3.1.tgz", + "integrity": "sha512-qhAVI1+Av2X7qelOfAIYwXONood6XlZE/fXaBSmW/T5SzLAmCgzi+eiWE7fUvbHaeNBQH13UftjpXxsfLkMpgw==", + "dev": true + }, + "node_modules/browserslist": { + "version": "4.22.2", + "resolved": "https://registry.npmjs.org/browserslist/-/browserslist-4.22.2.tgz", + "integrity": "sha512-0UgcrvQmBDvZHFGdYUehrCNIazki7/lUP3kkoi/r3YB2amZbFM9J43ZRkJTXBUZK4gmx56+Sqk9+Vs9mwZx9+A==", + "dev": true, + "funding": [ + { + "type": "opencollective", + "url": "https://opencollective.com/browserslist" + }, + { + "type": "tidelift", + "url": "https://tidelift.com/funding/github/npm/browserslist" + }, + { + "type": "github", + "url": "https://github.com/sponsors/ai" + } + ], + "dependencies": { + "caniuse-lite": "^1.0.30001565", + "electron-to-chromium": "^1.4.601", + "node-releases": "^2.0.14", + "update-browserslist-db": "^1.0.13" + }, + "bin": { + "browserslist": "cli.js" + }, + "engines": { + "node": "^6 || ^7 || ^8 || ^9 || ^10 || ^11 || ^12 || >=13.7" + } + }, + "node_modules/bser": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/bser/-/bser-2.1.1.tgz", + "integrity": "sha512-gQxTNE/GAfIIrmHLUE3oJyp5FO6HRBfhjnw4/wMmA63ZGDJnWBmgY/lyQBpnDUkGmAhbSe39tx2d/iTOAfglwQ==", + "dev": true, + "dependencies": { + "node-int64": "^0.4.0" + } + }, + "node_modules/buffer-from": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/buffer-from/-/buffer-from-1.1.2.tgz", + "integrity": "sha512-E+XQCRwSbaaiChtv6k6Dwgc+bx+Bs6vuKJHHl5kox/BaKbhiXzqQOwK4cO22yElGp2OCmjwVhT3HmxgyPGnJfQ==", + "dev": true + }, + "node_modules/cacache": { + "version": "16.1.3", + "resolved": "https://registry.npmjs.org/cacache/-/cacache-16.1.3.tgz", + "integrity": "sha512-/+Emcj9DAXxX4cwlLmRI9c166RuL3w30zp4R7Joiv2cQTtTtA+jeuCAjH3ZlGnYS3tKENSrKhAzVVP9GVyzeYQ==", + "dev": true, + "dependencies": { + "@npmcli/fs": "^2.1.0", + "@npmcli/move-file": "^2.0.0", + "chownr": "^2.0.0", + "fs-minipass": "^2.1.0", + "glob": "^8.0.1", + "infer-owner": "^1.0.4", + "lru-cache": "^7.7.1", + "minipass": "^3.1.6", + "minipass-collect": "^1.0.2", + "minipass-flush": "^1.0.5", + "minipass-pipeline": "^1.2.4", + "mkdirp": "^1.0.4", + "p-map": "^4.0.0", + "promise-inflight": "^1.0.1", + "rimraf": "^3.0.2", + "ssri": "^9.0.0", + "tar": "^6.1.11", + "unique-filename": "^2.0.0" + }, + "engines": { + "node": "^12.13.0 || ^14.15.0 || >=16.0.0" + } + }, + "node_modules/cacache/node_modules/brace-expansion": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/brace-expansion/-/brace-expansion-2.0.1.tgz", + "integrity": "sha512-XnAIvQ8eM+kC6aULx6wuQiwVsnzsi9d3WxzV3FpWTGA19F621kwdbsAcFKXgKUHZWsy+mY6iL1sHTxWEFCytDA==", + "dev": true, + "dependencies": { + "balanced-match": "^1.0.0" + } + }, + "node_modules/cacache/node_modules/glob": { + "version": "8.1.0", + "resolved": "https://registry.npmjs.org/glob/-/glob-8.1.0.tgz", + "integrity": "sha512-r8hpEjiQEYlF2QU0df3dS+nxxSIreXQS1qRhMJM0Q5NDdR386C7jb7Hwwod8Fgiuex+k0GFjgft18yvxm5XoCQ==", + "dev": true, + "dependencies": { + "fs.realpath": "^1.0.0", + "inflight": "^1.0.4", + "inherits": "2", + "minimatch": "^5.0.1", + "once": "^1.3.0" + }, + "engines": { + "node": ">=12" + }, + "funding": { + "url": "https://github.com/sponsors/isaacs" + } + }, + "node_modules/cacache/node_modules/lru-cache": { + "version": "7.18.3", + "resolved": "https://registry.npmjs.org/lru-cache/-/lru-cache-7.18.3.tgz", + "integrity": "sha512-jumlc0BIUrS3qJGgIkWZsyfAM7NCWiBcCDhnd+3NNM5KbBmLTgHVfWBcg6W+rLUsIpzpERPsvwUP7CckAQSOoA==", + "dev": true, + "engines": { + "node": ">=12" + } + }, + "node_modules/cacache/node_modules/minimatch": { + "version": "5.1.6", + "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-5.1.6.tgz", + "integrity": "sha512-lKwV/1brpG6mBUFHtb7NUmtABCb2WZZmm2wNiOA5hAb8VdCS4B3dtMWyvcoViccwAW/COERjXLt0zP1zXUN26g==", + "dev": true, + "dependencies": { + "brace-expansion": "^2.0.1" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/callsites": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/callsites/-/callsites-3.1.0.tgz", + "integrity": "sha512-P8BjAsXvZS+VIDUI11hHCQEv74YT67YUi5JJFNWIqL235sBmjX4+qx9Muvls5ivyNENctx46xQLQ3aTuE7ssaQ==", + "dev": true, + "engines": { + "node": ">=6" + } + }, + "node_modules/camelcase": { + "version": "5.3.1", + "resolved": "https://registry.npmjs.org/camelcase/-/camelcase-5.3.1.tgz", + "integrity": "sha512-L28STB170nwWS63UjtlEOE3dldQApaJXZkOI1uMFfzf3rRuPegHaHesyee+YxQ+W6SvRDQV6UrdOdRiR153wJg==", + "dev": true, + "engines": { + "node": ">=6" + } + }, + "node_modules/caniuse-lite": { + "version": "1.0.30001566", + "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001566.tgz", + "integrity": "sha512-ggIhCsTxmITBAMmK8yZjEhCO5/47jKXPu6Dha/wuCS4JePVL+3uiDEBuhu2aIoT+bqTOR8L76Ip1ARL9xYsEJA==", + "dev": true, + "funding": [ + { + "type": "opencollective", + "url": "https://opencollective.com/browserslist" + }, + { + "type": "tidelift", + "url": "https://tidelift.com/funding/github/npm/caniuse-lite" + }, + { + "type": "github", + "url": "https://github.com/sponsors/ai" + } + ] + }, + "node_modules/catharsis": { + "version": "0.9.0", + "resolved": "https://registry.npmjs.org/catharsis/-/catharsis-0.9.0.tgz", + "integrity": "sha512-prMTQVpcns/tzFgFVkVp6ak6RykZyWb3gu8ckUpd6YkTlacOd3DXGJjIpD4Q6zJirizvaiAjSSHlOsA+6sNh2A==", + "dev": true, + "dependencies": { + "lodash": "^4.17.15" + }, + "engines": { + "node": ">= 10" + } + }, + "node_modules/chalk": { + "version": "4.1.2", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-4.1.2.tgz", + "integrity": "sha512-oKnbhFyRIXpUuez8iBMmyEa4nbj4IOQyuhc/wy9kY7/WVPcwIO9VA668Pu8RkO7+0G76SLROeyw9CpQ061i4mA==", + "dev": true, + "dependencies": { + "ansi-styles": "^4.1.0", + "supports-color": "^7.1.0" + }, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/chalk/chalk?sponsor=1" + } + }, + "node_modules/char-regex": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/char-regex/-/char-regex-1.0.2.tgz", + "integrity": "sha512-kWWXztvZ5SBQV+eRgKFeh8q5sLuZY2+8WUIzlxWVTg+oGwY14qylx1KbKzHd8P6ZYkAg0xyIDU9JMHhyJMZ1jw==", + "dev": true, + "engines": { + "node": ">=10" + } + }, + "node_modules/chokidar": { + "version": "3.5.3", + "resolved": "https://registry.npmjs.org/chokidar/-/chokidar-3.5.3.tgz", + "integrity": "sha512-Dr3sfKRP6oTcjf2JmUmFJfeVMvXBdegxB0iVQ5eb2V10uFJUCAS8OByZdVAyVb8xXNz3GjjTgj9kLWsZTqE6kw==", + "dev": true, + "funding": [ + { + "type": "individual", + "url": "https://paulmillr.com/funding/" + } + ], + "dependencies": { + "anymatch": "~3.1.2", + "braces": "~3.0.2", + "glob-parent": "~5.1.2", + "is-binary-path": "~2.1.0", + "is-glob": "~4.0.1", + "normalize-path": "~3.0.0", + "readdirp": "~3.6.0" + }, + "engines": { + "node": ">= 8.10.0" + }, + "optionalDependencies": { + "fsevents": "~2.3.2" + } + }, + "node_modules/chownr": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/chownr/-/chownr-2.0.0.tgz", + "integrity": "sha512-bIomtDF5KGpdogkLd9VspvFzk9KfpyyGlS8YFVZl7TGPBHL5snIOnxeshwVgPteQ9b4Eydl+pVbIyE1DcvCWgQ==", + "engines": { + "node": ">=10" + } + }, + "node_modules/ci-info": { + "version": "3.9.0", + "resolved": "https://registry.npmjs.org/ci-info/-/ci-info-3.9.0.tgz", + "integrity": "sha512-NIxF55hv4nSqQswkAeiOi1r83xy8JldOFDTWiug55KBu9Jnblncd2U6ViHmYgHf01TPZS77NJBhBMKdWj9HQMQ==", + "dev": true, + "funding": [ + { + "type": "github", + "url": "https://github.com/sponsors/sibiraj-s" + } + ], + "engines": { + "node": ">=8" + } + }, + "node_modules/cjs-module-lexer": { + "version": "1.2.3", + "resolved": "https://registry.npmjs.org/cjs-module-lexer/-/cjs-module-lexer-1.2.3.tgz", + "integrity": "sha512-0TNiGstbQmCFwt4akjjBg5pLRTSyj/PkWQ1ZoO2zntmg9yLqSRxwEa4iCfQLGjqhiqBfOJa7W/E8wfGrTDmlZQ==", + "dev": true + }, + "node_modules/clean-stack": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/clean-stack/-/clean-stack-2.2.0.tgz", + "integrity": "sha512-4diC9HaTE+KRAMWhDhrGOECgWZxoevMc5TlkObMqNSsVU62PYzXZ/SMTjzyGAFF1YusgxGcSWTEXBhp0CPwQ1A==", + "dev": true, + "engines": { + "node": ">=6" + } + }, + "node_modules/cli": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/cli/-/cli-1.0.1.tgz", + "integrity": "sha512-41U72MB56TfUMGndAKK8vJ78eooOD4Z5NOL4xEfjc0c23s+6EYKXlXsmACBVclLP1yOfWCgEganVzddVrSNoTg==", + "dev": true, + "dependencies": { + "exit": "0.1.2", + "glob": "^7.1.1" + }, + "engines": { + "node": ">=0.2.5" + } + }, + "node_modules/cliui": { + "version": "8.0.1", + "resolved": "https://registry.npmjs.org/cliui/-/cliui-8.0.1.tgz", + "integrity": "sha512-BSeNnyus75C4//NQ9gQt1/csTXyo/8Sb+afLAkzAptFuMsod9HFokGNudZpi/oQV73hnVK+sR+5PVRMd+Dr7YQ==", + "dev": true, + "dependencies": { + "string-width": "^4.2.0", + "strip-ansi": "^6.0.1", + "wrap-ansi": "^7.0.0" + }, + "engines": { + "node": ">=12" + } + }, + "node_modules/co": { + "version": "4.6.0", + "resolved": "https://registry.npmjs.org/co/-/co-4.6.0.tgz", + "integrity": "sha512-QVb0dM5HvG+uaxitm8wONl7jltx8dqhfU33DcqtOZcLSVIKSDDLDi7+0LbAKiyI8hD9u42m2YxXSkMGWThaecQ==", + "dev": true, + "engines": { + "iojs": ">= 1.0.0", + "node": ">= 0.12.0" + } + }, + "node_modules/collect-v8-coverage": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/collect-v8-coverage/-/collect-v8-coverage-1.0.2.tgz", + "integrity": "sha512-lHl4d5/ONEbLlJvaJNtsF/Lz+WvB07u2ycqTYbdrq7UypDXailES4valYb2eWiJFxZlVmpGekfqoxQhzyFdT4Q==", + "dev": true + }, + "node_modules/color-convert": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/color-convert/-/color-convert-2.0.1.tgz", + "integrity": "sha512-RRECPsj7iu/xb5oKYcsFHSppFNnsj/52OVTRKb4zP5onXwVF3zVmmToNcOfGC+CRDpfK/U584fMg38ZHCaElKQ==", + "dev": true, + "dependencies": { + "color-name": "~1.1.4" + }, + "engines": { + "node": ">=7.0.0" + } + }, + "node_modules/color-name": { + "version": "1.1.4", + "resolved": "https://registry.npmjs.org/color-name/-/color-name-1.1.4.tgz", + "integrity": "sha512-dOy+3AuW3a2wNbZHIuMZpTcgjGuLU/uBL/ubcZF9OXbDo8ff4O8yVp5Bf0efS8uEoYo5q4Fx7dY9OgQGXgAsQA==", + "dev": true + }, + "node_modules/color-support": { + "version": "1.1.3", + "resolved": "https://registry.npmjs.org/color-support/-/color-support-1.1.3.tgz", + "integrity": "sha512-qiBjkpbMLO/HL68y+lh4q0/O1MZFj2RX6X/KmMa3+gJD3z+WwI1ZzDHysvqHGS3mP6mznPckpXmw1nI9cJjyRg==", + "bin": { + "color-support": "bin.js" + } + }, + "node_modules/concat-map": { + "version": "0.0.1", + "resolved": "https://registry.npmjs.org/concat-map/-/concat-map-0.0.1.tgz", + "integrity": "sha512-/Srv4dswyQNBfohGpz9o6Yb3Gz3SrUDqBH5rTuhGR7ahtlbYKnVxw2bCFMRljaA7EXHaXZ8wsHdodFvbkhKmqg==" + }, + "node_modules/console-browserify": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/console-browserify/-/console-browserify-1.1.0.tgz", + "integrity": "sha512-duS7VP5pvfsNLDvL1O4VOEbw37AI3A4ZUQYemvDlnpGrNu9tprR7BYWpDYwC0Xia0Zxz5ZupdiIrUp0GH1aXfg==", + "dev": true, + "dependencies": { + "date-now": "^0.1.4" + } + }, + "node_modules/console-control-strings": { + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/console-control-strings/-/console-control-strings-1.1.0.tgz", + "integrity": "sha512-ty/fTekppD2fIwRvnZAVdeOiGd1c7YXEixbgJTNzqcxJWKQnjJ/V1bNEEE6hygpM3WjwHFUVK6HTjWSzV4a8sQ==" + }, + "node_modules/convert-source-map": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/convert-source-map/-/convert-source-map-2.0.0.tgz", + "integrity": "sha512-Kvp459HrV2FEJ1CAsi1Ku+MY3kasH19TFykTz2xWmMeq6bk2NU3XXvfJ+Q61m0xktWwt+1HSYf3JZsTms3aRJg==", + "dev": true + }, + "node_modules/core-util-is": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/core-util-is/-/core-util-is-1.0.3.tgz", + "integrity": "sha512-ZQBvi1DcpJ4GDqanjucZ2Hj3wEO5pZDS89BWbkcrvdxksJorwUDDZamX9ldFkp9aw2lmBDLgkObEA4DWNJ9FYQ==", + "dev": true + }, + "node_modules/create-jest": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/create-jest/-/create-jest-29.7.0.tgz", + "integrity": "sha512-Adz2bdH0Vq3F53KEMJOoftQFutWCukm6J24wbPWRO4k1kMY7gS7ds/uoJkNuV8wDCtWWnuwGcJwpWcih+zEW1Q==", + "dev": true, + "dependencies": { + "@jest/types": "^29.6.3", + "chalk": "^4.0.0", + "exit": "^0.1.2", + "graceful-fs": "^4.2.9", + "jest-config": "^29.7.0", + "jest-util": "^29.7.0", + "prompts": "^2.0.1" + }, + "bin": { + "create-jest": "bin/create-jest.js" + }, + "engines": { + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + } + }, + "node_modules/cross-spawn": { + "version": "7.0.3", + "resolved": "https://registry.npmjs.org/cross-spawn/-/cross-spawn-7.0.3.tgz", + "integrity": "sha512-iRDPJKUPVEND7dHPO8rkbOnPpyDygcDFtWjpeWNCgy8WP2rXcxXL8TskReQl6OrB2G7+UJrags1q15Fudc7G6w==", + "dev": true, + "dependencies": { + "path-key": "^3.1.0", + "shebang-command": "^2.0.0", + "which": "^2.0.1" + }, + "engines": { + "node": ">= 8" + } + }, + "node_modules/date-now": { + "version": "0.1.4", + "resolved": "https://registry.npmjs.org/date-now/-/date-now-0.1.4.tgz", + "integrity": "sha512-AsElvov3LoNB7tf5k37H2jYSB+ZZPMT5sG2QjJCcdlV5chIv6htBUBUui2IKRjgtKAKtCBN7Zbwa+MtwLjSeNw==", + "dev": true + }, + "node_modules/debug": { + "version": "4.3.4", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", + "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", + "dependencies": { + "ms": "2.1.2" + }, + "engines": { + "node": ">=6.0" + }, + "peerDependenciesMeta": { + "supports-color": { + "optional": true + } + } + }, + "node_modules/decamelize": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/decamelize/-/decamelize-4.0.0.tgz", + "integrity": "sha512-9iE1PgSik9HeIIw2JO94IidnE3eBoQrFJ3w7sFuzSX4DpmZ3v5sZpUiV5Swcf6mQEF+Y0ru8Neo+p+nyh2J+hQ==", + "dev": true, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/dedent": { + "version": "1.5.1", + "resolved": "https://registry.npmjs.org/dedent/-/dedent-1.5.1.tgz", + "integrity": "sha512-+LxW+KLWxu3HW3M2w2ympwtqPrqYRzU8fqi6Fhd18fBALe15blJPI/I4+UHveMVG6lJqB4JNd4UG0S5cnVHwIg==", + "dev": true, + "peerDependencies": { + "babel-plugin-macros": "^3.1.0" + }, + "peerDependenciesMeta": { + "babel-plugin-macros": { + "optional": true + } + } + }, + "node_modules/deepmerge": { + "version": "4.3.1", + "resolved": "https://registry.npmjs.org/deepmerge/-/deepmerge-4.3.1.tgz", + "integrity": "sha512-3sUqbMEc77XqpdNO7FRyRog+eW3ph+GYCbj+rK+uYyRMuwsVy0rMiVtPn+QJlKFvWP/1PYpapqYn0Me2knFn+A==", + "dev": true, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/delegates": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/delegates/-/delegates-1.0.0.tgz", + "integrity": "sha512-bd2L678uiWATM6m5Z1VzNCErI3jiGzt6HGY8OVICs40JQq/HALfbyNJmp0UDakEY4pMMaN0Ly5om/B1VI/+xfQ==" + }, + "node_modules/detect-libc": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/detect-libc/-/detect-libc-2.0.2.tgz", + "integrity": "sha512-UX6sGumvvqSaXgdKGUsgZWqcUyIXZ/vZTrlRT/iobiKhGL0zL4d3osHj3uqllWJK+i+sixDS/3COVEOFbupFyw==", + "engines": { + "node": ">=8" + } + }, + "node_modules/detect-newline": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/detect-newline/-/detect-newline-3.1.0.tgz", + "integrity": "sha512-TLz+x/vEXm/Y7P7wn1EJFNLxYpUD4TgMosxY6fAVJUnJMbupHBOncxyWUG9OpTaH9EBD7uFI5LfEgmMOc54DsA==", + "dev": true, + "engines": { + "node": ">=8" + } + }, + "node_modules/diff": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/diff/-/diff-5.0.0.tgz", + "integrity": "sha512-/VTCrvm5Z0JGty/BWHljh+BAiw3IK+2j87NGMu8Nwc/f48WoDAC395uomO9ZD117ZOBaHmkX1oyLvkVM/aIT3w==", + "dev": true, + "engines": { + "node": ">=0.3.1" + } + }, + "node_modules/diff-sequences": { + "version": "29.6.3", + "resolved": "https://registry.npmjs.org/diff-sequences/-/diff-sequences-29.6.3.tgz", + "integrity": "sha512-EjePK1srD3P08o2j4f0ExnylqRs5B9tJjcp9t1krH2qRi8CCdsYfwe9JgSLurFBWwq4uOlipzfk5fHNvwFKr8Q==", + "dev": true, + "engines": { + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + } + }, + "node_modules/dom-serializer": { + "version": "0.2.2", + "resolved": "https://registry.npmjs.org/dom-serializer/-/dom-serializer-0.2.2.tgz", + "integrity": "sha512-2/xPb3ORsQ42nHYiSunXkDjPLBaEj/xTwUO4B7XCZQTRk7EBtTOPaygh10YAAh2OI1Qrp6NWfpAhzswj0ydt9g==", + "dev": true, + "dependencies": { + "domelementtype": "^2.0.1", + "entities": "^2.0.0" } }, "node_modules/dom-serializer/node_modules/domelementtype": { @@ -741,503 +2075,1277 @@ "resolved": "https://registry.npmjs.org/domelementtype/-/domelementtype-2.3.0.tgz", "integrity": "sha512-OLETBj6w0OsagBwdXnPdN0cnMfF9opN69co+7ZrbfPGrdpPVNBUj02spi6B1N7wChLQiPn4CSH/zJvXw56gmHw==", "dev": true, - "funding": [ - { - "type": "github", - "url": "https://github.com/sponsors/fb55" - } - ] + "funding": [ + { + "type": "github", + "url": "https://github.com/sponsors/fb55" + } + ] + }, + "node_modules/dom-serializer/node_modules/entities": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/entities/-/entities-2.2.0.tgz", + "integrity": "sha512-p92if5Nz619I0w+akJrLZH0MX0Pb5DX39XOwQTtXSdQQOaYH03S1uIQp4mhOZtAXrxq4ViO67YTiLBo2638o9A==", + "dev": true, + "funding": { + "url": "https://github.com/fb55/entities?sponsor=1" + } + }, + "node_modules/domelementtype": { + "version": "1.3.1", + "resolved": "https://registry.npmjs.org/domelementtype/-/domelementtype-1.3.1.tgz", + "integrity": "sha512-BSKB+TSpMpFI/HOxCNr1O8aMOTZ8hT3pM3GQ0w/mWRmkhEDSFJkkyzz4XQsBV44BChwGkrDfMyjVD0eA2aFV3w==", + "dev": true + }, + "node_modules/domhandler": { + "version": "2.3.0", + "resolved": "https://registry.npmjs.org/domhandler/-/domhandler-2.3.0.tgz", + "integrity": "sha512-q9bUwjfp7Eif8jWxxxPSykdRZAb6GkguBGSgvvCrhI9wB71W2K/Kvv4E61CF/mcCfnVJDeDWx/Vb/uAqbDj6UQ==", + "dev": true, + "dependencies": { + "domelementtype": "1" + } + }, + "node_modules/domutils": { + "version": "1.5.1", + "resolved": "https://registry.npmjs.org/domutils/-/domutils-1.5.1.tgz", + "integrity": "sha512-gSu5Oi/I+3wDENBsOWBiRK1eoGxcywYSqg3rR960/+EfY0CF4EX1VPkgHOZ3WiS/Jg2DtliF6BhWcHlfpYUcGw==", + "dev": true, + "dependencies": { + "dom-serializer": "0", + "domelementtype": "1" + } + }, + "node_modules/electron-to-chromium": { + "version": "1.4.609", + "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.609.tgz", + "integrity": "sha512-ihiCP7PJmjoGNuLpl7TjNA8pCQWu09vGyjlPYw1Rqww4gvNuCcmvl+44G+2QyJ6S2K4o+wbTS++Xz0YN8Q9ERw==", + "dev": true + }, + "node_modules/emittery": { + "version": "0.13.1", + "resolved": "https://registry.npmjs.org/emittery/-/emittery-0.13.1.tgz", + "integrity": "sha512-DeWwawk6r5yR9jFgnDKYt4sLS0LmHJJi3ZOnb5/JdbYwj3nW+FxQnHIjhBKz8YLC7oRNPVM9NQ47I3CVx34eqQ==", + "dev": true, + "engines": { + "node": ">=12" + }, + "funding": { + "url": "https://github.com/sindresorhus/emittery?sponsor=1" + } + }, + "node_modules/emoji-regex": { + "version": "8.0.0", + "resolved": "https://registry.npmjs.org/emoji-regex/-/emoji-regex-8.0.0.tgz", + "integrity": "sha512-MSjYzcWNOA0ewAHpz0MxpYFvwg6yjy1NG3xteoqz644VCo/RPgnr1/GGt+ic3iJTzQ8Eu3TdM14SawnVUmGE6A==" + }, + "node_modules/encoding": { + "version": "0.1.13", + "resolved": "https://registry.npmjs.org/encoding/-/encoding-0.1.13.tgz", + "integrity": "sha512-ETBauow1T35Y/WZMkio9jiM0Z5xjHHmJ4XmjZOq1l/dXz3lr2sRn87nJy20RupqSh1F2m3HHPSp8ShIPQJrJ3A==", + "optional": true, + "dependencies": { + "iconv-lite": "^0.6.2" + } + }, + "node_modules/entities": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/entities/-/entities-1.0.0.tgz", + "integrity": "sha512-LbLqfXgJMmy81t+7c14mnulFHJ170cM6E+0vMXR9k/ZiZwgX8i5pNgjTCX3SO4VeUsFLV+8InixoretwU+MjBQ==", + "dev": true + }, + "node_modules/env-paths": { + "version": "2.2.1", + "resolved": "https://registry.npmjs.org/env-paths/-/env-paths-2.2.1.tgz", + "integrity": "sha512-+h1lkLKhZMTYjog1VEpJNG7NZJWcuc2DDk/qsqSTRRCOXiLjeQ1d1/udrUGhqMxUgAlwKNZ0cf2uqan5GLuS2A==", + "dev": true, + "engines": { + "node": ">=6" + } + }, + "node_modules/err-code": { + "version": "2.0.3", + "resolved": "https://registry.npmjs.org/err-code/-/err-code-2.0.3.tgz", + "integrity": "sha512-2bmlRpNKBxT/CRmPOlyISQpNj+qSeYvcym/uT0Jx2bMOlKLtSy1ZmLuVxSEKKyor/N5yhvp/ZiG1oE3DEYMSFA==", + "dev": true + }, + "node_modules/error-ex": { + "version": "1.3.2", + "resolved": "https://registry.npmjs.org/error-ex/-/error-ex-1.3.2.tgz", + "integrity": "sha512-7dFHNmqeFSEt2ZBsCriorKnn3Z2pj+fd9kmI6QoWw4//DL+icEBfc0U7qJCisqrTsKTjw4fNFy2pW9OqStD84g==", + "dev": true, + "dependencies": { + "is-arrayish": "^0.2.1" + } + }, + "node_modules/escalade": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/escalade/-/escalade-3.1.1.tgz", + "integrity": "sha512-k0er2gUkLf8O0zKJiAhmkTnJlTvINGv7ygDNPbeIsX/TJjGJZHuh9B2UxbsaEkmlEo9MfhrSzmhIlhRlI2GXnw==", + "dev": true, + "engines": { + "node": ">=6" + } + }, + "node_modules/escape-string-regexp": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/escape-string-regexp/-/escape-string-regexp-2.0.0.tgz", + "integrity": "sha512-UpzcLCXolUWcNu5HtVMHYdXJjArjsF9C0aNnquZYY4uW/Vu0miy5YoWvbV345HauVvcAUnpRuhMMcqTcGOY2+w==", + "dev": true, + "engines": { + "node": ">=8" + } + }, + "node_modules/esprima": { + "version": "4.0.1", + "resolved": "https://registry.npmjs.org/esprima/-/esprima-4.0.1.tgz", + "integrity": "sha512-eGuFFw7Upda+g4p+QHvnW0RyTX/SVeJBDM/gCtMARO0cLuT2HcEKnTPvhjV6aGeqrCB/sbNop0Kszm0jsaWU4A==", + "dev": true, + "bin": { + "esparse": "bin/esparse.js", + "esvalidate": "bin/esvalidate.js" + }, + "engines": { + "node": ">=4" + } + }, + "node_modules/execa": { + "version": "5.1.1", + "resolved": "https://registry.npmjs.org/execa/-/execa-5.1.1.tgz", + "integrity": "sha512-8uSpZZocAZRBAPIEINJj3Lo9HyGitllczc27Eh5YYojjMFMn8yHMDMaUHE2Jqfq05D/wucwI4JGURyXt1vchyg==", + "dev": true, + "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" + }, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/sindresorhus/execa?sponsor=1" + } + }, + "node_modules/exit": { + "version": "0.1.2", + "resolved": "https://registry.npmjs.org/exit/-/exit-0.1.2.tgz", + "integrity": "sha512-Zk/eNKV2zbjpKzrsQ+n1G6poVbErQxJ0LBOJXaKZ1EViLzH+hrLu9cdXI4zw9dBQJslwBEpbQ2P1oS7nDxs6jQ==", + "dev": true, + "engines": { + "node": ">= 0.8.0" + } + }, + "node_modules/expect": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/expect/-/expect-29.7.0.tgz", + "integrity": "sha512-2Zks0hf1VLFYI1kbh0I5jP3KHHyCHpkfyHBzsSXRFgl/Bg9mWYfMW8oD+PdMPlEwy5HNsR9JutYy6pMeOh61nw==", + "dev": true, + "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" + }, + "engines": { + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + } + }, + "node_modules/exponential-backoff": { + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/exponential-backoff/-/exponential-backoff-3.1.1.tgz", + "integrity": "sha512-dX7e/LHVJ6W3DE1MHWi9S1EYzDESENfLrYohG2G++ovZrYOkm4Knwa0mc1cn84xJOR4KEU0WSchhLbd0UklbHw==", + "dev": true + }, + "node_modules/fast-json-stable-stringify": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/fast-json-stable-stringify/-/fast-json-stable-stringify-2.1.0.tgz", + "integrity": "sha512-lhd/wF+Lk98HZoTCtlVraHtfh5XYijIjalXck7saUtuanSDyLMxnHhSXEDJqHxD7msR8D0uCmqlkwjCV8xvwHw==", + "dev": true + }, + "node_modules/fb-watchman": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/fb-watchman/-/fb-watchman-2.0.2.tgz", + "integrity": "sha512-p5161BqbuCaSnB8jIbzQHOlpgsPmK5rJVDfDKO91Axs5NC1uu3HRQm6wt9cd9/+GtQQIO53JdGXXoyDpTAsgYA==", + "dev": true, + "dependencies": { + "bser": "2.1.1" + } + }, + "node_modules/file-uri-to-path": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/file-uri-to-path/-/file-uri-to-path-1.0.0.tgz", + "integrity": "sha512-0Zt+s3L7Vf1biwWZ29aARiVYLx7iMGnEUl9x33fbB/j3jR81u/O2LbqK+Bm1CDSNDKVtJ/YjwY7TUd5SkeLQLw==" + }, + "node_modules/fill-range": { + "version": "7.0.1", + "resolved": "https://registry.npmjs.org/fill-range/-/fill-range-7.0.1.tgz", + "integrity": "sha512-qOo9F+dMUmC2Lcb4BbVvnKJxTPjCm+RRpe4gDuGrzkL7mEVl/djYSu2OdQ2Pa302N4oqkSg9ir6jaLWJ2USVpQ==", + "dev": true, + "dependencies": { + "to-regex-range": "^5.0.1" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/find-up": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/find-up/-/find-up-4.1.0.tgz", + "integrity": "sha512-PpOwAdQ/YlXQ2vj8a3h8IipDuYRi3wceVQQGYWxNINccq40Anw7BlsEXCMbt1Zt+OLA6Fq9suIpIWD0OsnISlw==", + "dev": true, + "dependencies": { + "locate-path": "^5.0.0", + "path-exists": "^4.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/flat": { + "version": "5.0.2", + "resolved": "https://registry.npmjs.org/flat/-/flat-5.0.2.tgz", + "integrity": "sha512-b6suED+5/3rTpUBdG1gupIl8MPFCAMA0QXwmljLhvCUKcUvdE4gWky9zpuGCcXHOsz4J9wPGNWq6OKpmIzz3hQ==", + "dev": true, + "bin": { + "flat": "cli.js" + } + }, + "node_modules/fs-minipass": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/fs-minipass/-/fs-minipass-2.1.0.tgz", + "integrity": "sha512-V/JgOLFCS+R6Vcq0slCuaeWEdNC3ouDlJMNIsacH2VtALiu9mV4LPrHc5cDl8k5aw6J8jwgWWpiTo5RYhmIzvg==", + "dependencies": { + "minipass": "^3.0.0" + }, + "engines": { + "node": ">= 8" + } + }, + "node_modules/fs.realpath": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/fs.realpath/-/fs.realpath-1.0.0.tgz", + "integrity": "sha512-OO0pH2lK6a0hZnAdau5ItzHPI6pUlvI7jMVnxUQRtw4owF2wk8lOSabtGDCTP4Ggrg2MbGnWO9X8K1t4+fGMDw==" + }, + "node_modules/fsevents": { + "version": "2.3.3", + "resolved": "https://registry.npmjs.org/fsevents/-/fsevents-2.3.3.tgz", + "integrity": "sha512-5xoDfX+fL7faATnagmWPpbFtwh/R77WmMMqqHGS65C3vvB0YHrgF+B1YmZ3441tMj5n63k0212XNoJwzlhffQw==", + "dev": true, + "hasInstallScript": true, + "optional": true, + "os": [ + "darwin" + ], + "engines": { + "node": "^8.16.0 || ^10.6.0 || >=11.0.0" + } + }, + "node_modules/function-bind": { + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/function-bind/-/function-bind-1.1.2.tgz", + "integrity": "sha512-7XHNxH7qX9xG5mIwxkhumTox/MIRNcOgDrxWsMt2pAr23WHp6MrRlN7FBSFpCpr+oVO0F744iUgR82nJMfG2SA==", + "dev": true, + "funding": { + "url": "https://github.com/sponsors/ljharb" + } + }, + "node_modules/gauge": { + "version": "3.0.2", + "resolved": "https://registry.npmjs.org/gauge/-/gauge-3.0.2.tgz", + "integrity": "sha512-+5J6MS/5XksCuXq++uFRsnUd7Ovu1XenbeuIuNRJxYWjgQbPuFhT14lAvsWfqfAmnwluf1OwMjz39HjfLPci0Q==", + "dependencies": { + "aproba": "^1.0.3 || ^2.0.0", + "color-support": "^1.1.2", + "console-control-strings": "^1.0.0", + "has-unicode": "^2.0.1", + "object-assign": "^4.1.1", + "signal-exit": "^3.0.0", + "string-width": "^4.2.3", + "strip-ansi": "^6.0.1", + "wide-align": "^1.1.2" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/gensync": { + "version": "1.0.0-beta.2", + "resolved": "https://registry.npmjs.org/gensync/-/gensync-1.0.0-beta.2.tgz", + "integrity": "sha512-3hN7NaskYvMDLQY55gnW3NQ+mesEAepTqlg+VEbj7zzqEMBVNhzcGYYeqFo/TlYz6eQiFcp1HcsCZO+nGgS8zg==", + "dev": true, + "engines": { + "node": ">=6.9.0" + } + }, + "node_modules/get-caller-file": { + "version": "2.0.5", + "resolved": "https://registry.npmjs.org/get-caller-file/-/get-caller-file-2.0.5.tgz", + "integrity": "sha512-DyFP3BM/3YHTQOCUL/w0OZHR0lpKeGrxotcHWcqNEdnltqFwXVfhEBQ94eIo34AfQpo0rGki4cyIiftY06h2Fg==", + "dev": true, + "engines": { + "node": "6.* || 8.* || >= 10.*" + } + }, + "node_modules/get-package-type": { + "version": "0.1.0", + "resolved": "https://registry.npmjs.org/get-package-type/-/get-package-type-0.1.0.tgz", + "integrity": "sha512-pjzuKtY64GYfWizNAJ0fr9VqttZkNiK2iS430LtIHzjBEr6bX8Am2zm4sW4Ro5wjWW5cAlRL1qAMTcXbjNAO2Q==", + "dev": true, + "engines": { + "node": ">=8.0.0" + } + }, + "node_modules/get-stream": { + "version": "6.0.1", + "resolved": "https://registry.npmjs.org/get-stream/-/get-stream-6.0.1.tgz", + "integrity": "sha512-ts6Wi+2j3jQjqi70w5AlN8DFnkSwC+MqmxEzdEALB2qXZYV3X/b1CTfgPLGJNMeAWxdPfU8FO1ms3NUfaHCPYg==", + "dev": true, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/glob": { + "version": "7.2.3", + "resolved": "https://registry.npmjs.org/glob/-/glob-7.2.3.tgz", + "integrity": "sha512-nFR0zLpU2YCaRxwoCJvL6UvCH2JFyFVIvwTLsIf21AuHlMskA1hhTdk+LlYJtOlYt9v6dvszD2BGRqBL+iQK9Q==", + "dependencies": { + "fs.realpath": "^1.0.0", + "inflight": "^1.0.4", + "inherits": "2", + "minimatch": "^3.1.1", + "once": "^1.3.0", + "path-is-absolute": "^1.0.0" + }, + "engines": { + "node": "*" + }, + "funding": { + "url": "https://github.com/sponsors/isaacs" + } + }, + "node_modules/glob-parent": { + "version": "5.1.2", + "resolved": "https://registry.npmjs.org/glob-parent/-/glob-parent-5.1.2.tgz", + "integrity": "sha512-AOIgSQCepiJYwP3ARnGx+5VnTu2HBYdzbGP45eLw1vr3zB3vZLeyed1sC9hnbcOc9/SrMyM5RPQrkGz4aS9Zow==", + "dev": true, + "dependencies": { + "is-glob": "^4.0.1" + }, + "engines": { + "node": ">= 6" + } + }, + "node_modules/globals": { + "version": "11.12.0", + "resolved": "https://registry.npmjs.org/globals/-/globals-11.12.0.tgz", + "integrity": "sha512-WOBp/EEGUiIsJSp7wcv/y6MO+lV9UoncWqxuFfm8eBwzWNgyfBd6Gz+IeKQ9jCmyhoH99g15M3T+QaVHFjizVA==", + "dev": true, + "engines": { + "node": ">=4" + } + }, + "node_modules/graceful-fs": { + "version": "4.2.11", + "resolved": "https://registry.npmjs.org/graceful-fs/-/graceful-fs-4.2.11.tgz", + "integrity": "sha512-RbJ5/jmFcNNCcDV5o9eTnBLJ/HszWV0P73bc+Ff4nS/rJj+YaS6IGyiOL0VoBYX+l1Wrl3k63h/KrH+nhJ0XvQ==", + "dev": true + }, + "node_modules/has-flag": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/has-flag/-/has-flag-4.0.0.tgz", + "integrity": "sha512-EykJT/Q1KjTWctppgIAgfSO0tKVuZUjhgMr17kqTumMl6Afv3EISleU7qZUzoXDFTAHTDC4NOoG/ZxU3EvlMPQ==", + "dev": true, + "engines": { + "node": ">=8" + } }, - "node_modules/dom-serializer/node_modules/entities": { - "version": "2.2.0", - "resolved": "https://registry.npmjs.org/entities/-/entities-2.2.0.tgz", - "integrity": "sha512-p92if5Nz619I0w+akJrLZH0MX0Pb5DX39XOwQTtXSdQQOaYH03S1uIQp4mhOZtAXrxq4ViO67YTiLBo2638o9A==", + "node_modules/has-unicode": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/has-unicode/-/has-unicode-2.0.1.tgz", + "integrity": "sha512-8Rf9Y83NBReMnx0gFzA8JImQACstCYWUplepDa9xprwwtmgEZUF0h/i5xSA625zB/I37EtrswSST6OXxwaaIJQ==" + }, + "node_modules/hasown": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/hasown/-/hasown-2.0.0.tgz", + "integrity": "sha512-vUptKVTpIJhcczKBbgnS+RtcuYMB8+oNzPK2/Hp3hanz8JmpATdmmgLgSaadVREkDm+e2giHwY3ZRkyjSIDDFA==", "dev": true, - "funding": { - "url": "https://github.com/fb55/entities?sponsor=1" + "dependencies": { + "function-bind": "^1.1.2" + }, + "engines": { + "node": ">= 0.4" } }, - "node_modules/domelementtype": { - "version": "1.3.1", - "resolved": "https://registry.npmjs.org/domelementtype/-/domelementtype-1.3.1.tgz", - "integrity": "sha512-BSKB+TSpMpFI/HOxCNr1O8aMOTZ8hT3pM3GQ0w/mWRmkhEDSFJkkyzz4XQsBV44BChwGkrDfMyjVD0eA2aFV3w==", + "node_modules/he": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/he/-/he-1.2.0.tgz", + "integrity": "sha512-F/1DnUGPopORZi0ni+CvrCgHQ5FyEAHRLSApuYWMmrbSwoN2Mn/7k+Gl38gJnR7yyDZk6WLXwiGod1JOWNDKGw==", + "dev": true, + "bin": { + "he": "bin/he" + } + }, + "node_modules/html-escaper": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/html-escaper/-/html-escaper-2.0.2.tgz", + "integrity": "sha512-H2iMtd0I4Mt5eYiapRdIDjp+XzelXQ0tFE4JS7YFwFevXXMmOp9myNrUvCg0D6ws8iqkRPBfKHgbwig1SmlLfg==", "dev": true }, - "node_modules/domhandler": { - "version": "2.3.0", - "resolved": "https://registry.npmjs.org/domhandler/-/domhandler-2.3.0.tgz", - "integrity": "sha512-q9bUwjfp7Eif8jWxxxPSykdRZAb6GkguBGSgvvCrhI9wB71W2K/Kvv4E61CF/mcCfnVJDeDWx/Vb/uAqbDj6UQ==", + "node_modules/htmlparser2": { + "version": "3.8.3", + "resolved": "https://registry.npmjs.org/htmlparser2/-/htmlparser2-3.8.3.tgz", + "integrity": "sha512-hBxEg3CYXe+rPIua8ETe7tmG3XDn9B0edOE/e9wH2nLczxzgdu0m0aNHY+5wFZiviLWLdANPJTssa92dMcXQ5Q==", "dev": true, "dependencies": { - "domelementtype": "1" + "domelementtype": "1", + "domhandler": "2.3", + "domutils": "1.5", + "entities": "1.0", + "readable-stream": "1.1" } }, - "node_modules/domutils": { - "version": "1.5.1", - "resolved": "https://registry.npmjs.org/domutils/-/domutils-1.5.1.tgz", - "integrity": "sha512-gSu5Oi/I+3wDENBsOWBiRK1eoGxcywYSqg3rR960/+EfY0CF4EX1VPkgHOZ3WiS/Jg2DtliF6BhWcHlfpYUcGw==", + "node_modules/http-cache-semantics": { + "version": "4.1.1", + "resolved": "https://registry.npmjs.org/http-cache-semantics/-/http-cache-semantics-4.1.1.tgz", + "integrity": "sha512-er295DKPVsV82j5kw1Gjt+ADA/XYHsajl82cGNQG2eyoPkvgUhX+nDIyelzhIWbbsXP39EHcI6l5tYs2FYqYXQ==", + "dev": true + }, + "node_modules/http-proxy-agent": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/http-proxy-agent/-/http-proxy-agent-5.0.0.tgz", + "integrity": "sha512-n2hY8YdoRE1i7r6M0w9DIw5GgZN0G25P8zLCRQ8rjXtTU3vsNFBI/vWK/UIeE6g5MUUz6avwAPXmL6Fy9D/90w==", "dev": true, "dependencies": { - "dom-serializer": "0", - "domelementtype": "1" + "@tootallnate/once": "2", + "agent-base": "6", + "debug": "4" + }, + "engines": { + "node": ">= 6" } }, - "node_modules/eastasianwidth": { - "version": "0.2.0", - "resolved": "https://registry.npmjs.org/eastasianwidth/-/eastasianwidth-0.2.0.tgz", - "integrity": "sha512-I88TYZWc9XiYHRQ4/3c5rjjfgkjhLyW2luGIheGERbNQ6OY7yTybanSpDXZa8y7VUP9YmDcYa+eyq4ca7iLqWA==", - "dev": true + "node_modules/https-proxy-agent": { + "version": "5.0.1", + "resolved": "https://registry.npmjs.org/https-proxy-agent/-/https-proxy-agent-5.0.1.tgz", + "integrity": "sha512-dFcAjpTQFgoLMzC2VwU+C/CbS7uRL0lWmxDITmqm7C+7F0Odmj6s9l6alZc6AELXhrnggM2CeWSXHGOdX2YtwA==", + "dependencies": { + "agent-base": "6", + "debug": "4" + }, + "engines": { + "node": ">= 6" + } }, - "node_modules/emoji-regex": { - "version": "8.0.0", - "resolved": "https://registry.npmjs.org/emoji-regex/-/emoji-regex-8.0.0.tgz", - "integrity": "sha512-MSjYzcWNOA0ewAHpz0MxpYFvwg6yjy1NG3xteoqz644VCo/RPgnr1/GGt+ic3iJTzQ8Eu3TdM14SawnVUmGE6A==" + "node_modules/human-signals": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/human-signals/-/human-signals-2.1.0.tgz", + "integrity": "sha512-B4FFZ6q/T2jhhksgkbEW3HBvWIfDW85snkQgawt07S7J5QXTk6BkNV+0yAeZrM5QpMAdYlocGoljn0sJ/WQkFw==", + "dev": true, + "engines": { + "node": ">=10.17.0" + } }, - "node_modules/encoding": { - "version": "0.1.13", - "resolved": "https://registry.npmjs.org/encoding/-/encoding-0.1.13.tgz", - "integrity": "sha512-ETBauow1T35Y/WZMkio9jiM0Z5xjHHmJ4XmjZOq1l/dXz3lr2sRn87nJy20RupqSh1F2m3HHPSp8ShIPQJrJ3A==", + "node_modules/humanize-ms": { + "version": "1.2.1", + "resolved": "https://registry.npmjs.org/humanize-ms/-/humanize-ms-1.2.1.tgz", + "integrity": "sha512-Fl70vYtsAFb/C06PTS9dZBo7ihau+Tu/DNCk/OyHhea07S+aeMWpFFkUaXRa8fI+ScZbEI8dfSxwY7gxZ9SAVQ==", + "dev": true, + "dependencies": { + "ms": "^2.0.0" + } + }, + "node_modules/iconv-lite": { + "version": "0.6.3", + "resolved": "https://registry.npmjs.org/iconv-lite/-/iconv-lite-0.6.3.tgz", + "integrity": "sha512-4fCk79wshMdzMp2rH06qWrJE4iolqLhCUH+OiuIgU++RB0+94NlDL81atO7GX55uUKueo0txHNtvEyI6D7WdMw==", "optional": true, "dependencies": { - "iconv-lite": "^0.6.2" + "safer-buffer": ">= 2.1.2 < 3.0.0" + }, + "engines": { + "node": ">=0.10.0" } }, - "node_modules/entities": { - "version": "1.0.0", - "resolved": "https://registry.npmjs.org/entities/-/entities-1.0.0.tgz", - "integrity": "sha512-LbLqfXgJMmy81t+7c14mnulFHJ170cM6E+0vMXR9k/ZiZwgX8i5pNgjTCX3SO4VeUsFLV+8InixoretwU+MjBQ==", + "node_modules/import-local": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/import-local/-/import-local-3.1.0.tgz", + "integrity": "sha512-ASB07uLtnDs1o6EHjKpX34BKYDSqnFerfTOJL2HvMqF70LnxpjkzDB8J44oT9pu4AMPkQwf8jl6szgvNd2tRIg==", + "dev": true, + "dependencies": { + "pkg-dir": "^4.2.0", + "resolve-cwd": "^3.0.0" + }, + "bin": { + "import-local-fixture": "fixtures/cli.js" + }, + "engines": { + "node": ">=8" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/imurmurhash": { + "version": "0.1.4", + "resolved": "https://registry.npmjs.org/imurmurhash/-/imurmurhash-0.1.4.tgz", + "integrity": "sha512-JmXMZ6wuvDmLiHEml9ykzqO6lwFbof0GG4IkcGaENdCRDDmMVnny7s5HsIgHCbaq0w2MyPhDqkhTUgS2LU2PHA==", + "dev": true, + "engines": { + "node": ">=0.8.19" + } + }, + "node_modules/indent-string": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/indent-string/-/indent-string-4.0.0.tgz", + "integrity": "sha512-EdDDZu4A2OyIK7Lr/2zG+w5jmbuk1DVBnEwREQvBzspBJkCEbRa8GxU1lghYcaGJCnRWibjDXlq779X1/y5xwg==", + "dev": true, + "engines": { + "node": ">=8" + } + }, + "node_modules/infer-owner": { + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/infer-owner/-/infer-owner-1.0.4.tgz", + "integrity": "sha512-IClj+Xz94+d7irH5qRyfJonOdfTzuDaifE6ZPWfx0N0+/ATZCbuTPq2prFl526urkQd90WyUKIh1DfBQ2hMz9A==", "dev": true }, - "node_modules/env-paths": { - "version": "2.2.1", - "resolved": "https://registry.npmjs.org/env-paths/-/env-paths-2.2.1.tgz", - "integrity": "sha512-+h1lkLKhZMTYjog1VEpJNG7NZJWcuc2DDk/qsqSTRRCOXiLjeQ1d1/udrUGhqMxUgAlwKNZ0cf2uqan5GLuS2A==", + "node_modules/inflight": { + "version": "1.0.6", + "resolved": "https://registry.npmjs.org/inflight/-/inflight-1.0.6.tgz", + "integrity": "sha512-k92I/b08q4wvFscXCLvqfsHCrjrF7yiXsQuIVvVE7N82W3+aqpzuUdBbfhWcy/FZR3/4IgflMgKLOsvPDrGCJA==", + "dependencies": { + "once": "^1.3.0", + "wrappy": "1" + } + }, + "node_modules/inherits": { + "version": "2.0.4", + "resolved": "https://registry.npmjs.org/inherits/-/inherits-2.0.4.tgz", + "integrity": "sha512-k/vGaX4/Yla3WzyMCvTQOXYeIHvqOKtnqBduzTHpzpQZzAskKMhZ2K+EnBiSM9zGSoIFeMpXKxa4dYeZIQqewQ==" + }, + "node_modules/ip": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/ip/-/ip-2.0.0.tgz", + "integrity": "sha512-WKa+XuLG1A1R0UWhl2+1XQSi+fZWMsYKffMZTTYsiZaUD8k2yDAj5atimTUD2TZkyCkNEeYE5NhFZmupOGtjYQ==", + "dev": true + }, + "node_modules/is-arrayish": { + "version": "0.2.1", + "resolved": "https://registry.npmjs.org/is-arrayish/-/is-arrayish-0.2.1.tgz", + "integrity": "sha512-zz06S8t0ozoDXMG+ube26zeCTNXcKIPJZJi8hBrF4idCLms4CG9QtK7qBl1boi5ODzFpjswb5JPmHCbMpjaYzg==", + "dev": true + }, + "node_modules/is-binary-path": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/is-binary-path/-/is-binary-path-2.1.0.tgz", + "integrity": "sha512-ZMERYes6pDydyuGidse7OsHxtbI7WVeUEozgR/g7rd0xUimYNlvZRE/K2MgZTjWy725IfelLeVcEM97mmtRGXw==", + "dev": true, + "dependencies": { + "binary-extensions": "^2.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/is-core-module": { + "version": "2.13.1", + "resolved": "https://registry.npmjs.org/is-core-module/-/is-core-module-2.13.1.tgz", + "integrity": "sha512-hHrIjvZsftOsvKSn2TRYl63zvxsgE0K+0mYMoH6gD4omR5IWB2KynivBQczo3+wF1cCkjzvptnI9Q0sPU66ilw==", + "dev": true, + "dependencies": { + "hasown": "^2.0.0" + }, + "funding": { + "url": "https://github.com/sponsors/ljharb" + } + }, + "node_modules/is-extglob": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/is-extglob/-/is-extglob-2.1.1.tgz", + "integrity": "sha512-SbKbANkN603Vi4jEZv49LeVJMn4yGwsbzZworEoyEiutsN3nJYdbO36zfhGJ6QEDpOZIFkDtnq5JRxmvl3jsoQ==", + "dev": true, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/is-fullwidth-code-point": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/is-fullwidth-code-point/-/is-fullwidth-code-point-3.0.0.tgz", + "integrity": "sha512-zymm5+u+sCsSWyD9qNaejV3DFvhCKclKdizYaJUuHA83RLjb7nSuGnddCHGv0hk+KY7BMAlsWeK4Ueg6EV6XQg==", + "engines": { + "node": ">=8" + } + }, + "node_modules/is-generator-fn": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/is-generator-fn/-/is-generator-fn-2.1.0.tgz", + "integrity": "sha512-cTIB4yPYL/Grw0EaSzASzg6bBy9gqCofvWN8okThAYIxKJZC+udlRAmGbM0XLeniEJSs8uEgHPGuHSe1XsOLSQ==", "dev": true, "engines": { "node": ">=6" } }, - "node_modules/err-code": { - "version": "2.0.3", - "resolved": "https://registry.npmjs.org/err-code/-/err-code-2.0.3.tgz", - "integrity": "sha512-2bmlRpNKBxT/CRmPOlyISQpNj+qSeYvcym/uT0Jx2bMOlKLtSy1ZmLuVxSEKKyor/N5yhvp/ZiG1oE3DEYMSFA==", + "node_modules/is-glob": { + "version": "4.0.3", + "resolved": "https://registry.npmjs.org/is-glob/-/is-glob-4.0.3.tgz", + "integrity": "sha512-xelSayHH36ZgE7ZWhli7pW34hNbNl8Ojv5KVmkJD4hBdD3th8Tfk9vYasLM+mXWOZhFkgZfxhLSnrwRr4elSSg==", + "dev": true, + "dependencies": { + "is-extglob": "^2.1.1" + }, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/is-lambda": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/is-lambda/-/is-lambda-1.0.1.tgz", + "integrity": "sha512-z7CMFGNrENq5iFB9Bqo64Xk6Y9sg+epq1myIcdHaGnbMTYOxvzsEtdYqQUylB7LxfkvgrrjP32T6Ywciio9UIQ==", "dev": true }, - "node_modules/escalade": { - "version": "3.1.1", - "resolved": "https://registry.npmjs.org/escalade/-/escalade-3.1.1.tgz", - "integrity": "sha512-k0er2gUkLf8O0zKJiAhmkTnJlTvINGv7ygDNPbeIsX/TJjGJZHuh9B2UxbsaEkmlEo9MfhrSzmhIlhRlI2GXnw==", + "node_modules/is-number": { + "version": "7.0.0", + "resolved": "https://registry.npmjs.org/is-number/-/is-number-7.0.0.tgz", + "integrity": "sha512-41Cifkg6e8TylSpdtTpeLVMqvSBEVzTttHvERD741+pnZ8ANv0004MRL43QKPDlK9cGvNp6NZWZUBlbGXYxxng==", + "dev": true, + "engines": { + "node": ">=0.12.0" + } + }, + "node_modules/is-plain-obj": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/is-plain-obj/-/is-plain-obj-2.1.0.tgz", + "integrity": "sha512-YWnfyRwxL/+SsrWYfOpUtz5b3YD+nyfkHvjbcanzk8zgyO4ASD67uVMRt8k5bM4lLMDnXfriRhOpemw+NfT1eA==", "dev": true, "engines": { - "node": ">=6" + "node": ">=8" } }, - "node_modules/escape-string-regexp": { - "version": "2.0.0", - "resolved": "https://registry.npmjs.org/escape-string-regexp/-/escape-string-regexp-2.0.0.tgz", - "integrity": "sha512-UpzcLCXolUWcNu5HtVMHYdXJjArjsF9C0aNnquZYY4uW/Vu0miy5YoWvbV345HauVvcAUnpRuhMMcqTcGOY2+w==", + "node_modules/is-stream": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/is-stream/-/is-stream-2.0.1.tgz", + "integrity": "sha512-hFoiJiTl63nn+kstHGBtewWSKnQLpyb155KHheA1l39uvtO9nWIop1p3udqPcUd/xbF1VLMO4n7OI6p7RbngDg==", "dev": true, "engines": { "node": ">=8" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" } }, - "node_modules/exit": { - "version": "0.1.2", - "resolved": "https://registry.npmjs.org/exit/-/exit-0.1.2.tgz", - "integrity": "sha512-Zk/eNKV2zbjpKzrsQ+n1G6poVbErQxJ0LBOJXaKZ1EViLzH+hrLu9cdXI4zw9dBQJslwBEpbQ2P1oS7nDxs6jQ==", + "node_modules/is-unicode-supported": { + "version": "0.1.0", + "resolved": "https://registry.npmjs.org/is-unicode-supported/-/is-unicode-supported-0.1.0.tgz", + "integrity": "sha512-knxG2q4UC3u8stRGyAVJCOdxFmv5DZiRcdlIaAQXAbSfJya+OhopNotLQrstBhququ4ZpuKbDc/8S6mgXgPFPw==", "dev": true, "engines": { - "node": ">= 0.8.0" + "node": ">=10" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" } }, - "node_modules/exponential-backoff": { - "version": "3.1.1", - "resolved": "https://registry.npmjs.org/exponential-backoff/-/exponential-backoff-3.1.1.tgz", - "integrity": "sha512-dX7e/LHVJ6W3DE1MHWi9S1EYzDESENfLrYohG2G++ovZrYOkm4Knwa0mc1cn84xJOR4KEU0WSchhLbd0UklbHw==", + "node_modules/isarray": { + "version": "0.0.1", + "resolved": "https://registry.npmjs.org/isarray/-/isarray-0.0.1.tgz", + "integrity": "sha512-D2S+3GLxWH+uhrNEcoh/fnmYeP8E8/zHl644d/jdA0g2uyXvy3sb0qxotE+ne0LtccHknQzWwZEzhak7oJ0COQ==", "dev": true }, - "node_modules/file-uri-to-path": { - "version": "1.0.0", - "resolved": "https://registry.npmjs.org/file-uri-to-path/-/file-uri-to-path-1.0.0.tgz", - "integrity": "sha512-0Zt+s3L7Vf1biwWZ29aARiVYLx7iMGnEUl9x33fbB/j3jR81u/O2LbqK+Bm1CDSNDKVtJ/YjwY7TUd5SkeLQLw==" + "node_modules/isexe": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/isexe/-/isexe-2.0.0.tgz", + "integrity": "sha512-RHxMLp9lnKHGHRng9QFhRCMbYAcVpn69smSGcq3f36xjgVVWThj4qqLbTLlq7Ssj8B+fIQ1EuCEGI2lKsyQeIw==", + "dev": true }, - "node_modules/fill-range": { - "version": "7.0.1", - "resolved": "https://registry.npmjs.org/fill-range/-/fill-range-7.0.1.tgz", - "integrity": "sha512-qOo9F+dMUmC2Lcb4BbVvnKJxTPjCm+RRpe4gDuGrzkL7mEVl/djYSu2OdQ2Pa302N4oqkSg9ir6jaLWJ2USVpQ==", + "node_modules/istanbul-lib-coverage": { + "version": "3.2.2", + "resolved": "https://registry.npmjs.org/istanbul-lib-coverage/-/istanbul-lib-coverage-3.2.2.tgz", + "integrity": "sha512-O8dpsF+r0WV/8MNRKfnmrtCWhuKjxrq2w+jpzBL5UZKTi2LeVWnWOmWRxFlesJONmc+wLAGvKQZEOanko0LFTg==", "dev": true, - "dependencies": { - "to-regex-range": "^5.0.1" - }, "engines": { "node": ">=8" } }, - "node_modules/find-up": { - "version": "5.0.0", - "resolved": "https://registry.npmjs.org/find-up/-/find-up-5.0.0.tgz", - "integrity": "sha512-78/PXT1wlLLDgTzDs7sjq9hzz0vXD+zn+7wypEe4fXQxCmdmqfGsEPQxmiCSQI3ajFV91bVSsvNtrJRiW6nGng==", + "node_modules/istanbul-lib-instrument": { + "version": "6.0.1", + "resolved": "https://registry.npmjs.org/istanbul-lib-instrument/-/istanbul-lib-instrument-6.0.1.tgz", + "integrity": "sha512-EAMEJBsYuyyztxMxW3g7ugGPkrZsV57v0Hmv3mm1uQsmB+QnZuepg731CRaIgeUVSdmsTngOkSnauNF8p7FIhA==", "dev": true, "dependencies": { - "locate-path": "^6.0.0", - "path-exists": "^4.0.0" + "@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" }, "engines": { "node": ">=10" - }, - "funding": { - "url": "https://github.com/sponsors/sindresorhus" - } - }, - "node_modules/flat": { - "version": "5.0.2", - "resolved": "https://registry.npmjs.org/flat/-/flat-5.0.2.tgz", - "integrity": "sha512-b6suED+5/3rTpUBdG1gupIl8MPFCAMA0QXwmljLhvCUKcUvdE4gWky9zpuGCcXHOsz4J9wPGNWq6OKpmIzz3hQ==", - "dev": true, - "bin": { - "flat": "cli.js" } }, - "node_modules/foreground-child": { - "version": "3.1.1", - "resolved": "https://registry.npmjs.org/foreground-child/-/foreground-child-3.1.1.tgz", - "integrity": "sha512-TMKDUnIte6bfb5nWv7V/caI169OHgvwjb7V4WkeUvbQQdjr5rWKqHFiKWb/fcOwB+CzBT+qbWjvj+DVwRskpIg==", + "node_modules/istanbul-lib-report": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/istanbul-lib-report/-/istanbul-lib-report-3.0.1.tgz", + "integrity": "sha512-GCfE1mtsHGOELCU8e/Z7YWzpmybrx/+dSTfLrvY8qRmaY6zXTKWn6WQIjaAFw069icm6GVMNkgu0NzI4iPZUNw==", "dev": true, "dependencies": { - "cross-spawn": "^7.0.0", - "signal-exit": "^4.0.1" + "istanbul-lib-coverage": "^3.0.0", + "make-dir": "^4.0.0", + "supports-color": "^7.1.0" }, "engines": { - "node": ">=14" - }, - "funding": { - "url": "https://github.com/sponsors/isaacs" + "node": ">=10" } }, - "node_modules/foreground-child/node_modules/signal-exit": { - "version": "4.1.0", - "resolved": "https://registry.npmjs.org/signal-exit/-/signal-exit-4.1.0.tgz", - "integrity": "sha512-bzyZ1e88w9O1iNJbKnOlvYTrWPDl46O1bG0D3XInv+9tkPrxrN8jUUTiFlDkkmKWgn1M6CfIA13SuGqOa9Korw==", + "node_modules/istanbul-lib-report/node_modules/make-dir": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/make-dir/-/make-dir-4.0.0.tgz", + "integrity": "sha512-hXdUTZYIVOt1Ex//jAQi+wTZZpUpwBj/0QsOzqegb3rGMMeJiSEu5xLHnYfBrRV4RH2+OCSOO95Is/7x1WJ4bw==", "dev": true, + "dependencies": { + "semver": "^7.5.3" + }, "engines": { - "node": ">=14" + "node": ">=10" }, "funding": { - "url": "https://github.com/sponsors/isaacs" + "url": "https://github.com/sponsors/sindresorhus" } }, - "node_modules/fs-minipass": { - "version": "3.0.3", - "resolved": "https://registry.npmjs.org/fs-minipass/-/fs-minipass-3.0.3.tgz", - "integrity": "sha512-XUBA9XClHbnJWSfBzjkm6RvPsyg3sryZt06BEQoXcF7EK/xpGaQYJgQKDJSUH5SGZ76Y7pFx1QBnXz09rU5Fbw==", + "node_modules/istanbul-lib-source-maps": { + "version": "4.0.1", + "resolved": "https://registry.npmjs.org/istanbul-lib-source-maps/-/istanbul-lib-source-maps-4.0.1.tgz", + "integrity": "sha512-n3s8EwkdFIJCG3BPKBYvskgXGoy88ARzvegkitk60NxRdwltLOTaH7CUiMRXvwYorl0Q712iEjcWB+fK/MrWVw==", "dev": true, "dependencies": { - "minipass": "^7.0.3" + "debug": "^4.1.1", + "istanbul-lib-coverage": "^3.0.0", + "source-map": "^0.6.1" }, "engines": { - "node": "^14.17.0 || ^16.13.0 || >=18.0.0" + "node": ">=10" } }, - "node_modules/fs-minipass/node_modules/minipass": { - "version": "7.0.4", - "resolved": "https://registry.npmjs.org/minipass/-/minipass-7.0.4.tgz", - "integrity": "sha512-jYofLM5Dam9279rdkWzqHozUo4ybjdZmCsDHePy5V/PbBcVMiSZR97gmAy45aqi8CK1lG2ECd356FU86avfwUQ==", + "node_modules/istanbul-reports": { + "version": "3.1.6", + "resolved": "https://registry.npmjs.org/istanbul-reports/-/istanbul-reports-3.1.6.tgz", + "integrity": "sha512-TLgnMkKg3iTDsQ9PbPTdpfAK2DzjF9mqUG7RMgcQl8oFjad8ob4laGxv5XV5U9MAfx8D6tSJiUyuAwzLicaxlg==", "dev": true, + "dependencies": { + "html-escaper": "^2.0.0", + "istanbul-lib-report": "^3.0.0" + }, "engines": { - "node": ">=16 || 14 >=14.17" + "node": ">=8" } }, - "node_modules/fs.realpath": { - "version": "1.0.0", - "resolved": "https://registry.npmjs.org/fs.realpath/-/fs.realpath-1.0.0.tgz", - "integrity": "sha512-OO0pH2lK6a0hZnAdau5ItzHPI6pUlvI7jMVnxUQRtw4owF2wk8lOSabtGDCTP4Ggrg2MbGnWO9X8K1t4+fGMDw==" - }, - "node_modules/fsevents": { - "version": "2.3.3", - "resolved": "https://registry.npmjs.org/fsevents/-/fsevents-2.3.3.tgz", - "integrity": "sha512-5xoDfX+fL7faATnagmWPpbFtwh/R77WmMMqqHGS65C3vvB0YHrgF+B1YmZ3441tMj5n63k0212XNoJwzlhffQw==", + "node_modules/jest": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/jest/-/jest-29.7.0.tgz", + "integrity": "sha512-NIy3oAFp9shda19hy4HK0HRTWKtPJmGdnvywu01nOqNC2vZg+Z+fvJDxpMQA88eb2I9EcafcdjYgsDthnYTvGw==", "dev": true, - "hasInstallScript": true, - "optional": true, - "os": [ - "darwin" - ], - "engines": { - "node": "^8.16.0 || ^10.6.0 || >=11.0.0" - } - }, - "node_modules/gauge": { - "version": "3.0.2", - "resolved": "https://registry.npmjs.org/gauge/-/gauge-3.0.2.tgz", - "integrity": "sha512-+5J6MS/5XksCuXq++uFRsnUd7Ovu1XenbeuIuNRJxYWjgQbPuFhT14lAvsWfqfAmnwluf1OwMjz39HjfLPci0Q==", "dependencies": { - "aproba": "^1.0.3 || ^2.0.0", - "color-support": "^1.1.2", - "console-control-strings": "^1.0.0", - "has-unicode": "^2.0.1", - "object-assign": "^4.1.1", - "signal-exit": "^3.0.0", - "string-width": "^4.2.3", - "strip-ansi": "^6.0.1", - "wide-align": "^1.1.2" + "@jest/core": "^29.7.0", + "@jest/types": "^29.6.3", + "import-local": "^3.0.2", + "jest-cli": "^29.7.0" + }, + "bin": { + "jest": "bin/jest.js" }, "engines": { - "node": ">=10" + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + }, + "peerDependencies": { + "node-notifier": "^8.0.1 || ^9.0.0 || ^10.0.0" + }, + "peerDependenciesMeta": { + "node-notifier": { + "optional": true + } } }, - "node_modules/get-caller-file": { - "version": "2.0.5", - "resolved": "https://registry.npmjs.org/get-caller-file/-/get-caller-file-2.0.5.tgz", - "integrity": "sha512-DyFP3BM/3YHTQOCUL/w0OZHR0lpKeGrxotcHWcqNEdnltqFwXVfhEBQ94eIo34AfQpo0rGki4cyIiftY06h2Fg==", + "node_modules/jest-changed-files": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/jest-changed-files/-/jest-changed-files-29.7.0.tgz", + "integrity": "sha512-fEArFiwf1BpQ+4bXSprcDc3/x4HSzL4al2tozwVpDFpsxALjLYdyiIK4e5Vz66GQJIbXJ82+35PtysofptNX2w==", "dev": true, - "engines": { - "node": "6.* || 8.* || >= 10.*" - } - }, - "node_modules/glob": { - "version": "7.2.3", - "resolved": "https://registry.npmjs.org/glob/-/glob-7.2.3.tgz", - "integrity": "sha512-nFR0zLpU2YCaRxwoCJvL6UvCH2JFyFVIvwTLsIf21AuHlMskA1hhTdk+LlYJtOlYt9v6dvszD2BGRqBL+iQK9Q==", "dependencies": { - "fs.realpath": "^1.0.0", - "inflight": "^1.0.4", - "inherits": "2", - "minimatch": "^3.1.1", - "once": "^1.3.0", - "path-is-absolute": "^1.0.0" + "execa": "^5.0.0", + "jest-util": "^29.7.0", + "p-limit": "^3.1.0" }, "engines": { - "node": "*" - }, - "funding": { - "url": "https://github.com/sponsors/isaacs" + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" } }, - "node_modules/glob-parent": { - "version": "5.1.2", - "resolved": "https://registry.npmjs.org/glob-parent/-/glob-parent-5.1.2.tgz", - "integrity": "sha512-AOIgSQCepiJYwP3ARnGx+5VnTu2HBYdzbGP45eLw1vr3zB3vZLeyed1sC9hnbcOc9/SrMyM5RPQrkGz4aS9Zow==", + "node_modules/jest-circus": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/jest-circus/-/jest-circus-29.7.0.tgz", + "integrity": "sha512-3E1nCMgipcTkCocFwM90XXQab9bS+GMsjdpmPrlelaxwD93Ad8iVEjX/vvHPdLPnFf+L40u+5+iutRdA1N9myw==", "dev": true, "dependencies": { - "is-glob": "^4.0.1" + "@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" }, "engines": { - "node": ">= 6" + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" } }, - "node_modules/glob/node_modules/minimatch": { - "version": "3.1.2", - "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-3.1.2.tgz", - "integrity": "sha512-J7p63hRiAjw1NDEww1W7i37+ByIrOWO5XQQAzZ3VOcL0PNybwpfmV/N05zFAzwQ9USyEcX6t3UO+K5aqBQOIHw==", + "node_modules/jest-cli": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/jest-cli/-/jest-cli-29.7.0.tgz", + "integrity": "sha512-OVVobw2IubN/GSYsxETi+gOe7Ka59EFMR/twOU3Jb2GnKKeMGJB5SGUUrEz3SFVmJASUdZUzy83sLNNQ2gZslg==", + "dev": true, "dependencies": { - "brace-expansion": "^1.1.7" + "@jest/core": "^29.7.0", + "@jest/test-result": "^29.7.0", + "@jest/types": "^29.6.3", + "chalk": "^4.0.0", + "create-jest": "^29.7.0", + "exit": "^0.1.2", + "import-local": "^3.0.2", + "jest-config": "^29.7.0", + "jest-util": "^29.7.0", + "jest-validate": "^29.7.0", + "yargs": "^17.3.1" + }, + "bin": { + "jest": "bin/jest.js" }, "engines": { - "node": "*" + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + }, + "peerDependencies": { + "node-notifier": "^8.0.1 || ^9.0.0 || ^10.0.0" + }, + "peerDependenciesMeta": { + "node-notifier": { + "optional": true + } } }, - "node_modules/graceful-fs": { - "version": "4.2.11", - "resolved": "https://registry.npmjs.org/graceful-fs/-/graceful-fs-4.2.11.tgz", - "integrity": "sha512-RbJ5/jmFcNNCcDV5o9eTnBLJ/HszWV0P73bc+Ff4nS/rJj+YaS6IGyiOL0VoBYX+l1Wrl3k63h/KrH+nhJ0XvQ==", - "dev": true + "node_modules/jest-config": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/jest-config/-/jest-config-29.7.0.tgz", + "integrity": "sha512-uXbpfeQ7R6TZBqI3/TxCU4q4ttk3u0PJeC+E0zbfSoSjq6bJ7buBPxzQPL0ifrkY4DNu4JUdk0ImlBUYi840eQ==", + "dev": true, + "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" + }, + "engines": { + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + }, + "peerDependencies": { + "@types/node": "*", + "ts-node": ">=9.0.0" + }, + "peerDependenciesMeta": { + "@types/node": { + "optional": true + }, + "ts-node": { + "optional": true + } + } }, - "node_modules/has-flag": { - "version": "4.0.0", - "resolved": "https://registry.npmjs.org/has-flag/-/has-flag-4.0.0.tgz", - "integrity": "sha512-EykJT/Q1KjTWctppgIAgfSO0tKVuZUjhgMr17kqTumMl6Afv3EISleU7qZUzoXDFTAHTDC4NOoG/ZxU3EvlMPQ==", + "node_modules/jest-diff": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/jest-diff/-/jest-diff-29.7.0.tgz", + "integrity": "sha512-LMIgiIrhigmPrs03JHpxUh2yISK3vLFPkAodPeo0+BuF7wA2FoQbkEg1u8gBYBThncu7e1oEDUfIXVuTqLRUjw==", "dev": true, + "dependencies": { + "chalk": "^4.0.0", + "diff-sequences": "^29.6.3", + "jest-get-type": "^29.6.3", + "pretty-format": "^29.7.0" + }, "engines": { - "node": ">=8" + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" } }, - "node_modules/has-unicode": { - "version": "2.0.1", - "resolved": "https://registry.npmjs.org/has-unicode/-/has-unicode-2.0.1.tgz", - "integrity": "sha512-8Rf9Y83NBReMnx0gFzA8JImQACstCYWUplepDa9xprwwtmgEZUF0h/i5xSA625zB/I37EtrswSST6OXxwaaIJQ==" - }, - "node_modules/he": { - "version": "1.2.0", - "resolved": "https://registry.npmjs.org/he/-/he-1.2.0.tgz", - "integrity": "sha512-F/1DnUGPopORZi0ni+CvrCgHQ5FyEAHRLSApuYWMmrbSwoN2Mn/7k+Gl38gJnR7yyDZk6WLXwiGod1JOWNDKGw==", + "node_modules/jest-docblock": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/jest-docblock/-/jest-docblock-29.7.0.tgz", + "integrity": "sha512-q617Auw3A612guyaFgsbFeYpNP5t2aoUNLwBUbc/0kD1R4t9ixDbyFTHd1nok4epoVFpr7PmeWHrhvuV3XaJ4g==", "dev": true, - "bin": { - "he": "bin/he" + "dependencies": { + "detect-newline": "^3.0.0" + }, + "engines": { + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" } }, - "node_modules/htmlparser2": { - "version": "3.8.3", - "resolved": "https://registry.npmjs.org/htmlparser2/-/htmlparser2-3.8.3.tgz", - "integrity": "sha512-hBxEg3CYXe+rPIua8ETe7tmG3XDn9B0edOE/e9wH2nLczxzgdu0m0aNHY+5wFZiviLWLdANPJTssa92dMcXQ5Q==", + "node_modules/jest-each": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/jest-each/-/jest-each-29.7.0.tgz", + "integrity": "sha512-gns+Er14+ZrEoC5fhOfYCY1LOHHr0TI+rQUHZS8Ttw2l7gl+80eHc/gFf2Ktkw0+SIACDTeWvpFcv3B04VembQ==", "dev": true, "dependencies": { - "domelementtype": "1", - "domhandler": "2.3", - "domutils": "1.5", - "entities": "1.0", - "readable-stream": "1.1" + "@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" + }, + "engines": { + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" } }, - "node_modules/http-cache-semantics": { - "version": "4.1.1", - "resolved": "https://registry.npmjs.org/http-cache-semantics/-/http-cache-semantics-4.1.1.tgz", - "integrity": "sha512-er295DKPVsV82j5kw1Gjt+ADA/XYHsajl82cGNQG2eyoPkvgUhX+nDIyelzhIWbbsXP39EHcI6l5tYs2FYqYXQ==", - "dev": true - }, - "node_modules/http-proxy-agent": { - "version": "5.0.0", - "resolved": "https://registry.npmjs.org/http-proxy-agent/-/http-proxy-agent-5.0.0.tgz", - "integrity": "sha512-n2hY8YdoRE1i7r6M0w9DIw5GgZN0G25P8zLCRQ8rjXtTU3vsNFBI/vWK/UIeE6g5MUUz6avwAPXmL6Fy9D/90w==", + "node_modules/jest-environment-node": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/jest-environment-node/-/jest-environment-node-29.7.0.tgz", + "integrity": "sha512-DOSwCRqXirTOyheM+4d5YZOrWcdu0LNZ87ewUoywbcb2XR4wKgqiG8vNeYwhjFMbEkfju7wx2GYH0P2gevGvFw==", "dev": true, "dependencies": { - "@tootallnate/once": "2", - "agent-base": "6", - "debug": "4" + "@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" }, "engines": { - "node": ">= 6" + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" } }, - "node_modules/https-proxy-agent": { - "version": "5.0.1", - "resolved": "https://registry.npmjs.org/https-proxy-agent/-/https-proxy-agent-5.0.1.tgz", - "integrity": "sha512-dFcAjpTQFgoLMzC2VwU+C/CbS7uRL0lWmxDITmqm7C+7F0Odmj6s9l6alZc6AELXhrnggM2CeWSXHGOdX2YtwA==", - "dependencies": { - "agent-base": "6", - "debug": "4" - }, + "node_modules/jest-get-type": { + "version": "29.6.3", + "resolved": "https://registry.npmjs.org/jest-get-type/-/jest-get-type-29.6.3.tgz", + "integrity": "sha512-zrteXnqYxfQh7l5FHyL38jL39di8H8rHoecLH3JNxH3BwOrBsNeabdap5e0I23lD4HHI8W5VFBZqG4Eaq5LNcw==", + "dev": true, "engines": { - "node": ">= 6" + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" } }, - "node_modules/humanize-ms": { - "version": "1.2.1", - "resolved": "https://registry.npmjs.org/humanize-ms/-/humanize-ms-1.2.1.tgz", - "integrity": "sha512-Fl70vYtsAFb/C06PTS9dZBo7ihau+Tu/DNCk/OyHhea07S+aeMWpFFkUaXRa8fI+ScZbEI8dfSxwY7gxZ9SAVQ==", + "node_modules/jest-haste-map": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/jest-haste-map/-/jest-haste-map-29.7.0.tgz", + "integrity": "sha512-fP8u2pyfqx0K1rGn1R9pyE0/KTn+G7PxktWidOBTqFPLYX0b9ksaMFkhK5vrS3DVun09pckLdlx90QthlW7AmA==", "dev": true, "dependencies": { - "ms": "^2.0.0" + "@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" + }, + "engines": { + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + }, + "optionalDependencies": { + "fsevents": "^2.3.2" } - }, - "node_modules/iconv-lite": { - "version": "0.6.3", - "resolved": "https://registry.npmjs.org/iconv-lite/-/iconv-lite-0.6.3.tgz", - "integrity": "sha512-4fCk79wshMdzMp2rH06qWrJE4iolqLhCUH+OiuIgU++RB0+94NlDL81atO7GX55uUKueo0txHNtvEyI6D7WdMw==", - "optional": true, + }, + "node_modules/jest-leak-detector": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/jest-leak-detector/-/jest-leak-detector-29.7.0.tgz", + "integrity": "sha512-kYA8IJcSYtST2BY9I+SMC32nDpBT3J2NvWJx8+JCuCdl/CR1I4EKUJROiP8XtCcxqgTTBGJNdbB1A8XRKbTetw==", + "dev": true, "dependencies": { - "safer-buffer": ">= 2.1.2 < 3.0.0" + "jest-get-type": "^29.6.3", + "pretty-format": "^29.7.0" }, "engines": { - "node": ">=0.10.0" + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" } }, - "node_modules/imurmurhash": { - "version": "0.1.4", - "resolved": "https://registry.npmjs.org/imurmurhash/-/imurmurhash-0.1.4.tgz", - "integrity": "sha512-JmXMZ6wuvDmLiHEml9ykzqO6lwFbof0GG4IkcGaENdCRDDmMVnny7s5HsIgHCbaq0w2MyPhDqkhTUgS2LU2PHA==", + "node_modules/jest-matcher-utils": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/jest-matcher-utils/-/jest-matcher-utils-29.7.0.tgz", + "integrity": "sha512-sBkD+Xi9DtcChsI3L3u0+N0opgPYnCRPtGcQYrgXmR+hmt/fYfWAL0xRXYU8eWOdfuLgBe0YCW3AFtnRLagq/g==", "dev": true, + "dependencies": { + "chalk": "^4.0.0", + "jest-diff": "^29.7.0", + "jest-get-type": "^29.6.3", + "pretty-format": "^29.7.0" + }, "engines": { - "node": ">=0.8.19" + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" } }, - "node_modules/indent-string": { - "version": "4.0.0", - "resolved": "https://registry.npmjs.org/indent-string/-/indent-string-4.0.0.tgz", - "integrity": "sha512-EdDDZu4A2OyIK7Lr/2zG+w5jmbuk1DVBnEwREQvBzspBJkCEbRa8GxU1lghYcaGJCnRWibjDXlq779X1/y5xwg==", + "node_modules/jest-message-util": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/jest-message-util/-/jest-message-util-29.7.0.tgz", + "integrity": "sha512-GBEV4GRADeP+qtB2+6u61stea8mGcOT4mCtrYISZwfu9/ISHFJ/5zOMXYbpBE9RsS5+Gb63DW4FgmnKJ79Kf6w==", "dev": true, + "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" + }, "engines": { - "node": ">=8" + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" } }, - "node_modules/inflight": { - "version": "1.0.6", - "resolved": "https://registry.npmjs.org/inflight/-/inflight-1.0.6.tgz", - "integrity": "sha512-k92I/b08q4wvFscXCLvqfsHCrjrF7yiXsQuIVvVE7N82W3+aqpzuUdBbfhWcy/FZR3/4IgflMgKLOsvPDrGCJA==", + "node_modules/jest-mock": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/jest-mock/-/jest-mock-29.7.0.tgz", + "integrity": "sha512-ITOMZn+UkYS4ZFh83xYAOzWStloNzJFO2s8DWrE4lhtGD+AorgnbkiKERe4wQVBydIGPx059g6riW5Btp6Llnw==", + "dev": true, "dependencies": { - "once": "^1.3.0", - "wrappy": "1" + "@jest/types": "^29.6.3", + "@types/node": "*", + "jest-util": "^29.7.0" + }, + "engines": { + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" } }, - "node_modules/inherits": { - "version": "2.0.4", - "resolved": "https://registry.npmjs.org/inherits/-/inherits-2.0.4.tgz", - "integrity": "sha512-k/vGaX4/Yla3WzyMCvTQOXYeIHvqOKtnqBduzTHpzpQZzAskKMhZ2K+EnBiSM9zGSoIFeMpXKxa4dYeZIQqewQ==" + "node_modules/jest-pnp-resolver": { + "version": "1.2.3", + "resolved": "https://registry.npmjs.org/jest-pnp-resolver/-/jest-pnp-resolver-1.2.3.tgz", + "integrity": "sha512-+3NpwQEnRoIBtx4fyhblQDPgJI0H1IEIkX7ShLUjPGA7TtUTvI1oiKi3SR4oBR0hQhQR80l4WAe5RrXBwWMA8w==", + "dev": true, + "engines": { + "node": ">=6" + }, + "peerDependencies": { + "jest-resolve": "*" + }, + "peerDependenciesMeta": { + "jest-resolve": { + "optional": true + } + } }, - "node_modules/ip": { - "version": "2.0.0", - "resolved": "https://registry.npmjs.org/ip/-/ip-2.0.0.tgz", - "integrity": "sha512-WKa+XuLG1A1R0UWhl2+1XQSi+fZWMsYKffMZTTYsiZaUD8k2yDAj5atimTUD2TZkyCkNEeYE5NhFZmupOGtjYQ==", - "dev": true + "node_modules/jest-regex-util": { + "version": "29.6.3", + "resolved": "https://registry.npmjs.org/jest-regex-util/-/jest-regex-util-29.6.3.tgz", + "integrity": "sha512-KJJBsRCyyLNWCNBOvZyRDnAIfUiRJ8v+hOBQYGn8gDyF3UegwiP4gwRR3/SDa42g1YbVycTidUF3rKjyLFDWbg==", + "dev": true, + "engines": { + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + } }, - "node_modules/is-binary-path": { - "version": "2.1.0", - "resolved": "https://registry.npmjs.org/is-binary-path/-/is-binary-path-2.1.0.tgz", - "integrity": "sha512-ZMERYes6pDydyuGidse7OsHxtbI7WVeUEozgR/g7rd0xUimYNlvZRE/K2MgZTjWy725IfelLeVcEM97mmtRGXw==", + "node_modules/jest-resolve": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/jest-resolve/-/jest-resolve-29.7.0.tgz", + "integrity": "sha512-IOVhZSrg+UvVAshDSDtHyFCCBUl/Q3AAJv8iZ6ZjnZ74xzvwuzLXid9IIIPgTnY62SJjfuupMKZsZQRsCvxEgA==", "dev": true, "dependencies": { - "binary-extensions": "^2.0.0" + "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" }, "engines": { - "node": ">=8" + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" } }, - "node_modules/is-extglob": { - "version": "2.1.1", - "resolved": "https://registry.npmjs.org/is-extglob/-/is-extglob-2.1.1.tgz", - "integrity": "sha512-SbKbANkN603Vi4jEZv49LeVJMn4yGwsbzZworEoyEiutsN3nJYdbO36zfhGJ6QEDpOZIFkDtnq5JRxmvl3jsoQ==", - "dev": true, + "node_modules/jest-resolve-dependencies": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/jest-resolve-dependencies/-/jest-resolve-dependencies-29.7.0.tgz", + "integrity": "sha512-un0zD/6qxJ+S0et7WxeI3H5XSe9lTBBR7bOHCHXkKR6luG5mwDDlIzVQ0V5cZCuoTgEdcdwzTghYkTWfubi+nA==", + "dev": true, + "dependencies": { + "jest-regex-util": "^29.6.3", + "jest-snapshot": "^29.7.0" + }, "engines": { - "node": ">=0.10.0" + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" } }, - "node_modules/is-fullwidth-code-point": { - "version": "3.0.0", - "resolved": "https://registry.npmjs.org/is-fullwidth-code-point/-/is-fullwidth-code-point-3.0.0.tgz", - "integrity": "sha512-zymm5+u+sCsSWyD9qNaejV3DFvhCKclKdizYaJUuHA83RLjb7nSuGnddCHGv0hk+KY7BMAlsWeK4Ueg6EV6XQg==", + "node_modules/jest-runner": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/jest-runner/-/jest-runner-29.7.0.tgz", + "integrity": "sha512-fsc4N6cPCAahybGBfTRcq5wFR6fpLznMg47sY5aDpsoejOcVYFb07AHuSnR0liMcPTgBsA3ZJL6kFOjPdoNipQ==", + "dev": true, + "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" + }, "engines": { - "node": ">=8" + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" } }, - "node_modules/is-glob": { - "version": "4.0.3", - "resolved": "https://registry.npmjs.org/is-glob/-/is-glob-4.0.3.tgz", - "integrity": "sha512-xelSayHH36ZgE7ZWhli7pW34hNbNl8Ojv5KVmkJD4hBdD3th8Tfk9vYasLM+mXWOZhFkgZfxhLSnrwRr4elSSg==", + "node_modules/jest-runtime": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/jest-runtime/-/jest-runtime-29.7.0.tgz", + "integrity": "sha512-gUnLjgwdGqW7B4LvOIkbKs9WGbn+QLqRQQ9juC6HndeDiezIwhDP+mhMwHWCEcfQ5RUXa6OPnFF8BJh5xegwwQ==", "dev": true, "dependencies": { - "is-extglob": "^2.1.1" + "@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" }, "engines": { - "node": ">=0.10.0" + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" } }, - "node_modules/is-lambda": { - "version": "1.0.1", - "resolved": "https://registry.npmjs.org/is-lambda/-/is-lambda-1.0.1.tgz", - "integrity": "sha512-z7CMFGNrENq5iFB9Bqo64Xk6Y9sg+epq1myIcdHaGnbMTYOxvzsEtdYqQUylB7LxfkvgrrjP32T6Ywciio9UIQ==", - "dev": true - }, - "node_modules/is-number": { - "version": "7.0.0", - "resolved": "https://registry.npmjs.org/is-number/-/is-number-7.0.0.tgz", - "integrity": "sha512-41Cifkg6e8TylSpdtTpeLVMqvSBEVzTttHvERD741+pnZ8ANv0004MRL43QKPDlK9cGvNp6NZWZUBlbGXYxxng==", + "node_modules/jest-snapshot": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/jest-snapshot/-/jest-snapshot-29.7.0.tgz", + "integrity": "sha512-Rm0BMWtxBcioHr1/OX5YCP8Uov4riHvKPknOGs804Zg9JGZgmIBkbtlxJC/7Z4msKYVbIJtfU+tKb8xlYNfdkw==", "dev": true, + "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" + }, "engines": { - "node": ">=0.12.0" + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" } }, - "node_modules/is-plain-obj": { - "version": "2.1.0", - "resolved": "https://registry.npmjs.org/is-plain-obj/-/is-plain-obj-2.1.0.tgz", - "integrity": "sha512-YWnfyRwxL/+SsrWYfOpUtz5b3YD+nyfkHvjbcanzk8zgyO4ASD67uVMRt8k5bM4lLMDnXfriRhOpemw+NfT1eA==", + "node_modules/jest-util": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/jest-util/-/jest-util-29.7.0.tgz", + "integrity": "sha512-z6EbKajIpqGKU56y5KBUgy1dt1ihhQJgWzUlZHArA/+X2ad7Cb5iF+AK1EWVL/Bo7Rz9uurpqw6SiBCefUbCGA==", + "dev": true, + "dependencies": { + "@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" + }, + "engines": { + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + } + }, + "node_modules/jest-validate": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/jest-validate/-/jest-validate-29.7.0.tgz", + "integrity": "sha512-ZB7wHqaRGVw/9hST/OuFUReG7M8vKeq0/J2egIGLdvjHCmYqGARhzXmtgi+gVeZ5uXFF219aOc3Ls2yLg27tkw==", "dev": true, + "dependencies": { + "@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" + }, "engines": { - "node": ">=8" + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" } }, - "node_modules/is-unicode-supported": { - "version": "0.1.0", - "resolved": "https://registry.npmjs.org/is-unicode-supported/-/is-unicode-supported-0.1.0.tgz", - "integrity": "sha512-knxG2q4UC3u8stRGyAVJCOdxFmv5DZiRcdlIaAQXAbSfJya+OhopNotLQrstBhququ4ZpuKbDc/8S6mgXgPFPw==", + "node_modules/jest-validate/node_modules/camelcase": { + "version": "6.3.0", + "resolved": "https://registry.npmjs.org/camelcase/-/camelcase-6.3.0.tgz", + "integrity": "sha512-Gmy6FhYlCY7uOElZUSbxo2UCDH8owEk996gkbrpsgGtrJLM3J7jGxl9Ic7Qwwj4ivOE5AWZWRMecDdF7hqGjFA==", "dev": true, "engines": { "node": ">=10" @@ -1246,43 +3354,69 @@ "url": "https://github.com/sponsors/sindresorhus" } }, - "node_modules/isarray": { - "version": "0.0.1", - "resolved": "https://registry.npmjs.org/isarray/-/isarray-0.0.1.tgz", - "integrity": "sha512-D2S+3GLxWH+uhrNEcoh/fnmYeP8E8/zHl644d/jdA0g2uyXvy3sb0qxotE+ne0LtccHknQzWwZEzhak7oJ0COQ==", - "dev": true + "node_modules/jest-watcher": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/jest-watcher/-/jest-watcher-29.7.0.tgz", + "integrity": "sha512-49Fg7WXkU3Vl2h6LbLtMQ/HyB6rXSIX7SqvBLQmssRBGN9I0PNvPmAmCWSOY6SOvrjhI/F7/bGAv9RtnsPA03g==", + "dev": true, + "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" + }, + "engines": { + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + } }, - "node_modules/isexe": { - "version": "2.0.0", - "resolved": "https://registry.npmjs.org/isexe/-/isexe-2.0.0.tgz", - "integrity": "sha512-RHxMLp9lnKHGHRng9QFhRCMbYAcVpn69smSGcq3f36xjgVVWThj4qqLbTLlq7Ssj8B+fIQ1EuCEGI2lKsyQeIw==", - "dev": true + "node_modules/jest-worker": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/jest-worker/-/jest-worker-29.7.0.tgz", + "integrity": "sha512-eIz2msL/EzL9UFTFFx7jBTkeZfku0yUAyZZZmJ93H2TYEiroIx2PQjEXcwYtYl8zXCxb+PAmA2hLIt/6ZEkPHw==", + "dev": true, + "dependencies": { + "@types/node": "*", + "jest-util": "^29.7.0", + "merge-stream": "^2.0.0", + "supports-color": "^8.0.0" + }, + "engines": { + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + } }, - "node_modules/jackspeak": { - "version": "2.3.6", - "resolved": "https://registry.npmjs.org/jackspeak/-/jackspeak-2.3.6.tgz", - "integrity": "sha512-N3yCS/NegsOBokc8GAdM8UcmfsKiSS8cipheD/nivzr700H+nsMOxJjQnvwOcRYVuFkdH0wGUvW2WbXGmrZGbQ==", + "node_modules/jest-worker/node_modules/supports-color": { + "version": "8.1.1", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-8.1.1.tgz", + "integrity": "sha512-MpUEN2OodtUzxvKQl72cUF7RQ5EiHsGvSsVG0ia9c5RbWGL2CI4C7EpPS8UTBIplnlzZiNuV56w+FuNxy3ty2Q==", "dev": true, "dependencies": { - "@isaacs/cliui": "^8.0.2" + "has-flag": "^4.0.0" }, "engines": { - "node": ">=14" + "node": ">=10" }, "funding": { - "url": "https://github.com/sponsors/isaacs" - }, - "optionalDependencies": { - "@pkgjs/parseargs": "^0.11.0" + "url": "https://github.com/chalk/supports-color?sponsor=1" } }, + "node_modules/js-tokens": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/js-tokens/-/js-tokens-4.0.0.tgz", + "integrity": "sha512-RdJUflcE3cUzKiMqQgsCu06FPu9UdIJO0beYbPhHN4k6apgJtifcoCtT9bcxOpYBtpD2kCM6Sbzg4CausW/PKQ==", + "dev": true + }, "node_modules/js-yaml": { - "version": "4.1.0", - "resolved": "https://registry.npmjs.org/js-yaml/-/js-yaml-4.1.0.tgz", - "integrity": "sha512-wpxZs9NoxZaJESJGIZTyDEaYpl0FKSA+FB9aJiyemKhMwkxQg63h4T1KJgUGHpTqPDNRcmmYLugrRjJlBtWvRA==", + "version": "3.14.1", + "resolved": "https://registry.npmjs.org/js-yaml/-/js-yaml-3.14.1.tgz", + "integrity": "sha512-okMH7OXXJ7YrN9Ok3/SXrnu4iX9yOk+25nqX4imS2npuvTYDmo/QEZoqwZkYaIDk3jVvBOTOIEgEhaLOynBS9g==", "dev": true, "dependencies": { - "argparse": "^2.0.1" + "argparse": "^1.0.7", + "esprima": "^4.0.0" }, "bin": { "js-yaml": "bin/js-yaml.js" @@ -1326,6 +3460,18 @@ "node": ">=12.0.0" } }, + "node_modules/jsesc": { + "version": "2.5.2", + "resolved": "https://registry.npmjs.org/jsesc/-/jsesc-2.5.2.tgz", + "integrity": "sha512-OYu7XEzjkCQ3C5Ps3QIZsQfNpqoJyZZA99wd9aWd05NCtC5pWOkShK2mkL6HXQR6/Cy2lbNdPlZBpuQHXE63gA==", + "dev": true, + "bin": { + "jsesc": "bin/jsesc" + }, + "engines": { + "node": ">=4" + } + }, "node_modules/jshint": { "version": "2.13.6", "resolved": "https://registry.npmjs.org/jshint/-/jshint-2.13.6.tgz", @@ -1344,6 +3490,18 @@ "jshint": "bin/jshint" } }, + "node_modules/jshint/node_modules/minimatch": { + "version": "3.0.8", + "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-3.0.8.tgz", + "integrity": "sha512-6FsRAQsxQ61mw+qP1ZzbL9Bc78x2p5OqNgNpnoAFLTrX8n5Kxph0CsnhmKKNXTWjXqU5L0pGPR7hYk+XWZr60Q==", + "dev": true, + "dependencies": { + "brace-expansion": "^1.1.7" + }, + "engines": { + "node": "*" + } + }, "node_modules/jshint/node_modules/strip-json-comments": { "version": "1.0.4", "resolved": "https://registry.npmjs.org/strip-json-comments/-/strip-json-comments-1.0.4.tgz", @@ -1356,6 +3514,24 @@ "node": ">=0.8.0" } }, + "node_modules/json-parse-even-better-errors": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/json-parse-even-better-errors/-/json-parse-even-better-errors-2.3.1.tgz", + "integrity": "sha512-xyFwyhro/JEof6Ghe2iz2NcXoj2sloNsWr/XsERDK/oiPCfaNhl5ONfp+jQdAZRQQ0IJWNzH9zIZF7li91kh2w==", + "dev": true + }, + "node_modules/json5": { + "version": "2.2.3", + "resolved": "https://registry.npmjs.org/json5/-/json5-2.2.3.tgz", + "integrity": "sha512-XmOWe7eyHYH14cLdVPoyg+GOH3rYX++KpzrylJwSW98t3Nk+U8XOl8FWKOgwtzdb8lXGf6zYwDUzeHMWfxasyg==", + "dev": true, + "bin": { + "json5": "lib/cli.js" + }, + "engines": { + "node": ">=6" + } + }, "node_modules/klaw": { "version": "3.0.0", "resolved": "https://registry.npmjs.org/klaw/-/klaw-3.0.0.tgz", @@ -1365,6 +3541,30 @@ "graceful-fs": "^4.1.9" } }, + "node_modules/kleur": { + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/kleur/-/kleur-3.0.3.tgz", + "integrity": "sha512-eTIzlVOSUR+JxdDFepEYcBMtZ9Qqdef+rnzWdRZuMbOywu5tO2w2N7rqjoANZ5k9vywhL6Br1VRjUIgTQx4E8w==", + "dev": true, + "engines": { + "node": ">=6" + } + }, + "node_modules/leven": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/leven/-/leven-3.1.0.tgz", + "integrity": "sha512-qsda+H8jTaUaN/x5vzW2rzc+8Rw4TAQ/4KjB46IwK5VH+IlVeeeje/EoZRpiXvIqjFgK84QffqPztGI3VBLG1A==", + "dev": true, + "engines": { + "node": ">=6" + } + }, + "node_modules/lines-and-columns": { + "version": "1.2.4", + "resolved": "https://registry.npmjs.org/lines-and-columns/-/lines-and-columns-1.2.4.tgz", + "integrity": "sha512-7ylylesZQ/PV29jhEDl3Ufjo6ZX7gCqJr5F7PKrqc93v7fzSymt1BpwEU8nAUXs8qzzvqhbjhK5QZg6Mt/HkBg==", + "dev": true + }, "node_modules/linkify-it": { "version": "3.0.3", "resolved": "https://registry.npmjs.org/linkify-it/-/linkify-it-3.0.3.tgz", @@ -1375,18 +3575,15 @@ } }, "node_modules/locate-path": { - "version": "6.0.0", - "resolved": "https://registry.npmjs.org/locate-path/-/locate-path-6.0.0.tgz", - "integrity": "sha512-iPZK6eYjbxRu3uB4/WZ3EsEIMJFMqAoopl3R+zuq0UjcAm/MO6KCweDgPfP3elTztoKP3KtnVHxTn2NHBSDVUw==", + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/locate-path/-/locate-path-5.0.0.tgz", + "integrity": "sha512-t7hw9pI+WvuwNJXwk5zVHpyhIqzg2qTlklJOf0mVxGSbe3Fp2VieZcduNYjaLDoy6p9uGpQEGWG87WpMKlNq8g==", "dev": true, "dependencies": { - "p-locate": "^5.0.0" + "p-locate": "^4.1.0" }, "engines": { - "node": ">=10" - }, - "funding": { - "url": "https://github.com/sponsors/sindresorhus" + "node": ">=8" } }, "node_modules/lodash": { @@ -1412,12 +3609,12 @@ } }, "node_modules/lru-cache": { - "version": "7.18.3", - "resolved": "https://registry.npmjs.org/lru-cache/-/lru-cache-7.18.3.tgz", - "integrity": "sha512-jumlc0BIUrS3qJGgIkWZsyfAM7NCWiBcCDhnd+3NNM5KbBmLTgHVfWBcg6W+rLUsIpzpERPsvwUP7CckAQSOoA==", + "version": "5.1.1", + "resolved": "https://registry.npmjs.org/lru-cache/-/lru-cache-5.1.1.tgz", + "integrity": "sha512-KpNARQA3Iwv+jTA0utUVVbrh+Jlrr1Fv0e56GGzAFOXN7dk/FviaDW8LHmK52DlcH4WP2n6gI8vN1aesBFgo9w==", "dev": true, - "engines": { - "node": ">=12" + "dependencies": { + "yallist": "^3.0.2" } }, "node_modules/make-dir": { @@ -1443,29 +3640,48 @@ } }, "node_modules/make-fetch-happen": { - "version": "11.1.1", - "resolved": "https://registry.npmjs.org/make-fetch-happen/-/make-fetch-happen-11.1.1.tgz", - "integrity": "sha512-rLWS7GCSTcEujjVBs2YqG7Y4643u8ucvCJeSRqiLYhesrDuzeuFIk37xREzAsfQaqzl8b9rNCE4m6J8tvX4Q8w==", + "version": "10.2.1", + "resolved": "https://registry.npmjs.org/make-fetch-happen/-/make-fetch-happen-10.2.1.tgz", + "integrity": "sha512-NgOPbRiaQM10DYXvN3/hhGVI2M5MtITFryzBGxHM5p4wnFxsVCbxkrBrDsk+EZ5OB4jEOT7AjDxtdF+KVEFT7w==", "dev": true, "dependencies": { "agentkeepalive": "^4.2.1", - "cacache": "^17.0.0", - "http-cache-semantics": "^4.1.1", + "cacache": "^16.1.0", + "http-cache-semantics": "^4.1.0", "http-proxy-agent": "^5.0.0", "https-proxy-agent": "^5.0.0", "is-lambda": "^1.0.1", "lru-cache": "^7.7.1", - "minipass": "^5.0.0", - "minipass-fetch": "^3.0.0", + "minipass": "^3.1.6", + "minipass-collect": "^1.0.2", + "minipass-fetch": "^2.0.3", "minipass-flush": "^1.0.5", "minipass-pipeline": "^1.2.4", "negotiator": "^0.6.3", "promise-retry": "^2.0.1", "socks-proxy-agent": "^7.0.0", - "ssri": "^10.0.0" + "ssri": "^9.0.0" }, "engines": { - "node": "^14.17.0 || ^16.13.0 || >=18.0.0" + "node": "^12.13.0 || ^14.15.0 || >=16.0.0" + } + }, + "node_modules/make-fetch-happen/node_modules/lru-cache": { + "version": "7.18.3", + "resolved": "https://registry.npmjs.org/lru-cache/-/lru-cache-7.18.3.tgz", + "integrity": "sha512-jumlc0BIUrS3qJGgIkWZsyfAM7NCWiBcCDhnd+3NNM5KbBmLTgHVfWBcg6W+rLUsIpzpERPsvwUP7CckAQSOoA==", + "dev": true, + "engines": { + "node": ">=12" + } + }, + "node_modules/makeerror": { + "version": "1.0.12", + "resolved": "https://registry.npmjs.org/makeerror/-/makeerror-1.0.12.tgz", + "integrity": "sha512-JmqCvUhmt43madlpFzG4BQzG2Z3m6tvQDNKdClZnO3VbIudJYmxsT0FNJMeiB2+JTSlTQTSbU8QdesVmwJcmLg==", + "dev": true, + "dependencies": { + "tmpl": "1.0.5" } }, "node_modules/markdown-it": { @@ -1494,6 +3710,12 @@ "markdown-it": "*" } }, + "node_modules/markdown-it/node_modules/argparse": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/argparse/-/argparse-2.0.1.tgz", + "integrity": "sha512-8+9WqebbFzpX9OR+Wa6O29asIogeRMzcGtAINdpMHHyAg10f05aSFVBbcEqGf/PXw1EjAZ+q2/bEBg3DvurK3Q==", + "dev": true + }, "node_modules/markdown-it/node_modules/entities": { "version": "2.1.0", "resolved": "https://registry.npmjs.org/entities/-/entities-2.1.0.tgz", @@ -1521,76 +3743,85 @@ "integrity": "sha512-/sKlQJCBYVY9Ers9hqzKou4H6V5UWc/M59TH2dvkt+84itfnq7uFOMLpOiOS4ujvHP4etln18fmIxA5R5fll0g==", "dev": true }, - "node_modules/minimatch": { - "version": "3.0.8", - "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-3.0.8.tgz", - "integrity": "sha512-6FsRAQsxQ61mw+qP1ZzbL9Bc78x2p5OqNgNpnoAFLTrX8n5Kxph0CsnhmKKNXTWjXqU5L0pGPR7hYk+XWZr60Q==", + "node_modules/merge-stream": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/merge-stream/-/merge-stream-2.0.0.tgz", + "integrity": "sha512-abv/qOcuPfk3URPfDzmZU1LKmuw8kT+0nIHvKrKgFrwifol/doWcdA4ZqsWQ8ENrFKkd67Mfpo/LovbIUsbt3w==", + "dev": true + }, + "node_modules/micromatch": { + "version": "4.0.5", + "resolved": "https://registry.npmjs.org/micromatch/-/micromatch-4.0.5.tgz", + "integrity": "sha512-DMy+ERcEW2q8Z2Po+WNXuw3c5YaUSFjAO5GsJqfEl7UjvtIuFKO6ZrKvcItdy98dwFI2N1tg3zNIdKaQT+aNdA==", "dev": true, "dependencies": { - "brace-expansion": "^1.1.7" + "braces": "^3.0.2", + "picomatch": "^2.3.1" }, "engines": { - "node": "*" + "node": ">=8.6" } }, - "node_modules/minipass": { - "version": "5.0.0", - "resolved": "https://registry.npmjs.org/minipass/-/minipass-5.0.0.tgz", - "integrity": "sha512-3FnjYuehv9k6ovOEbyOswadCDPX1piCfhV8ncmYtHOjuPwylVWsghTLo7rabjC3Rx5xD4HDx8Wm1xnMF7S5qFQ==", + "node_modules/mimic-fn": { + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/mimic-fn/-/mimic-fn-2.1.0.tgz", + "integrity": "sha512-OqbOk5oEQeAZ8WXWydlu9HJjz9WVdEIvamMCcXmuqUYjTknH/sqsWvhQ3vgwKFRR1HpjvNBKQ37nbJgYzGqGcg==", + "dev": true, "engines": { - "node": ">=8" + "node": ">=6" } }, - "node_modules/minipass-collect": { - "version": "1.0.2", - "resolved": "https://registry.npmjs.org/minipass-collect/-/minipass-collect-1.0.2.tgz", - "integrity": "sha512-6T6lH0H8OG9kITm/Jm6tdooIbogG9e0tLgpY6mphXSm/A9u8Nq1ryBG+Qspiub9LjWlBPsPS3tWQ/Botq4FdxA==", - "dev": true, + "node_modules/minimatch": { + "version": "3.1.2", + "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-3.1.2.tgz", + "integrity": "sha512-J7p63hRiAjw1NDEww1W7i37+ByIrOWO5XQQAzZ3VOcL0PNybwpfmV/N05zFAzwQ9USyEcX6t3UO+K5aqBQOIHw==", "dependencies": { - "minipass": "^3.0.0" + "brace-expansion": "^1.1.7" }, "engines": { - "node": ">= 8" + "node": "*" } }, - "node_modules/minipass-collect/node_modules/minipass": { + "node_modules/minipass": { "version": "3.3.6", "resolved": "https://registry.npmjs.org/minipass/-/minipass-3.3.6.tgz", "integrity": "sha512-DxiNidxSEK+tHG6zOIklvNOwm3hvCrbUrdtzY74U6HKTJxvIDfOUL5W5P2Ghd3DTkhhKPYGqeNUIh5qcM4YBfw==", + "dependencies": { + "yallist": "^4.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/minipass-collect": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/minipass-collect/-/minipass-collect-1.0.2.tgz", + "integrity": "sha512-6T6lH0H8OG9kITm/Jm6tdooIbogG9e0tLgpY6mphXSm/A9u8Nq1ryBG+Qspiub9LjWlBPsPS3tWQ/Botq4FdxA==", "dev": true, "dependencies": { - "yallist": "^4.0.0" + "minipass": "^3.0.0" }, "engines": { - "node": ">=8" + "node": ">= 8" } }, "node_modules/minipass-fetch": { - "version": "3.0.4", - "resolved": "https://registry.npmjs.org/minipass-fetch/-/minipass-fetch-3.0.4.tgz", - "integrity": "sha512-jHAqnA728uUpIaFm7NWsCnqKT6UqZz7GcI/bDpPATuwYyKwJwW0remxSCxUlKiEty+eopHGa3oc8WxgQ1FFJqg==", + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/minipass-fetch/-/minipass-fetch-2.1.2.tgz", + "integrity": "sha512-LT49Zi2/WMROHYoqGgdlQIZh8mLPZmOrN2NdJjMXxYe4nkN6FUyuPuOAOedNJDrx0IRGg9+4guZewtp8hE6TxA==", "dev": true, "dependencies": { - "minipass": "^7.0.3", + "minipass": "^3.1.6", "minipass-sized": "^1.0.3", "minizlib": "^2.1.2" }, "engines": { - "node": "^14.17.0 || ^16.13.0 || >=18.0.0" + "node": "^12.13.0 || ^14.15.0 || >=16.0.0" }, "optionalDependencies": { "encoding": "^0.1.13" } }, - "node_modules/minipass-fetch/node_modules/minipass": { - "version": "7.0.4", - "resolved": "https://registry.npmjs.org/minipass/-/minipass-7.0.4.tgz", - "integrity": "sha512-jYofLM5Dam9279rdkWzqHozUo4ybjdZmCsDHePy5V/PbBcVMiSZR97gmAy45aqi8CK1lG2ECd356FU86avfwUQ==", - "dev": true, - "engines": { - "node": ">=16 || 14 >=14.17" - } - }, "node_modules/minipass-flush": { "version": "1.0.5", "resolved": "https://registry.npmjs.org/minipass-flush/-/minipass-flush-1.0.5.tgz", @@ -1603,18 +3834,6 @@ "node": ">= 8" } }, - "node_modules/minipass-flush/node_modules/minipass": { - "version": "3.3.6", - "resolved": "https://registry.npmjs.org/minipass/-/minipass-3.3.6.tgz", - "integrity": "sha512-DxiNidxSEK+tHG6zOIklvNOwm3hvCrbUrdtzY74U6HKTJxvIDfOUL5W5P2Ghd3DTkhhKPYGqeNUIh5qcM4YBfw==", - "dev": true, - "dependencies": { - "yallist": "^4.0.0" - }, - "engines": { - "node": ">=8" - } - }, "node_modules/minipass-pipeline": { "version": "1.2.4", "resolved": "https://registry.npmjs.org/minipass-pipeline/-/minipass-pipeline-1.2.4.tgz", @@ -1627,18 +3846,6 @@ "node": ">=8" } }, - "node_modules/minipass-pipeline/node_modules/minipass": { - "version": "3.3.6", - "resolved": "https://registry.npmjs.org/minipass/-/minipass-3.3.6.tgz", - "integrity": "sha512-DxiNidxSEK+tHG6zOIklvNOwm3hvCrbUrdtzY74U6HKTJxvIDfOUL5W5P2Ghd3DTkhhKPYGqeNUIh5qcM4YBfw==", - "dev": true, - "dependencies": { - "yallist": "^4.0.0" - }, - "engines": { - "node": ">=8" - } - }, "node_modules/minipass-sized": { "version": "1.0.3", "resolved": "https://registry.npmjs.org/minipass-sized/-/minipass-sized-1.0.3.tgz", @@ -1651,17 +3858,10 @@ "node": ">=8" } }, - "node_modules/minipass-sized/node_modules/minipass": { - "version": "3.3.6", - "resolved": "https://registry.npmjs.org/minipass/-/minipass-3.3.6.tgz", - "integrity": "sha512-DxiNidxSEK+tHG6zOIklvNOwm3hvCrbUrdtzY74U6HKTJxvIDfOUL5W5P2Ghd3DTkhhKPYGqeNUIh5qcM4YBfw==", - "dev": true, - "dependencies": { - "yallist": "^4.0.0" - }, - "engines": { - "node": ">=8" - } + "node_modules/minipass/node_modules/yallist": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/yallist/-/yallist-4.0.0.tgz", + "integrity": "sha512-3wdGidZyq5PB084XLES5TpOSRA3wjXAlIWMhum2kRcv/41Sn2emQ0dycQW4uZXLejwKvg6EsvbdlVL+FYEct7A==" }, "node_modules/minizlib": { "version": "2.1.2", @@ -1675,16 +3875,10 @@ "node": ">= 8" } }, - "node_modules/minizlib/node_modules/minipass": { - "version": "3.3.6", - "resolved": "https://registry.npmjs.org/minipass/-/minipass-3.3.6.tgz", - "integrity": "sha512-DxiNidxSEK+tHG6zOIklvNOwm3hvCrbUrdtzY74U6HKTJxvIDfOUL5W5P2Ghd3DTkhhKPYGqeNUIh5qcM4YBfw==", - "dependencies": { - "yallist": "^4.0.0" - }, - "engines": { - "node": ">=8" - } + "node_modules/minizlib/node_modules/yallist": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/yallist/-/yallist-4.0.0.tgz", + "integrity": "sha512-3wdGidZyq5PB084XLES5TpOSRA3wjXAlIWMhum2kRcv/41Sn2emQ0dycQW4uZXLejwKvg6EsvbdlVL+FYEct7A==" }, "node_modules/mkdirp": { "version": "1.0.4", @@ -1737,6 +3931,23 @@ "url": "https://opencollective.com/mochajs" } }, + "node_modules/mocha/node_modules/argparse": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/argparse/-/argparse-2.0.1.tgz", + "integrity": "sha512-8+9WqebbFzpX9OR+Wa6O29asIogeRMzcGtAINdpMHHyAg10f05aSFVBbcEqGf/PXw1EjAZ+q2/bEBg3DvurK3Q==", + "dev": true + }, + "node_modules/mocha/node_modules/cliui": { + "version": "7.0.4", + "resolved": "https://registry.npmjs.org/cliui/-/cliui-7.0.4.tgz", + "integrity": "sha512-OcRE68cOsVMXp1Yvonl/fzkQOyjLSu/8bhPDfQt0e0/Eb283TKP20Fs2MqoPsr9SwA595rRCA+QMzYc9nBP+JQ==", + "dev": true, + "dependencies": { + "string-width": "^4.2.0", + "strip-ansi": "^6.0.0", + "wrap-ansi": "^7.0.0" + } + }, "node_modules/mocha/node_modules/escape-string-regexp": { "version": "4.0.0", "resolved": "https://registry.npmjs.org/escape-string-regexp/-/escape-string-regexp-4.0.0.tgz", @@ -1749,6 +3960,22 @@ "url": "https://github.com/sponsors/sindresorhus" } }, + "node_modules/mocha/node_modules/find-up": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/find-up/-/find-up-5.0.0.tgz", + "integrity": "sha512-78/PXT1wlLLDgTzDs7sjq9hzz0vXD+zn+7wypEe4fXQxCmdmqfGsEPQxmiCSQI3ajFV91bVSsvNtrJRiW6nGng==", + "dev": true, + "dependencies": { + "locate-path": "^6.0.0", + "path-exists": "^4.0.0" + }, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, "node_modules/mocha/node_modules/glob": { "version": "7.2.0", "resolved": "https://registry.npmjs.org/glob/-/glob-7.2.0.tgz", @@ -1781,6 +4008,33 @@ "node": "*" } }, + "node_modules/mocha/node_modules/js-yaml": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/js-yaml/-/js-yaml-4.1.0.tgz", + "integrity": "sha512-wpxZs9NoxZaJESJGIZTyDEaYpl0FKSA+FB9aJiyemKhMwkxQg63h4T1KJgUGHpTqPDNRcmmYLugrRjJlBtWvRA==", + "dev": true, + "dependencies": { + "argparse": "^2.0.1" + }, + "bin": { + "js-yaml": "bin/js-yaml.js" + } + }, + "node_modules/mocha/node_modules/locate-path": { + "version": "6.0.0", + "resolved": "https://registry.npmjs.org/locate-path/-/locate-path-6.0.0.tgz", + "integrity": "sha512-iPZK6eYjbxRu3uB4/WZ3EsEIMJFMqAoopl3R+zuq0UjcAm/MO6KCweDgPfP3elTztoKP3KtnVHxTn2NHBSDVUw==", + "dev": true, + "dependencies": { + "p-locate": "^5.0.0" + }, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, "node_modules/mocha/node_modules/minimatch": { "version": "5.0.1", "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-5.0.1.tgz", @@ -1802,12 +4056,65 @@ "balanced-match": "^1.0.0" } }, - "node_modules/ms": { + "node_modules/mocha/node_modules/ms": { "version": "2.1.3", "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.3.tgz", "integrity": "sha512-6FlzubTLZG3J2a/NVCAleEhjzq5oxgHyaCU9yYXvcLsvoVaHJq/s5xXI6/XXP6tz7R9xAOtHnSO/tXtF3WRTlA==", "dev": true }, + "node_modules/mocha/node_modules/p-locate": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/p-locate/-/p-locate-5.0.0.tgz", + "integrity": "sha512-LaNjtRWUBY++zB5nE/NwcaoMylSPk+S+ZHNB1TzdbMJMny6dynpAGt7X/tl/QYq3TIeE6nxHppbo2LGymrG5Pw==", + "dev": true, + "dependencies": { + "p-limit": "^3.0.2" + }, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/mocha/node_modules/supports-color": { + "version": "8.1.1", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-8.1.1.tgz", + "integrity": "sha512-MpUEN2OodtUzxvKQl72cUF7RQ5EiHsGvSsVG0ia9c5RbWGL2CI4C7EpPS8UTBIplnlzZiNuV56w+FuNxy3ty2Q==", + "dev": true, + "dependencies": { + "has-flag": "^4.0.0" + }, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/chalk/supports-color?sponsor=1" + } + }, + "node_modules/mocha/node_modules/yargs": { + "version": "16.2.0", + "resolved": "https://registry.npmjs.org/yargs/-/yargs-16.2.0.tgz", + "integrity": "sha512-D1mvvtDG0L5ft/jGWkLpG1+m0eQxOfaBvTNELraWj22wSVUMWxZUvYgJYcKh6jGGIkJFhH4IZPQhR4TKpc8mBw==", + "dev": true, + "dependencies": { + "cliui": "^7.0.2", + "escalade": "^3.1.1", + "get-caller-file": "^2.0.5", + "require-directory": "^2.1.1", + "string-width": "^4.2.0", + "y18n": "^5.0.5", + "yargs-parser": "^20.2.2" + }, + "engines": { + "node": ">=10" + } + }, + "node_modules/ms": { + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.2.tgz", + "integrity": "sha512-sGkPx+VjMtmA6MX27oA4FBFELFCZZ4S4XqeGOXCv68tT+jb3vk/RyaKWP0PTKyWtmLSM0b+adUTEvbs1PEaH2w==" + }, "node_modules/nan": { "version": "2.18.0", "resolved": "https://registry.npmjs.org/nan/-/nan-2.18.0.tgz", @@ -1825,6 +4132,12 @@ "node": "^10 || ^12 || ^13.7 || ^14 || >=15.0.1" } }, + "node_modules/natural-compare": { + "version": "1.4.0", + "resolved": "https://registry.npmjs.org/natural-compare/-/natural-compare-1.4.0.tgz", + "integrity": "sha512-OWND8ei3VtNC9h7V60qff3SVobHr996CTwgxubgyQYEpg290h9J0buyECNNJexkFm5sOajh5G116RYA1c8ZMSw==", + "dev": true + }, "node_modules/negotiator": { "version": "0.6.3", "resolved": "https://registry.npmjs.org/negotiator/-/negotiator-0.6.3.tgz", @@ -1835,9 +4148,9 @@ } }, "node_modules/node-fetch": { - "version": "2.6.12", - "resolved": "https://registry.npmjs.org/node-fetch/-/node-fetch-2.6.12.tgz", - "integrity": "sha512-C/fGU2E8ToujUivIO0H+tpQ6HWo4eEmchoPIoXtxCrVghxdKq+QOHqEZW7tuP3KlV3bC8FRMO5nMCC7Zm1VP6g==", + "version": "2.7.0", + "resolved": "https://registry.npmjs.org/node-fetch/-/node-fetch-2.7.0.tgz", + "integrity": "sha512-c4FRfUm/dbcWZ7U+1Wq0AwCyFL+3nt2bEw05wfxSz+DWpWsitgmSgYmy2dQdWyKC1694ELPqMs/YzUSNozLt8A==", "dependencies": { "whatwg-url": "^5.0.0" }, @@ -1854,16 +4167,16 @@ } }, "node_modules/node-gyp": { - "version": "9.4.0", - "resolved": "https://registry.npmjs.org/node-gyp/-/node-gyp-9.4.0.tgz", - "integrity": "sha512-dMXsYP6gc9rRbejLXmTbVRYjAHw7ppswsKyMxuxJxxOHzluIO1rGp9TOQgjFJ+2MCqcOcQTOPB/8Xwhr+7s4Eg==", + "version": "9.4.1", + "resolved": "https://registry.npmjs.org/node-gyp/-/node-gyp-9.4.1.tgz", + "integrity": "sha512-OQkWKbjQKbGkMf/xqI1jjy3oCTgMKJac58G2+bjZb3fza6gW2YrCSdMQYaoTb70crvE//Gngr4f0AgVHmqHvBQ==", "dev": true, "dependencies": { "env-paths": "^2.2.0", "exponential-backoff": "^3.1.1", "glob": "^7.1.4", "graceful-fs": "^4.2.6", - "make-fetch-happen": "^11.0.3", + "make-fetch-happen": "^10.0.3", "nopt": "^6.0.0", "npmlog": "^6.0.0", "rimraf": "^3.0.2", @@ -1963,6 +4276,18 @@ "safe-buffer": "~5.2.0" } }, + "node_modules/node-int64": { + "version": "0.4.0", + "resolved": "https://registry.npmjs.org/node-int64/-/node-int64-0.4.0.tgz", + "integrity": "sha512-O5lz91xSOeoXP6DulyHfllpq+Eg00MWitZIbtPfoSEvqIHdl5gfcY6hYzDWnj0qD5tz52PI08u9qUvSVeUBeHw==", + "dev": true + }, + "node_modules/node-releases": { + "version": "2.0.14", + "resolved": "https://registry.npmjs.org/node-releases/-/node-releases-2.0.14.tgz", + "integrity": "sha512-y10wOWt8yZpqXmOgRo77WaHEmhYQYGNA6y421PKsKYWEK8aW+cqAphborZDhqfyKrbZEN92CN1X2KbafY2s7Yw==", + "dev": true + }, "node_modules/nopt": { "version": "5.0.0", "resolved": "https://registry.npmjs.org/nopt/-/nopt-5.0.0.tgz", @@ -1986,6 +4311,18 @@ "node": ">=0.10.0" } }, + "node_modules/npm-run-path": { + "version": "4.0.1", + "resolved": "https://registry.npmjs.org/npm-run-path/-/npm-run-path-4.0.1.tgz", + "integrity": "sha512-S48WzZW777zhNIrn7gxOlISNAqi9ZC/uQFnRdbeIHhZhCA6UqpkOT8T1G7BvfdgP4Er8gF4sUbaS0i7QvIfCWw==", + "dev": true, + "dependencies": { + "path-key": "^3.0.0" + }, + "engines": { + "node": ">=8" + } + }, "node_modules/npmlog": { "version": "5.0.1", "resolved": "https://registry.npmjs.org/npmlog/-/npmlog-5.0.1.tgz", @@ -2013,6 +4350,21 @@ "wrappy": "1" } }, + "node_modules/onetime": { + "version": "5.1.2", + "resolved": "https://registry.npmjs.org/onetime/-/onetime-5.1.2.tgz", + "integrity": "sha512-kbpaSSGJTWdAY5KPVeMOKXSrPtr8C8C7wodJbcsd51jRnmD+GZu8Y0VoU6Dm5Z4vWr0Ig/1NKuWRKf7j5aaYSg==", + "dev": true, + "dependencies": { + "mimic-fn": "^2.1.0" + }, + "engines": { + "node": ">=6" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, "node_modules/p-limit": { "version": "3.1.0", "resolved": "https://registry.npmjs.org/p-limit/-/p-limit-3.1.0.tgz", @@ -2029,15 +4381,27 @@ } }, "node_modules/p-locate": { - "version": "5.0.0", - "resolved": "https://registry.npmjs.org/p-locate/-/p-locate-5.0.0.tgz", - "integrity": "sha512-LaNjtRWUBY++zB5nE/NwcaoMylSPk+S+ZHNB1TzdbMJMny6dynpAGt7X/tl/QYq3TIeE6nxHppbo2LGymrG5Pw==", + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/p-locate/-/p-locate-4.1.0.tgz", + "integrity": "sha512-R79ZZ/0wAxKGu3oYMlz8jy/kbhsNrS7SKZ7PxEHBgJ5+F2mtFW2fK2cOtBh1cHYkQsbzFV7I+EoRKe6Yt0oK7A==", "dev": true, "dependencies": { - "p-limit": "^3.0.2" + "p-limit": "^2.2.0" }, "engines": { - "node": ">=10" + "node": ">=8" + } + }, + "node_modules/p-locate/node_modules/p-limit": { + "version": "2.3.0", + "resolved": "https://registry.npmjs.org/p-limit/-/p-limit-2.3.0.tgz", + "integrity": "sha512-//88mFWSJx8lxCzwdAABTJL2MyWB12+eIY7MDL2SqLmAkeKU9qxRvWuSyTjm3FUmpBEMuFfckAIqEaVGUDxb6w==", + "dev": true, + "dependencies": { + "p-try": "^2.0.0" + }, + "engines": { + "node": ">=6" }, "funding": { "url": "https://github.com/sponsors/sindresorhus" @@ -2058,6 +4422,33 @@ "url": "https://github.com/sponsors/sindresorhus" } }, + "node_modules/p-try": { + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/p-try/-/p-try-2.2.0.tgz", + "integrity": "sha512-R4nPAVTAU0B9D35/Gk3uJf/7XYbQcyohSKdvAxIRSNghFl4e71hVoGnBNQz9cWaXxO2I10KTC+3jMdvvoKw6dQ==", + "dev": true, + "engines": { + "node": ">=6" + } + }, + "node_modules/parse-json": { + "version": "5.2.0", + "resolved": "https://registry.npmjs.org/parse-json/-/parse-json-5.2.0.tgz", + "integrity": "sha512-ayCKvm/phCGxOkYRSCM82iDwct8/EonSEgCSxWxD7ve6jHggsFl4fZVQBPRNgQoKiuV/odhFrGzQXZwbifC8Rg==", + "dev": true, + "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" + }, + "engines": { + "node": ">=8" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, "node_modules/path-exists": { "version": "4.0.0", "resolved": "https://registry.npmjs.org/path-exists/-/path-exists-4.0.0.tgz", @@ -2084,43 +4475,83 @@ "node": ">=8" } }, - "node_modules/path-scurry": { - "version": "1.10.1", - "resolved": "https://registry.npmjs.org/path-scurry/-/path-scurry-1.10.1.tgz", - "integrity": "sha512-MkhCqzzBEpPvxxQ71Md0b1Kk51W01lrYvlMzSUaIzNsODdd7mqhiimSZlr+VegAz5Z6Vzt9Xg2ttE//XBhH3EQ==", + "node_modules/path-parse": { + "version": "1.0.7", + "resolved": "https://registry.npmjs.org/path-parse/-/path-parse-1.0.7.tgz", + "integrity": "sha512-LDJzPVEEEPR+y48z93A0Ed0yXb8pAByGWo/k5YYdYgpY2/2EsOsksJrq7lOHxryrVOn1ejG6oAp8ahvOIQD8sw==", + "dev": true + }, + "node_modules/picocolors": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/picocolors/-/picocolors-1.0.0.tgz", + "integrity": "sha512-1fygroTLlHu66zi26VoTDv8yRgm0Fccecssto+MhsZ0D/DGW2sm8E8AjW7NU5VVTRt5GxbeZ5qBuJr+HyLYkjQ==", + "dev": true + }, + "node_modules/picomatch": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/picomatch/-/picomatch-2.3.1.tgz", + "integrity": "sha512-JU3teHTNjmE2VCGFzuY8EXzCDVwEqB2a8fsIvwaStHhAWJEeVd1o1QD80CU6+ZdEXXSLbSsuLwJjkCBWqRQUVA==", "dev": true, - "dependencies": { - "lru-cache": "^9.1.1 || ^10.0.0", - "minipass": "^5.0.0 || ^6.0.2 || ^7.0.0" - }, "engines": { - "node": ">=16 || 14 >=14.17" + "node": ">=8.6" }, "funding": { - "url": "https://github.com/sponsors/isaacs" + "url": "https://github.com/sponsors/jonschlinkert" } }, - "node_modules/path-scurry/node_modules/lru-cache": { - "version": "10.0.1", - "resolved": "https://registry.npmjs.org/lru-cache/-/lru-cache-10.0.1.tgz", - "integrity": "sha512-IJ4uwUTi2qCccrioU6g9g/5rvvVl13bsdczUUcqbciD9iLr095yj8DQKdObriEvuNSx325N1rV1O0sJFszx75g==", + "node_modules/pirates": { + "version": "4.0.6", + "resolved": "https://registry.npmjs.org/pirates/-/pirates-4.0.6.tgz", + "integrity": "sha512-saLsH7WeYYPiD25LDuLRRY/i+6HaPYr6G1OUlN39otzkSTxKnubR9RTxS3/Kk50s1g2JTgFwWQDQyplC5/SHZg==", "dev": true, "engines": { - "node": "14 || >=16.14" + "node": ">= 6" } }, - "node_modules/picomatch": { - "version": "2.3.1", - "resolved": "https://registry.npmjs.org/picomatch/-/picomatch-2.3.1.tgz", - "integrity": "sha512-JU3teHTNjmE2VCGFzuY8EXzCDVwEqB2a8fsIvwaStHhAWJEeVd1o1QD80CU6+ZdEXXSLbSsuLwJjkCBWqRQUVA==", + "node_modules/pkg-dir": { + "version": "4.2.0", + "resolved": "https://registry.npmjs.org/pkg-dir/-/pkg-dir-4.2.0.tgz", + "integrity": "sha512-HRDzbaKjC+AOWVXxAU/x54COGeIv9eb+6CkDSQoNTt4XyWoIJvuPsXizxu/Fr23EiekbtZwmh1IcIG/l/a10GQ==", + "dev": true, + "dependencies": { + "find-up": "^4.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/pretty-format": { + "version": "29.7.0", + "resolved": "https://registry.npmjs.org/pretty-format/-/pretty-format-29.7.0.tgz", + "integrity": "sha512-Pdlw/oPxN+aXdmM9R00JVC9WVFoCLTKJvDVLgmJ+qAffBMxsV85l/Lu7sNx4zSzPyoL2euImuEwHhOXdEgNFZQ==", + "dev": true, + "dependencies": { + "@jest/schemas": "^29.6.3", + "ansi-styles": "^5.0.0", + "react-is": "^18.0.0" + }, + "engines": { + "node": "^14.15.0 || ^16.10.0 || >=18.0.0" + } + }, + "node_modules/pretty-format/node_modules/ansi-styles": { + "version": "5.2.0", + "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-5.2.0.tgz", + "integrity": "sha512-Cxwpt2SfTzTtXcfOlzGEee8O+c+MmUgGrNiBcXnuWxuFJHe6a5Hz7qwhwe5OgaSYI0IJvkLqWX1ASG+cJOkEiA==", "dev": true, "engines": { - "node": ">=8.6" + "node": ">=10" }, "funding": { - "url": "https://github.com/sponsors/jonschlinkert" + "url": "https://github.com/chalk/ansi-styles?sponsor=1" } }, + "node_modules/promise-inflight": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/promise-inflight/-/promise-inflight-1.0.1.tgz", + "integrity": "sha512-6zWPyEOFaQBJYcGMHBKTKJ3u6TBsnMFOIZSa6ce1e/ZrrsOlnHRHbabMjLiBYKp+n44X9eUI6VUPaukCXHuG4g==", + "dev": true + }, "node_modules/promise-retry": { "version": "2.0.1", "resolved": "https://registry.npmjs.org/promise-retry/-/promise-retry-2.0.1.tgz", @@ -2134,6 +4565,35 @@ "node": ">=10" } }, + "node_modules/prompts": { + "version": "2.4.2", + "resolved": "https://registry.npmjs.org/prompts/-/prompts-2.4.2.tgz", + "integrity": "sha512-NxNv/kLguCA7p3jE8oL2aEBsrJWgAakBpgmgK6lpPWV+WuOmY6r2/zbAVnP+T8bQlA0nzHXSJSJW0Hq7ylaD2Q==", + "dev": true, + "dependencies": { + "kleur": "^3.0.3", + "sisteransi": "^1.0.5" + }, + "engines": { + "node": ">= 6" + } + }, + "node_modules/pure-rand": { + "version": "6.0.4", + "resolved": "https://registry.npmjs.org/pure-rand/-/pure-rand-6.0.4.tgz", + "integrity": "sha512-LA0Y9kxMYv47GIPJy6MI84fqTd2HmYZI83W/kM/SkKfDlajnZYfmXFTxkbY+xSBPkLJxltMa9hIkmdc29eguMA==", + "dev": true, + "funding": [ + { + "type": "individual", + "url": "https://github.com/sponsors/dubzzz" + }, + { + "type": "opencollective", + "url": "https://opencollective.com/fast-check" + } + ] + }, "node_modules/randombytes": { "version": "2.1.0", "resolved": "https://registry.npmjs.org/randombytes/-/randombytes-2.1.0.tgz", @@ -2143,6 +4603,12 @@ "safe-buffer": "^5.1.0" } }, + "node_modules/react-is": { + "version": "18.2.0", + "resolved": "https://registry.npmjs.org/react-is/-/react-is-18.2.0.tgz", + "integrity": "sha512-xWGDIW6x921xtzPkhiULtthJHoJvBbF3q26fzloPCK0hsvxtPVelvftw3zjbHWSkR2km9Z+4uxbDDK/6Zw9B8w==", + "dev": true + }, "node_modules/readable-stream": { "version": "1.1.14", "resolved": "https://registry.npmjs.org/readable-stream/-/readable-stream-1.1.14.tgz", @@ -2185,6 +4651,53 @@ "lodash": "^4.17.21" } }, + "node_modules/resolve": { + "version": "1.22.8", + "resolved": "https://registry.npmjs.org/resolve/-/resolve-1.22.8.tgz", + "integrity": "sha512-oKWePCxqpd6FlLvGV1VU0x7bkPmmCNolxzjMf4NczoDnQcIWrAF+cPtZn5i6n+RfD2d9i0tzpKnG6Yk168yIyw==", + "dev": true, + "dependencies": { + "is-core-module": "^2.13.0", + "path-parse": "^1.0.7", + "supports-preserve-symlinks-flag": "^1.0.0" + }, + "bin": { + "resolve": "bin/resolve" + }, + "funding": { + "url": "https://github.com/sponsors/ljharb" + } + }, + "node_modules/resolve-cwd": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/resolve-cwd/-/resolve-cwd-3.0.0.tgz", + "integrity": "sha512-OrZaX2Mb+rJCpH/6CpSqt9xFVpN++x01XnN2ie9g6P5/3xelLAkXWVADpdz1IHD/KFfEXyE6V0U01OQ3UO2rEg==", + "dev": true, + "dependencies": { + "resolve-from": "^5.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/resolve-from": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/resolve-from/-/resolve-from-5.0.0.tgz", + "integrity": "sha512-qYg9KP24dD5qka9J47d0aVky0N+b4fTU89LN9iDnjB5waksiC49rvMB0PrUJQGoTmH50XPiqOvAjDfaijGxYZw==", + "dev": true, + "engines": { + "node": ">=8" + } + }, + "node_modules/resolve.exports": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/resolve.exports/-/resolve.exports-2.0.2.tgz", + "integrity": "sha512-X2UW6Nw3n/aMgDVy+0rSqgHlv39WZAlZrXCdnbyEiKm17DSqHX4MmQMaST3FbeWR5FTuRcUwYAziZajji0Y7mg==", + "dev": true, + "engines": { + "node": ">=10" + } + }, "node_modules/retry": { "version": "0.12.0", "resolved": "https://registry.npmjs.org/retry/-/retry-0.12.0.tgz", @@ -2258,6 +4771,11 @@ "node": ">=10" } }, + "node_modules/semver/node_modules/yallist": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/yallist/-/yallist-4.0.0.tgz", + "integrity": "sha512-3wdGidZyq5PB084XLES5TpOSRA3wjXAlIWMhum2kRcv/41Sn2emQ0dycQW4uZXLejwKvg6EsvbdlVL+FYEct7A==" + }, "node_modules/serialize-javascript": { "version": "6.0.0", "resolved": "https://registry.npmjs.org/serialize-javascript/-/serialize-javascript-6.0.0.tgz", @@ -2298,6 +4816,21 @@ "resolved": "https://registry.npmjs.org/signal-exit/-/signal-exit-3.0.7.tgz", "integrity": "sha512-wnD2ZE+l+SPC/uoS0vXeE9L1+0wuaMqKlfz9AMUo38JsyLSBWSFcHR1Rri62LZc12vLr1gb3jl7iwQhgwpAbGQ==" }, + "node_modules/sisteransi": { + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/sisteransi/-/sisteransi-1.0.5.tgz", + "integrity": "sha512-bLGGlR1QxBcynn2d5YmDX4MGjlZvy2MRBDRNHLJ8VI6l6+9FUiyTFNJ0IveOSP0bcXgVDPRcfGqA0pjaqUpfVg==", + "dev": true + }, + "node_modules/slash": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/slash/-/slash-3.0.0.tgz", + "integrity": "sha512-g9Q1haeby36OSStwb4ntCGGGaKsaVSjQ68fBxoQcutl5fS1vuY18H3wSt3jFyFtrkx+Kz0V1G85A4MyAdDMi2Q==", + "dev": true, + "engines": { + "node": ">=8" + } + }, "node_modules/smart-buffer": { "version": "4.2.0", "resolved": "https://registry.npmjs.org/smart-buffer/-/smart-buffer-4.2.0.tgz", @@ -2336,25 +4869,53 @@ "node": ">= 10" } }, + "node_modules/source-map": { + "version": "0.6.1", + "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.6.1.tgz", + "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==", + "dev": true, + "engines": { + "node": ">=0.10.0" + } + }, + "node_modules/source-map-support": { + "version": "0.5.13", + "resolved": "https://registry.npmjs.org/source-map-support/-/source-map-support-0.5.13.tgz", + "integrity": "sha512-SHSKFHadjVA5oR4PPqhtAVdcBWwRYVd6g6cAXnIbRiIwc2EhPrTuKUBdSLvlEKyIP3GCf89fltvcZiP9MMFA1w==", + "dev": true, + "dependencies": { + "buffer-from": "^1.0.0", + "source-map": "^0.6.0" + } + }, + "node_modules/sprintf-js": { + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/sprintf-js/-/sprintf-js-1.0.3.tgz", + "integrity": "sha512-D9cPgkvLlV3t3IzL0D0YLvGA9Ahk4PcvVwUbN0dSGr1aP0Nrt4AEnTUbuGvquEC0mA64Gqt1fzirlRs5ibXx8g==", + "dev": true + }, "node_modules/ssri": { - "version": "10.0.5", - "resolved": "https://registry.npmjs.org/ssri/-/ssri-10.0.5.tgz", - "integrity": "sha512-bSf16tAFkGeRlUNDjXu8FzaMQt6g2HZJrun7mtMbIPOddxt3GLMSz5VWUWcqTJUPfLEaDIepGxv+bYQW49596A==", + "version": "9.0.1", + "resolved": "https://registry.npmjs.org/ssri/-/ssri-9.0.1.tgz", + "integrity": "sha512-o57Wcn66jMQvfHG1FlYbWeZWW/dHZhJXjpIcTfXldXEk5nz5lStPo3mK0OJQfGR3RbZUlbISexbljkJzuEj/8Q==", "dev": true, "dependencies": { - "minipass": "^7.0.3" + "minipass": "^3.1.1" }, "engines": { - "node": "^14.17.0 || ^16.13.0 || >=18.0.0" + "node": "^12.13.0 || ^14.15.0 || >=16.0.0" } }, - "node_modules/ssri/node_modules/minipass": { - "version": "7.0.4", - "resolved": "https://registry.npmjs.org/minipass/-/minipass-7.0.4.tgz", - "integrity": "sha512-jYofLM5Dam9279rdkWzqHozUo4ybjdZmCsDHePy5V/PbBcVMiSZR97gmAy45aqi8CK1lG2ECd356FU86avfwUQ==", + "node_modules/stack-utils": { + "version": "2.0.6", + "resolved": "https://registry.npmjs.org/stack-utils/-/stack-utils-2.0.6.tgz", + "integrity": "sha512-XlkWvfIm6RmsWtNJx+uqtKLS8eqFbxUg0ZzLXqY0caEy9l7hruX8IpiDnjsLavoBgqCCR71TqWO8MaXYheJ3RQ==", "dev": true, + "dependencies": { + "escape-string-regexp": "^2.0.0" + }, "engines": { - "node": ">=16 || 14 >=14.17" + "node": ">=10" } }, "node_modules/string_decoder": { @@ -2363,25 +4924,23 @@ "integrity": "sha512-ev2QzSzWPYmy9GuqfIVildA4OdcGLeFZQrq5ys6RtiuF+RQQiZWr8TZNyAcuVXyQRYfEO+MsoB/1BuQVhOJuoQ==", "dev": true }, - "node_modules/string-width": { - "version": "4.2.3", - "resolved": "https://registry.npmjs.org/string-width/-/string-width-4.2.3.tgz", - "integrity": "sha512-wKyQRQpjJ0sIp62ErSZdGsjMJWsap5oRNihHhu6G7JVO/9jIB6UyevL+tXuOqrng8j/cxKTWyWUwvSTriiZz/g==", + "node_modules/string-length": { + "version": "4.0.2", + "resolved": "https://registry.npmjs.org/string-length/-/string-length-4.0.2.tgz", + "integrity": "sha512-+l6rNN5fYHNhZZy41RXsYptCjA2Igmq4EG7kZAYFQI1E1VTXarr6ZPXBg6eq7Y6eK4FEhY6AJlyuFIb/v/S0VQ==", + "dev": true, "dependencies": { - "emoji-regex": "^8.0.0", - "is-fullwidth-code-point": "^3.0.0", - "strip-ansi": "^6.0.1" + "char-regex": "^1.0.2", + "strip-ansi": "^6.0.0" }, "engines": { - "node": ">=8" + "node": ">=10" } }, - "node_modules/string-width-cjs": { - "name": "string-width", + "node_modules/string-width": { "version": "4.2.3", "resolved": "https://registry.npmjs.org/string-width/-/string-width-4.2.3.tgz", "integrity": "sha512-wKyQRQpjJ0sIp62ErSZdGsjMJWsap5oRNihHhu6G7JVO/9jIB6UyevL+tXuOqrng8j/cxKTWyWUwvSTriiZz/g==", - "dev": true, "dependencies": { "emoji-regex": "^8.0.0", "is-fullwidth-code-point": "^3.0.0", @@ -2402,19 +4961,24 @@ "node": ">=8" } }, - "node_modules/strip-ansi-cjs": { - "name": "strip-ansi", - "version": "6.0.1", - "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-6.0.1.tgz", - "integrity": "sha512-Y38VPSHcqkFrCpFnQ9vuSXmquuv5oXOKpGeT6aGrr3o3Gc9AlVa6JBfUSOCnbxGGZF+/0ooI7KrPuUSztUdU5A==", + "node_modules/strip-bom": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/strip-bom/-/strip-bom-4.0.0.tgz", + "integrity": "sha512-3xurFv5tEgii33Zi8Jtp55wEIILR9eh34FAW00PZf+JnSsTmV/ioewSgQl97JHvgjoRGwPShsWm+IdrxB35d0w==", "dev": true, - "dependencies": { - "ansi-regex": "^5.0.1" - }, "engines": { "node": ">=8" } }, + "node_modules/strip-final-newline": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/strip-final-newline/-/strip-final-newline-2.0.0.tgz", + "integrity": "sha512-BrpvfNAE3dcvq7ll3xVumzjKjZQ5tI1sEUIKr3Uoks0XUl45St3FlatVqef9prk4jRDzhW6WZg+3bk93y6pLjA==", + "dev": true, + "engines": { + "node": ">=6" + } + }, "node_modules/strip-json-comments": { "version": "3.1.1", "resolved": "https://registry.npmjs.org/strip-json-comments/-/strip-json-comments-3.1.1.tgz", @@ -2428,18 +4992,27 @@ } }, "node_modules/supports-color": { - "version": "8.1.1", - "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-8.1.1.tgz", - "integrity": "sha512-MpUEN2OodtUzxvKQl72cUF7RQ5EiHsGvSsVG0ia9c5RbWGL2CI4C7EpPS8UTBIplnlzZiNuV56w+FuNxy3ty2Q==", + "version": "7.2.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-7.2.0.tgz", + "integrity": "sha512-qpCAvRl9stuOHveKsn7HncJRvv501qIacKzQlO/+Lwxc9+0q2wLyv4Dfvt80/DPn2pqOBsJdDiogXGR9+OvwRw==", "dev": true, "dependencies": { "has-flag": "^4.0.0" }, "engines": { - "node": ">=10" + "node": ">=8" + } + }, + "node_modules/supports-preserve-symlinks-flag": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/supports-preserve-symlinks-flag/-/supports-preserve-symlinks-flag-1.0.0.tgz", + "integrity": "sha512-ot0WnXS9fgdkgIcePe6RHNk1WA8+muPa6cSjeR3V8K27q9BB1rTE3R1p7Hv0z1ZyAc8s6Vvv8DIyWf681MAt0w==", + "dev": true, + "engines": { + "node": ">= 0.4" }, "funding": { - "url": "https://github.com/chalk/supports-color?sponsor=1" + "url": "https://github.com/sponsors/ljharb" } }, "node_modules/taffydb": { @@ -2464,28 +5037,48 @@ "node": ">=10" } }, - "node_modules/tar/node_modules/fs-minipass": { - "version": "2.1.0", - "resolved": "https://registry.npmjs.org/fs-minipass/-/fs-minipass-2.1.0.tgz", - "integrity": "sha512-V/JgOLFCS+R6Vcq0slCuaeWEdNC3ouDlJMNIsacH2VtALiu9mV4LPrHc5cDl8k5aw6J8jwgWWpiTo5RYhmIzvg==", - "dependencies": { - "minipass": "^3.0.0" - }, + "node_modules/tar/node_modules/minipass": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/minipass/-/minipass-5.0.0.tgz", + "integrity": "sha512-3FnjYuehv9k6ovOEbyOswadCDPX1piCfhV8ncmYtHOjuPwylVWsghTLo7rabjC3Rx5xD4HDx8Wm1xnMF7S5qFQ==", "engines": { - "node": ">= 8" + "node": ">=8" } }, - "node_modules/tar/node_modules/fs-minipass/node_modules/minipass": { - "version": "3.3.6", - "resolved": "https://registry.npmjs.org/minipass/-/minipass-3.3.6.tgz", - "integrity": "sha512-DxiNidxSEK+tHG6zOIklvNOwm3hvCrbUrdtzY74U6HKTJxvIDfOUL5W5P2Ghd3DTkhhKPYGqeNUIh5qcM4YBfw==", + "node_modules/tar/node_modules/yallist": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/yallist/-/yallist-4.0.0.tgz", + "integrity": "sha512-3wdGidZyq5PB084XLES5TpOSRA3wjXAlIWMhum2kRcv/41Sn2emQ0dycQW4uZXLejwKvg6EsvbdlVL+FYEct7A==" + }, + "node_modules/test-exclude": { + "version": "6.0.0", + "resolved": "https://registry.npmjs.org/test-exclude/-/test-exclude-6.0.0.tgz", + "integrity": "sha512-cAGWPIyOHU6zlmg88jwm7VRyXnMN7iV68OGAbYDk/Mh/xC/pzVPlQtY6ngoIH/5/tciuhGfvESU8GrHrcxD56w==", + "dev": true, "dependencies": { - "yallist": "^4.0.0" + "@istanbuljs/schema": "^0.1.2", + "glob": "^7.1.4", + "minimatch": "^3.0.4" }, "engines": { "node": ">=8" } }, + "node_modules/tmpl": { + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/tmpl/-/tmpl-1.0.5.tgz", + "integrity": "sha512-3f0uOEAQwIqGuWW2MVzYg8fV/QNnc/IpuJNG837rLuczAaLVHslWHZQj4IGiEl5Hs3kkbhwL9Ab7Hrsmuj+Smw==", + "dev": true + }, + "node_modules/to-fast-properties": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/to-fast-properties/-/to-fast-properties-2.0.0.tgz", + "integrity": "sha512-/OaKK0xYrs3DmxRYqL/yDc+FxFUVYhDlXMhRmv3z915w2HF1tnN1omB354j8VUGO/hbRzyD6Y3sA7v7GS/ceog==", + "dev": true, + "engines": { + "node": ">=4" + } + }, "node_modules/to-regex-range": { "version": "5.0.1", "resolved": "https://registry.npmjs.org/to-regex-range/-/to-regex-range-5.0.1.tgz", @@ -2509,10 +5102,31 @@ "resolved": "https://registry.npmjs.org/tr46/-/tr46-0.0.3.tgz", "integrity": "sha512-N3WMsuqV66lT30CrXNbEjx4GEwlow3v6rr4mCcv6prnfwhS01rkgyFdjPNBYd9br7LpXV1+Emh01fHnq2Gdgrw==" }, + "node_modules/type-detect": { + "version": "4.0.8", + "resolved": "https://registry.npmjs.org/type-detect/-/type-detect-4.0.8.tgz", + "integrity": "sha512-0fr/mIH1dlO+x7TlcMy+bIDqKPsw/70tVyeHW787goQjhmqaZe10uwLujubK9q9Lg6Fiho1KUKDYz0Z7k7g5/g==", + "dev": true, + "engines": { + "node": ">=4" + } + }, + "node_modules/type-fest": { + "version": "0.21.3", + "resolved": "https://registry.npmjs.org/type-fest/-/type-fest-0.21.3.tgz", + "integrity": "sha512-t0rzBq87m3fVcduHDUFhKmyyX+9eo6WQjZvf51Ea/M0Q7+T374Jp1aUiyUl0GKxp8M/OETVHSDvmkyPgvX+X2w==", + "dev": true, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, "node_modules/typescript": { - "version": "5.1.6", - "resolved": "https://registry.npmjs.org/typescript/-/typescript-5.1.6.tgz", - "integrity": "sha512-zaWCozRZ6DLEWAWFrVDz1H6FVXzUSfTy5FUMWsQlU8Ym5JP9eO4xkTIROFCQvhQf61z6O/G6ugw3SgAnvvm+HA==", + "version": "5.3.3", + "resolved": "https://registry.npmjs.org/typescript/-/typescript-5.3.3.tgz", + "integrity": "sha512-pXWcraxM0uxAS+tN0AG/BF2TyqmHO014Z070UsJ+pFvYuRSq8KH8DmWpnbXe0pEPDHXZV3FcAbJkijJ5oNEnWw==", "dev": true, "bin": { "tsc": "bin/tsc", @@ -2541,27 +5155,57 @@ "dev": true }, "node_modules/unique-filename": { - "version": "3.0.0", - "resolved": "https://registry.npmjs.org/unique-filename/-/unique-filename-3.0.0.tgz", - "integrity": "sha512-afXhuC55wkAmZ0P18QsVE6kp8JaxrEokN2HGIoIVv2ijHQd419H0+6EigAFcIzXeMIkcIkNBpB3L/DXB3cTS/g==", + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/unique-filename/-/unique-filename-2.0.1.tgz", + "integrity": "sha512-ODWHtkkdx3IAR+veKxFV+VBkUMcN+FaqzUUd7IZzt+0zhDZFPFxhlqwPF3YQvMHx1TD0tdgYl+kuPnJ8E6ql7A==", "dev": true, "dependencies": { - "unique-slug": "^4.0.0" + "unique-slug": "^3.0.0" }, "engines": { - "node": "^14.17.0 || ^16.13.0 || >=18.0.0" + "node": "^12.13.0 || ^14.15.0 || >=16.0.0" } }, "node_modules/unique-slug": { - "version": "4.0.0", - "resolved": "https://registry.npmjs.org/unique-slug/-/unique-slug-4.0.0.tgz", - "integrity": "sha512-WrcA6AyEfqDX5bWige/4NQfPZMtASNVxdmWR76WESYQVAACSgWcR6e9i0mofqqBxYFtL4oAxPIptY73/0YE1DQ==", + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/unique-slug/-/unique-slug-3.0.0.tgz", + "integrity": "sha512-8EyMynh679x/0gqE9fT9oilG+qEt+ibFyqjuVTsZn1+CMxH+XLlpvr2UZx4nVcCwTpx81nICr2JQFkM+HPLq4w==", "dev": true, "dependencies": { "imurmurhash": "^0.1.4" }, "engines": { - "node": "^14.17.0 || ^16.13.0 || >=18.0.0" + "node": "^12.13.0 || ^14.15.0 || >=16.0.0" + } + }, + "node_modules/update-browserslist-db": { + "version": "1.0.13", + "resolved": "https://registry.npmjs.org/update-browserslist-db/-/update-browserslist-db-1.0.13.tgz", + "integrity": "sha512-xebP81SNcPuNpPP3uzeW1NYXxI3rxyJzF3pD6sH4jE7o/IX+WtSpwnVU+qIsDPyk0d3hmFQ7mjqc6AtV604hbg==", + "dev": true, + "funding": [ + { + "type": "opencollective", + "url": "https://opencollective.com/browserslist" + }, + { + "type": "tidelift", + "url": "https://tidelift.com/funding/github/npm/browserslist" + }, + { + "type": "github", + "url": "https://github.com/sponsors/ai" + } + ], + "dependencies": { + "escalade": "^3.1.1", + "picocolors": "^1.0.0" + }, + "bin": { + "update-browserslist-db": "cli.js" + }, + "peerDependencies": { + "browserslist": ">= 4.21.0" } }, "node_modules/util-deprecate": { @@ -2569,6 +5213,29 @@ "resolved": "https://registry.npmjs.org/util-deprecate/-/util-deprecate-1.0.2.tgz", "integrity": "sha512-EPD5q1uXyFxJpCrLnCc1nHnq3gOa6DZBocAIiI2TaSCA7VCJ1UJDMagCzIkXNsUYfD1daK//LTEQ8xiIbrHtcw==" }, + "node_modules/v8-to-istanbul": { + "version": "9.2.0", + "resolved": "https://registry.npmjs.org/v8-to-istanbul/-/v8-to-istanbul-9.2.0.tgz", + "integrity": "sha512-/EH/sDgxU2eGxajKdwLCDmQ4FWq+kpi3uCmBGpw1xJtnAxEjlD8j8PEiGWpCIMIs3ciNAgH0d3TTJiUkYzyZjA==", + "dev": true, + "dependencies": { + "@jridgewell/trace-mapping": "^0.3.12", + "@types/istanbul-lib-coverage": "^2.0.1", + "convert-source-map": "^2.0.0" + }, + "engines": { + "node": ">=10.12.0" + } + }, + "node_modules/walker": { + "version": "1.0.8", + "resolved": "https://registry.npmjs.org/walker/-/walker-1.0.8.tgz", + "integrity": "sha512-ts/8E8l5b7kY0vlWLewOkDXMmPdLcVV4GmOQLyxuSswIJsweeFZtAsMF7k1Nszz+TYBQrlYRmzOnr398y1JemQ==", + "dev": true, + "dependencies": { + "makeerror": "1.0.12" + } + }, "node_modules/webidl-conversions": { "version": "3.0.1", "resolved": "https://registry.npmjs.org/webidl-conversions/-/webidl-conversions-3.0.1.tgz", @@ -2629,29 +5296,24 @@ "url": "https://github.com/chalk/wrap-ansi?sponsor=1" } }, - "node_modules/wrap-ansi-cjs": { - "name": "wrap-ansi", - "version": "7.0.0", - "resolved": "https://registry.npmjs.org/wrap-ansi/-/wrap-ansi-7.0.0.tgz", - "integrity": "sha512-YVGIj2kamLSTxw6NsZjoBxfSwsn0ycdesmc4p+Q21c5zPuZ1pl+NfxVdxPtdHvmNVOQ6XSYG4AUtyt/Fi7D16Q==", + "node_modules/wrappy": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/wrappy/-/wrappy-1.0.2.tgz", + "integrity": "sha512-l4Sp/DRseor9wL6EvV2+TuQn63dMkPjZ/sp9XkghTEbV9KlPS1xUsZ3u7/IQO4wxtcFB4bgpQPRcR3QCvezPcQ==" + }, + "node_modules/write-file-atomic": { + "version": "4.0.2", + "resolved": "https://registry.npmjs.org/write-file-atomic/-/write-file-atomic-4.0.2.tgz", + "integrity": "sha512-7KxauUdBmSdWnmpaGFg+ppNjKF8uNLry8LyzjauQDOVONfFLNKrKvQOxZ/VuTIcS/gge/YNahf5RIIQWTSarlg==", "dev": true, "dependencies": { - "ansi-styles": "^4.0.0", - "string-width": "^4.1.0", - "strip-ansi": "^6.0.0" + "imurmurhash": "^0.1.4", + "signal-exit": "^3.0.7" }, "engines": { - "node": ">=10" - }, - "funding": { - "url": "https://github.com/chalk/wrap-ansi?sponsor=1" + "node": "^12.13.0 || ^14.15.0 || >=16.0.0" } }, - "node_modules/wrappy": { - "version": "1.0.2", - "resolved": "https://registry.npmjs.org/wrappy/-/wrappy-1.0.2.tgz", - "integrity": "sha512-l4Sp/DRseor9wL6EvV2+TuQn63dMkPjZ/sp9XkghTEbV9KlPS1xUsZ3u7/IQO4wxtcFB4bgpQPRcR3QCvezPcQ==" - }, "node_modules/xmlcreate": { "version": "2.0.4", "resolved": "https://registry.npmjs.org/xmlcreate/-/xmlcreate-2.0.4.tgz", @@ -2668,26 +5330,27 @@ } }, "node_modules/yallist": { - "version": "4.0.0", - "resolved": "https://registry.npmjs.org/yallist/-/yallist-4.0.0.tgz", - "integrity": "sha512-3wdGidZyq5PB084XLES5TpOSRA3wjXAlIWMhum2kRcv/41Sn2emQ0dycQW4uZXLejwKvg6EsvbdlVL+FYEct7A==" + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/yallist/-/yallist-3.1.1.tgz", + "integrity": "sha512-a4UGQaWPH59mOXUYnAG2ewncQS4i4F43Tv3JoAM+s2VDAmS9NsK8GpDMLrCHPksFT7h3K6TOoUNn2pb7RoXx4g==", + "dev": true }, "node_modules/yargs": { - "version": "16.2.0", - "resolved": "https://registry.npmjs.org/yargs/-/yargs-16.2.0.tgz", - "integrity": "sha512-D1mvvtDG0L5ft/jGWkLpG1+m0eQxOfaBvTNELraWj22wSVUMWxZUvYgJYcKh6jGGIkJFhH4IZPQhR4TKpc8mBw==", + "version": "17.7.2", + "resolved": "https://registry.npmjs.org/yargs/-/yargs-17.7.2.tgz", + "integrity": "sha512-7dSzzRQ++CKnNI/krKnYRV7JKKPUXMEh61soaHKg9mrWEhzFWhFnxPxGl+69cD1Ou63C13NUPCnmIcrvqCuM6w==", "dev": true, "dependencies": { - "cliui": "^7.0.2", + "cliui": "^8.0.1", "escalade": "^3.1.1", "get-caller-file": "^2.0.5", "require-directory": "^2.1.1", - "string-width": "^4.2.0", + "string-width": "^4.2.3", "y18n": "^5.0.5", - "yargs-parser": "^20.2.2" + "yargs-parser": "^21.1.1" }, "engines": { - "node": ">=10" + "node": ">=12" } }, "node_modules/yargs-parser": { @@ -2714,6 +5377,27 @@ "node": ">=10" } }, + "node_modules/yargs-unparser/node_modules/camelcase": { + "version": "6.3.0", + "resolved": "https://registry.npmjs.org/camelcase/-/camelcase-6.3.0.tgz", + "integrity": "sha512-Gmy6FhYlCY7uOElZUSbxo2UCDH8owEk996gkbrpsgGtrJLM3J7jGxl9Ic7Qwwj4ivOE5AWZWRMecDdF7hqGjFA==", + "dev": true, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/yargs/node_modules/yargs-parser": { + "version": "21.1.1", + "resolved": "https://registry.npmjs.org/yargs-parser/-/yargs-parser-21.1.1.tgz", + "integrity": "sha512-tVpsJW7DdjecAiFpbIB1e3qxIQsE6NoPc5/eTdrbbIC4h0LVsWhnoa3g+m2HclBIujHzsxZ4VJVA+GUuc2/LBw==", + "dev": true, + "engines": { + "node": ">=12" + } + }, "node_modules/yocto-queue": { "version": "0.1.0", "resolved": "https://registry.npmjs.org/yocto-queue/-/yocto-queue-0.1.0.tgz", diff --git a/package.json b/package.json index 5e46e181..45d1ec63 100644 --- a/package.json +++ b/package.json @@ -42,6 +42,7 @@ "devDependencies": { "@types/node": "^20.4.5", "bluebird": "^3.5.3", + "jest": "^29.7.0", "jsdoc": "^3.4.0", "jshint": "^2.10.1", "mocha": "^10.2.0", diff --git a/test/promisified/consumer/subscribe.spec.js b/test/promisified/consumer/subscribe.spec.js new file mode 100644 index 00000000..bbbcd483 --- /dev/null +++ b/test/promisified/consumer/subscribe.spec.js @@ -0,0 +1,157 @@ +const { Kafka, ErrorCodes } = require('../../../lib').KafkaJS; +const { secureRandom, + createTopic, + waitForMessages, + waitForConsumerToJoinGroup, + createProducer, + createConsumer } = require('../testhelpers'); + +describe('Consumer', () => { + let groupId, consumer, producer; + + beforeEach(async () => { + groupId = `consumer-group-id-${secureRandom()}`; + + consumer = createConsumer({ + groupId, + maxWaitTimeInMs: 1, + maxBytesPerPartition: 180, + rdKafka: { + topicConfig: { + 'auto.offset.reset': 'earliest', + } + } + }); + + producer = createProducer({}); + }) + + afterEach(async () => { + consumer && (await consumer.disconnect()); + producer && (await producer.disconnect()); + }) + + describe('when subscribing to multiple topics', () => { + it('throws an error if one of the topics is invalid', async () => { + await consumer.connect(); + await expect(consumer.subscribe({ topics: [1] })).rejects.toHaveProperty( + 'code', + ErrorCodes.ERR__INVALID_ARG, + ); + }); + + it('subscribes by topic name as a string or regex', async () => { + const testScope = secureRandom(); + const regexMatchingTopic = `pattern-${testScope}-regex-${secureRandom()}`; + const topics = [`topic-${secureRandom()}`, `topic-${secureRandom()}`, regexMatchingTopic]; + + await Promise.all(topics.map(topic => createTopic({ topic }))); + + const messagesConsumed = []; + await consumer.connect(); + await consumer.subscribe({ + topics: [topics[0], topics[1], new RegExp(`^pattern-${testScope}-regex-.*`, 'i')], + }); + + consumer.run({ eachMessage: async event => messagesConsumed.push(event) }); + await waitForConsumerToJoinGroup(consumer); + + await producer.connect(); + let records = await producer.sendBatch({ + topicMessages: [ + { topic: topics[0], messages: [{ key: 'drink', value: 'drink' }] }, + { topic: topics[1], messages: [{ key: 'your', value: 'your' }] }, + { topic: topics[2], messages: [{ key: 'ovaltine', value: 'ovaltine' }] }, + ], + }); + + await waitForMessages(messagesConsumed, { number: 3 }); + expect(messagesConsumed.map(m => m.message.value.toString())).toEqual( + expect.arrayContaining(['drink', 'your', 'ovaltine']) + ); + }) + }) + + describe('Deprecated "topic" interface', () => { + describe('when subscribing', () => { + it('throws an error if the topic is invalid', async () => { + await consumer.connect(); + await expect(consumer.subscribe({ topic: null })).rejects.toHaveProperty( + 'code', + ErrorCodes.ERR__INVALID_ARG + ); + }); + + it('throws an error if the topic is not a String or RegExp', async () => { + await consumer.connect(); + await expect(consumer.subscribe({ topic: 1 })).rejects.toHaveProperty( + 'code', + ErrorCodes.ERR__INVALID_ARG + ); + }); + + describe('with a string', () => { + it('subscribes to the topic', async () => { + const topic = `topic-${secureRandom()}` + + await createTopic({ topic }); + + const messagesConsumed = []; + await consumer.connect(); + await consumer.subscribe({ topic }); + + consumer.run({ eachMessage: async event => messagesConsumed.push(event) }); + await waitForConsumerToJoinGroup(consumer); + + await producer.connect(); + await producer.sendBatch({ + topicMessages: [{ topic, messages: [{ key: 'key-a', value: 'value-a' }] }], + }); + + await waitForMessages(messagesConsumed, { number: 1 }); + expect(messagesConsumed.map(m => m.message.value.toString()).sort()).toEqual(['value-a']); + }); + }); + + describe('with regex', () => { + it('subscribes to all matching topics', async () => { + const testScope = secureRandom(); + const topicUS = `pattern-${testScope}-us-${secureRandom()}`; + const topicSE = `pattern-${testScope}-se-${secureRandom()}`; + const topicUK = `pattern-${testScope}-uk-${secureRandom()}`; + const topicBR = `pattern-${testScope}-br-${secureRandom()}`; + + await Promise.all( + [topicUS, topicSE, topicUK, topicBR].map(topic => createTopic({ topic })) + ); + + const messagesConsumed = []; + await consumer.connect(); + await consumer.subscribe({ + topic: new RegExp(`pattern-${testScope}-(se|br)-.*`, 'i'), + }); + + consumer.run({ eachMessage: async event => messagesConsumed.push(event) }); + await waitForConsumerToJoinGroup(consumer); + + await producer.connect(); + await producer.sendBatch({ + topicMessages: [ + { topic: topicUS, messages: [{ key: `key-us`, value: `value-us` }] }, + { topic: topicUK, messages: [{ key: `key-uk`, value: `value-uk` }] }, + { topic: topicSE, messages: [{ key: `key-se`, value: `value-se` }] }, + { topic: topicBR, messages: [{ key: `key-br`, value: `value-br` }] }, + ], + }); + + await waitForMessages(messagesConsumed, { number: 2 }); + expect(messagesConsumed.map(m => m.message.value.toString()).sort()).toEqual([ + 'value-br', + 'value-se', + ]); + }); + }); + }); + }); +}); + diff --git a/test/promisified/testhelpers.js b/test/promisified/testhelpers.js new file mode 100644 index 00000000..e3acdc45 --- /dev/null +++ b/test/promisified/testhelpers.js @@ -0,0 +1,61 @@ +const crypto = require('crypto') +const { Kafka, ErrorCodes } = require('../../lib').KafkaJS; + +// TODO: pick this up from a file +const clusterInformation = { + brokers: ['localhost:9092'], +}; + +function createConsumer(config) { + const kafka = new Kafka(Object.assign(config, clusterInformation)); + return kafka.consumer(); +} + +function createProducer(config) { + const kafka = new Kafka(Object.assign(config, clusterInformation)); + return kafka.producer(); +} + +function createAdmin(config) { + const kafka = new Kafka(Object.assign(config, clusterInformation)); + return kafka.admin(); +} + +function secureRandom(length = 10) { + return `${crypto.randomBytes(length).toString('hex')}-${process.pid}-${crypto.randomUUID()}`; +} + +async function createTopic(args) { + const { topic } = args; + const admin = createAdmin({}); + await admin.connect(); + await admin.createTopics({ topics: [{topic}] }); + await admin.disconnect(); +} + +async function waitForConsumerToJoinGroup(consumer) { + // We don't yet have a deterministic way to test this, so we just wait for a bit. + return new Promise(resolve => setTimeout(resolve, 2500)); +} + +async function waitForMessages(messagesConsumed, { number } = {number: 0}) { + return new Promise(resolve => { + const interval = setInterval(() => { + if (messagesConsumed.length >= number) { + clearInterval(interval); + resolve(); + } + }, 200); + }); +} + +module.exports = { + createConsumer, + createProducer, + createAdmin, + secureRandom, + waitForMessages, + waitForMessages, + createTopic, + waitForConsumerToJoinGroup, +} diff --git a/types/kafkajs.d.ts b/types/kafkajs.d.ts index 906ef267..59aa5efc 100644 --- a/types/kafkajs.d.ts +++ b/types/kafkajs.d.ts @@ -77,6 +77,18 @@ export interface ProducerRecord { compression?: CompressionTypes } +export interface TopicMessages { + topic: string + messages: Message[] +} + +export interface ProducerBatch { + acks?: number + timeout?: number + compression?: CompressionTypes + topicMessages?: TopicMessages[] +} + export type RecordMetadata = { topicName: string partition: number @@ -364,6 +376,8 @@ export type EachBatchHandler = (payload: EachBatchPayload) => Promise export type EachMessageHandler = (payload: EachMessagePayload) => Promise +export type ConsumerSubscribeTopic = { topic: string | RegExp; fromBeginning?: boolean } + export type ConsumerSubscribeTopics = { topics: (string | RegExp)[]; fromBeginning?: boolean } export type ConsumerRunConfig = { From 85b1341377e2245965f97848ca842bcb8758cec6 Mon Sep 17 00:00:00 2001 From: Milind L Date: Mon, 11 Dec 2023 18:26:46 +0530 Subject: [PATCH 10/26] Add tests for consumer.seek() --- MIGRATION.md | 3 + lib/kafkajs/_consumer.js | 173 ++++++++++-- test/promisified/consumer/seek.spec.js | 359 +++++++++++++++++++++++++ test/promisified/testhelpers.js | 8 +- 4 files changed, 522 insertions(+), 21 deletions(-) create mode 100644 test/promisified/consumer/seek.spec.js diff --git a/MIGRATION.md b/MIGRATION.md index e3e70c79..f4ab3003 100644 --- a/MIGRATION.md +++ b/MIGRATION.md @@ -291,6 +291,9 @@ * `commitOffsets` does not (YET) support sending metadata for topic partitions being committed. * `paused()` is not (YET) supported. * Custom partition assignors are not supported. + * Changes to `seek`: + * The restriction to call seek only after `run` is removed. + * Rather than the `autoCommit` property of `run` deciding if the offset is committed, the librdkafka property `enable.auto.commit` of the consumer config is used. ### Admin Client diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 410a8e4f..889fc6ae 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -54,6 +54,19 @@ class Consumer { */ #state = ConsumerState.INIT; + /** + * Denotes if there are any new pending seeks we need to check. + * @type {boolean} + */ + #checkPendingSeeks = false; + + /** + * Contains a mapping of topic+partition to an offset that the user wants to seek to. + * The keys are of the type "|". + * @type {Map} + */ + #pendingSeeks = new Map(); + /** * @constructor * @param {import("../../types/kafkajs").ConsumerConfig} kJSConfig @@ -99,13 +112,32 @@ class Consumer { } call - .finally(() => { + .finally(async () => { // Emit the event this.#internalClient.emit('rebalance', err, assignment); try { if (err.code === LibrdKafkaError.codes.ERR__ASSIGN_PARTITIONS) { + + if (this.#checkPendingSeeks) + assignment = this.#assignAsPerSeekedOffsets(assignment); + this.#internalClient.assign(assignment); + + if (this.#checkPendingSeeks) { + const offsetsToCommit = assignment + .filter((topicPartition) => topicPartition.offset !== undefined) + .map((topicPartition) => ({ + topic: topicPartition.topic, + partition: topicPartition.partition, + offset: String(topicPartition.offset), + })); + + if (offsetsToCommit.length !== 0 && this.#rdKafkaConfig.globalConfig['enable.auto.commit']) { + await this.#commitOffsetsUntilNoStateErr(offsetsToCommit); + } + } + } else { this.#internalClient.unassign(); } @@ -191,9 +223,13 @@ class Consumer { } globalConfig['offset_commit_cb'] = true; - if (this.#kJSConfig.rebalanceListener) { - globalConfig['rebalance_cb'] = this.#rebalanceCallback.bind(this); + + if (!Object.hasOwn(this.#kJSConfig, 'rebalanceListener')) { + /* We might want to do certain things to maintain internal state in rebalance listener, so we need to set it to an empty object. */ + this.#kJSConfig.rebalanceListener = {}; } + globalConfig['rebalance_cb'] = this.#rebalanceCallback.bind(this); + return { globalConfig, topicConfig }; } @@ -417,6 +453,14 @@ class Consumer { * array/list until it can be processed, because librdkafka marks it as * 'stored'... but anyway - we can implement something like this. */ + if (this.#checkPendingSeeks) { + const invalidateMessage = await this.#seekInternal({ topic: m.topic, partition: m.partition }); + if (invalidateMessage) { + /* Don't pass this message on to the user if this topic partition was seeked to. */ + continue; + } + } + await config.eachMessage( this.#createPayload(m) ) @@ -429,6 +473,17 @@ class Consumer { } } + async #commitOffsetsUntilNoStateErr(offsetsToCommit) { + let err = { code: error.ErrorCodes.ERR_NO_ERROR }; + do { + try { + await this.commitOffsets(offsetsToCommit); + } catch (e) { + err = e; + } + } while (err.code && err.code === error.ErrorCodes.ERR__STATE); + } + /** * Commit offsets for the given topic partitions. If topic partitions are not specified, commits all offsets. * @param {import("../../types/kafkajs").TopicPartitionOffsetAndMetadata[]?} topicPartitions @@ -443,9 +498,9 @@ class Consumer { if (topicPartitions === null) { this.#internalClient.commitSync(); } else { - const topicPartitions = topicPartitions.map( + const topicPartitionsRdKafka = topicPartitions.map( topicPartitionOffsetToRdKafka); - this.#internalClient.commitSync(topicPartitions); + this.#internalClient.commitSync(topicPartitionsRdKafka); } } catch (e) { if (!e.code || e.code !== error.ErrorCodes.ERR__NO_OFFSET) { @@ -454,27 +509,109 @@ class Consumer { } } + /** + * Apply pending seeks to topic partitions we have just obtained as a result of a rebalance. + * @param {{topic: string, partition: number}[]} assignment The list of topic partitions to check for pending seeks. + * @returns {{topic: string, partition: number, offset: number}[]} the new assignment with the offsets seeked to, which can be passed to assign(). + */ + #assignAsPerSeekedOffsets(assignment) { + const offsetsToCommit = []; + + for (let i = 0; i < assignment.length; i++) { + const topicPartition = assignment[i]; + const key = `${topicPartition.topic}|${topicPartition.partition}`; + if (!this.#pendingSeeks.has(key)) + continue; + + const offset = this.#pendingSeeks.get(key); + this.#pendingSeeks.delete(key); + + assignment[i].offset = offset; + + offsetsToCommit.push({ + topic: topicPartition.topic, + partition: topicPartition.partition, + offset: String(offset), + }); + } + return assignment; + } + + /** + * This method processes any pending seeks on partitions that are assigned to this consumer. + * @param {{topic: string, partition: number}} messageTopicPartition If this method was triggered by a message, pass the topic partition of the message, else it's optional. + * @returns whether the message that triggered this should be invalidated (if any). + */ + async #seekInternal(messageTopicPartition) { + this.#checkPendingSeeks = false; + + const assignment = this.assignment(); + const offsetsToCommit = []; + let invalidateMessage = false; + + for (const topicPartition of assignment) { + const key = `${topicPartition.topic}|${topicPartition.partition}`; + if (!this.#pendingSeeks.has(key)) + continue; + + const offset = this.#pendingSeeks.get(key); + this.#pendingSeeks.delete(key); + + const topicPartitionOffset = { + topic: topicPartition.topic, + partition: topicPartition.partition, + offset + }; + + /* It's assumed that topicPartition is already assigned, and thus can be seeked to and committed to. + * Errors are logged to detect bugs in the internal code. */ + this.#internalClient.seek(topicPartitionOffset, 0, err => err ? console.error(err) : null); + offsetsToCommit.push({ + topic: topicPartition.topic, + partition: topicPartition.partition, + offset: String(offset), + }); + + /* If we're seeking the same topic partition as in the message that triggers it, invalidate + * the message. */ + if (messageTopicPartition && topicPartition.topic === messageTopicPartition.topic && topicPartition.partition === messageTopicPartition.partition) { + invalidateMessage = true; + } + } + + if (offsetsToCommit.length !== 0 && this.#rdKafkaConfig.globalConfig['enable.auto.commit']) { + await this.#commitOffsetsUntilNoStateErr(offsetsToCommit); + } + + return invalidateMessage; + } + /** * Seek to the given offset for the topic partition. + * This method is completely asynchronous, and does not wait for the seek to complete. + * In case any partitions that are seeked to, are not a part of the current assignment, they are stored internally. + * If at any time, the consumer is assigned the partition, the seek will be performed. + * Depending on the value of the librdkafka property 'enable.auto.commit', the consumer will commit the offset seeked to. * @param {import("../../types/kafkajs").TopicPartitionOffset} topicPartitionOffset - * @returns {Promise} a promise that resolves when the consumer has seeked. */ seek(topicPartitionOffset) { if (this.#state !== ConsumerState.CONNECTED) { throw new error.KafkaJSError('Seek can only be called while connected.', { code: error.ErrorCodes.ERR__STATE }); } - return new Promise((resolve, reject) => { - const rdKafkaTopicPartitionOffset = - topicPartitionOffsetToRdKafka(topicPartitionOffset); - this.#internalClient.seek(rdKafkaTopicPartitionOffset, 0, (err) => { - if (err) { - reject(createKafkaJsErrorFromLibRdKafkaError(err)); - } else { - resolve(); - } - }); - }); + const rdKafkaTopicPartitionOffset = + topicPartitionOffsetToRdKafka(topicPartitionOffset); + + if (typeof rdKafkaTopicPartitionOffset.topic !== 'string') { + throw new error.KafkaJSError('Topic must be a string.', { code: error.ErrorCodes.ERR__INVALID_ARG }); + } + + if (isNaN(rdKafkaTopicPartitionOffset.offset) || (rdKafkaTopicPartitionOffset.offset < 0 && rdKafkaTopicPartitionOffset.offset !== -2 && rdKafkaTopicPartitionOffset.offset !== -3)) { + throw new error.KafkaJSError('Offset must be >= 0, or a special value.', { code: error.ErrorCodes.ERR__INVALID_ARG }); + } + + this.#checkPendingSeeks = true; + this.#pendingSeeks.set(`${rdKafkaTopicPartitionOffset.topic}|${rdKafkaTopicPartitionOffset.partition}`, rdKafkaTopicPartitionOffset.offset); } async describeGroup() { @@ -483,7 +620,7 @@ class Consumer { /** * Find the assigned topic partitions for the consumer. - * @returns {import("../../types").TopicPartition[]} the current assignment. + * @returns {import("../../types/kafkajs").TopicPartition[]} the current assignment. */ assignment() { if (this.#state !== ConsumerState.CONNECTED) { diff --git a/test/promisified/consumer/seek.spec.js b/test/promisified/consumer/seek.spec.js new file mode 100644 index 00000000..50001c6e --- /dev/null +++ b/test/promisified/consumer/seek.spec.js @@ -0,0 +1,359 @@ +const { + createConsumer, + createProducer, + secureRandom, + createTopic, + waitForMessages, +} = require('../testhelpers') + +describe('Consumer', () => { + let topicName, groupId, producer, consumer; + + beforeEach(async () => { + topicName = `test-topic-${secureRandom()}`; + groupId = `consumer-group-id-${secureRandom()}`; + + producer = createProducer({}); + + consumer = createConsumer({ + groupId, + rdKafka: { + topicConfig: { + 'auto.offset.reset': 'earliest', + }, + } + }); + }); + + afterEach(async () => { + consumer && (await consumer.disconnect()); + producer && (await producer.disconnect()); + }); + + describe('when seek offset', () => { + describe('with one partition', () => { + beforeEach(async () => { + await createTopic({ topic: topicName, partitions: 1 }) + }); + + it('throws an error if the topic is invalid', async () => { + await consumer.connect(); + expect(() => consumer.seek({ topic: null })).toThrow('must be a string'); + }); + + it('throws an error if the partition is not a number', async () => { + await consumer.connect(); + expect(() => consumer.seek({ topic: topicName, partition: 'ABC' })).toThrow('Offset must be'); + }); + + it('throws an error if the offset is not a number', async () => { + await consumer.connect(); + expect(() => consumer.seek({ topic: topicName, partition: 0, offset: 'ABC' })).toThrow('Offset must be'); + }); + + it('throws an error if the offset is negative and not a special offset', async () => { + await consumer.connect(); + expect(() => consumer.seek({ topic: topicName, partition: 0, offset: '-32' })).toThrow('Offset must be'); + }); + + it('recovers from offset out of range', async () => { + await consumer.connect(); + await producer.connect(); + + const key1 = secureRandom(); + const message1 = { key: `key-${key1}`, value: `value-${key1}` }; + + await producer.send({ topic: topicName, messages: [message1] }); + await consumer.subscribe({ topic: topicName, }); + + const messagesConsumed = []; + consumer.seek({ topic: topicName, partition: 0, offset: 100 }); + consumer.run({ + eachMessage: async event => { + messagesConsumed.push(event); + } + }); + + await expect(waitForMessages(messagesConsumed, { number: 1 })).resolves.toEqual([ + expect.objectContaining({ + topic: topicName, + partition: 0, + message: expect.objectContaining({ offset: '0' }), + }), + ]); + }, 10000); + + + describe('When "enable.auto.commit" is false', () => { + beforeEach(() => { + consumer = createConsumer({ + groupId, + rdKafka: { + globalConfig: { + 'enable.auto.commit': false, + }, + topicConfig: { + 'auto.offset.reset': 'earliest', + }, + } + }); + }); + + it('should not commit the offset', async () => { + await producer.connect(); + await consumer.connect(); + + await producer.send({ + topic: topicName, + messages: [1, 2, 3].map(n => ({ key: `key-${n}`, value: `value-${n}`, partition: 0 })), + }); + await consumer.subscribe({ topic: topicName }); + + let messagesConsumed = [] + consumer.seek({ topic: topicName, partition: 0, offset: 2 }) + consumer.run({ + eachMessage: async event => messagesConsumed.push(event), + }); + + await expect(waitForMessages(messagesConsumed, { number: 1 })).resolves.toEqual([ + expect.objectContaining({ + topic: topicName, + partition: 0, + message: expect.objectContaining({ offset: '2' }), + }), + ]); + + /* We disconnect this consumer, and create another one of the same consumer group. + * This new consumer should start from 0, despite the fact that we've sought to 2 */ + await consumer.disconnect(); + + consumer = createConsumer({ + groupId, + rdKafka: { + globalConfig: { + 'enable.auto.commit': false, + }, + topicConfig: { + 'auto.offset.reset': 'earliest', + }, + } + }); + await consumer.connect(); + await consumer.subscribe({ topic: topicName }); + + messagesConsumed = []; + consumer.run({ + eachMessage: async event => messagesConsumed.push(event), + }); + + await expect(waitForMessages(messagesConsumed, { number: 3 })).resolves.toEqual([ + expect.objectContaining({ + topic: topicName, + partition: 0, + message: expect.objectContaining({ offset: '0' }), + }), + expect.objectContaining({ + topic: topicName, + partition: 0, + message: expect.objectContaining({ offset: '1' }), + }), + expect.objectContaining({ + topic: topicName, + partition: 0, + message: expect.objectContaining({ offset: '2' }), + }), + ]); + }, 10000); + }); + }); + + describe('with two partitions', () => { + beforeEach(async () => { + await createTopic({ topic: topicName, partitions: 2 }) + }); + + it('updates the partition offset to the given offset', async () => { + await consumer.connect(); + await producer.connect(); + + const value1 = secureRandom(); + const message1 = { key: `key-1`, value: `value-${value1}`, partition: 1, }; + const value2 = secureRandom(); + const message2 = { key: `key-1`, value: `value-${value2}`, partition: 1, }; + const value3 = secureRandom(); + const message3 = { key: `key-1`, value: `value-${value3}`, partition: 1, }; + const value4 = secureRandom(); + const message4 = { key: `key-0`, value: `value-${value4}`, partition: 0, }; + + await producer.send({ + topic: topicName, + messages: [message1, message2, message3, message4], + }); + + await consumer.subscribe({ topic: topicName }); + + const messagesConsumed = [] + consumer.seek({ topic: topicName, partition: 1, offset: 1 }); + consumer.run({ + eachMessage: async event => { + messagesConsumed.push(event); + } + }); + + let check = await expect(waitForMessages(messagesConsumed, { number: 3 })).resolves; + + await check.toEqual( + expect.arrayContaining([ + expect.objectContaining({ + topic: topicName, + partition: 0, + message: expect.objectContaining({ offset: '0' }), + }), + expect.objectContaining({ + topic: topicName, + partition: 1, + message: expect.objectContaining({ offset: '1' }), + }), + expect.objectContaining({ + topic: topicName, + partition: 1, + message: expect.objectContaining({ offset: '2' }), + }), + ]) + ); + + await check.toEqual( + expect.not.arrayContaining([ + expect.objectContaining({ + topic: topicName, + partition: 1, + message: expect.objectContaining({ offset: '0' }), + }), + ]) + ); + + }, 10000); + + it('works for both partitions', async () => { + await consumer.connect(); + await producer.connect(); + + const value1 = secureRandom(); + const message1 = { key: `key-1`, value: `value-${value1}`, partition: 1, }; + const value2 = secureRandom(); + const message2 = { key: `key-1`, value: `value-${value2}`, partition: 1, }; + const value3 = secureRandom(); + const message3 = { key: `key-0`, value: `value-${value3}`, partition: 0 }; + const value4 = secureRandom(); + const message4 = { key: `key-0`, value: `value-${value4}`, partition: 0, }; + const value5 = secureRandom(); + const message5 = { key: `key-0`, value: `value-${value5}`, partition: 0 }; + + await producer.send({ + topic: topicName, + messages: [message1, message2, message3, message4, message5], + }); + await consumer.subscribe({ topic: topicName }) + + const messagesConsumed = []; + consumer.seek({ topic: topicName, partition: 0, offset: 2 }); + consumer.seek({ topic: topicName, partition: 1, offset: 1 }); + consumer.run({ + eachMessage: async event => { + messagesConsumed.push(event); + } + }); + + let check = await expect(waitForMessages(messagesConsumed, { number: 2 })).resolves; + + await check.toEqual( + expect.arrayContaining([ + expect.objectContaining({ + topic: topicName, + partition: 0, + message: expect.objectContaining({ offset: '2' }), + }), + expect.objectContaining({ + topic: topicName, + partition: 1, + message: expect.objectContaining({ offset: '1' }), + }), + ]) + ); + + await check.toEqual( + expect.not.arrayContaining([ + expect.objectContaining({ + topic: topicName, + partition: 0, + message: expect.objectContaining({ offset: '0' }), + }), + expect.objectContaining({ + topic: topicName, + partition: 0, + message: expect.objectContaining({ offset: '1' }), + }), + expect.objectContaining({ + topic: topicName, + partition: 1, + message: expect.objectContaining({ offset: '0' }), + }), + ]) + ); + + }, 10000); + + it('uses the last seek for a given topic/partition', async () => { + await consumer.connect() + await producer.connect() + + const value1 = secureRandom() + const message1 = { key: `key-0`, value: `value-${value1}` } + const value2 = secureRandom() + const message2 = { key: `key-0`, value: `value-${value2}` } + const value3 = secureRandom() + const message3 = { key: `key-0`, value: `value-${value3}` } + + await producer.send({ topic: topicName, messages: [message1, message2, message3] }) + await consumer.subscribe({ topic: topicName, }) + + const messagesConsumed = [] + consumer.seek({ topic: topicName, partition: 0, offset: 0 }); + consumer.seek({ topic: topicName, partition: 0, offset: 1 }); + consumer.seek({ topic: topicName, partition: 0, offset: 2 }); + consumer.run({ + eachMessage: async event => { + messagesConsumed.push(event); + } + }); + + let check = await expect(waitForMessages(messagesConsumed, { number: 1 })).resolves; + + await check.toEqual( + expect.arrayContaining([ + expect.objectContaining({ + topic: topicName, + partition: 0, + message: expect.objectContaining({ offset: '2' }), + }), + ]) + ); + + await check.toEqual( + expect.not.arrayContaining([ + expect.objectContaining({ + topic: topicName, + partition: 0, + message: expect.objectContaining({ offset: '0' }), + }), + expect.objectContaining({ + topic: topicName, + partition: 0, + message: expect.objectContaining({ offset: '1' }), + }), + ]) + ); + }, 10000); + }); + }); +}) diff --git a/test/promisified/testhelpers.js b/test/promisified/testhelpers.js index e3acdc45..632788f9 100644 --- a/test/promisified/testhelpers.js +++ b/test/promisified/testhelpers.js @@ -26,10 +26,12 @@ function secureRandom(length = 10) { } async function createTopic(args) { - const { topic } = args; + const { topic, partitions } = args; const admin = createAdmin({}); await admin.connect(); - await admin.createTopics({ topics: [{topic}] }); + await admin.createTopics({ topics: [ + { topic, numPartitions: partitions ?? 1 } + ] }); await admin.disconnect(); } @@ -43,7 +45,7 @@ async function waitForMessages(messagesConsumed, { number } = {number: 0}) { const interval = setInterval(() => { if (messagesConsumed.length >= number) { clearInterval(interval); - resolve(); + resolve(messagesConsumed); } }, 200); }); From 818ce7b135ee654287e5d1b604590aab0fa9d991 Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 12 Dec 2023 10:28:35 +0530 Subject: [PATCH 11/26] Add tests for consumer.pause() and consumer.resume() --- MIGRATION.md | 6 +- lib/kafkajs/_consumer.js | 110 +++- test/promisified/consumer/pause.spec.js | 660 ++++++++++++++++++++++++ test/promisified/testhelpers.js | 13 +- 4 files changed, 759 insertions(+), 30 deletions(-) create mode 100644 test/promisified/consumer/pause.spec.js diff --git a/MIGRATION.md b/MIGRATION.md index f4ab3003..ae2029fe 100644 --- a/MIGRATION.md +++ b/MIGRATION.md @@ -289,11 +289,13 @@ * The `partitionsConsumedConcurrently` property is not supported (YET). * The `eachBatch` method is not supported. * `commitOffsets` does not (YET) support sending metadata for topic partitions being committed. - * `paused()` is not (YET) supported. + * `paused()` is supported without any changes. * Custom partition assignors are not supported. * Changes to `seek`: - * The restriction to call seek only after `run` is removed. + * The restriction to call seek only after `run` is removed. It can be called any time. * Rather than the `autoCommit` property of `run` deciding if the offset is committed, the librdkafka property `enable.auto.commit` of the consumer config is used. + * `pause` and `resume` MUST be called after the consumer group is joined. In practice, this means it can be called whenever `consumer.assignment()` has a non-zero size, or within the `eachMessage` + callback. ### Admin Client diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 889fc6ae..98266ce1 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -67,6 +67,12 @@ class Consumer { */ #pendingSeeks = new Map(); + /** + * Stores the list of paused partitions, as a set of JSON.stringify'd TopicPartition objects. + * @type {Set} + */ + #pausedPartitions = new Set(); + /** * @constructor * @param {import("../../types/kafkajs").ConsumerConfig} kJSConfig @@ -300,7 +306,7 @@ class Consumer { headers }, heartbeat: async () => { /* no op */ }, - pause: () => this.pause([{ topic: message.topic, partitions: [message.partition] }]), + pause: this.pause.bind(this, [{ topic: message.topic, partitions: [message.partition] }]), }; } @@ -443,33 +449,56 @@ class Consumer { while (this.#state === ConsumerState.CONNECTED) { const m = await this.#consumeSingle(); - if (m) { - /* TODO: add partitionsConsumedConcurrently-based concurrency here. - * If we maintain a map of topic partitions to promises, and a counter, - * we can probably achieve it with the correct guarantees of ordering - * though to maximize performance, we need to consume only from partitions for which - * an eachMessage call is not already going. - * It's risky to consume, and then store the message in something like an - * array/list until it can be processed, because librdkafka marks it as - * 'stored'... but anyway - we can implement something like this. - */ - if (this.#checkPendingSeeks) { - const invalidateMessage = await this.#seekInternal({ topic: m.topic, partition: m.partition }); - if (invalidateMessage) { - /* Don't pass this message on to the user if this topic partition was seeked to. */ - continue; - } + if (!m) { + continue; + } + + /* TODO: add partitionsConsumedConcurrently-based concurrency here. + * If we maintain a map of topic partitions to promises, and a counter, + * we can probably achieve it with the correct guarantees of ordering + * though to maximize performance, we need to consume only from partitions for which + * an eachMessage call is not already going. + * It's risky to consume, and then store the message in something like an + * array/list until it can be processed, because librdkafka marks it as + * 'stored'... but anyway - we can implement something like this. + */ + + /* Make pending seeks 'concrete'. */ + if (this.#checkPendingSeeks) { + const invalidateMessage = await this.#seekInternal({ topic: m.topic, partition: m.partition }); + if (invalidateMessage) { + /* Don't pass this message on to the user if this topic partition was seeked to. */ + continue; } + } + try { await config.eachMessage( this.#createPayload(m) ) - /* TODO: another check we need to do here is to see how kafkaJS is handling - * commits. Are they commmitting after a message is _processed_? - * In that case we need to turn off librdkafka's auto-commit, and commit - * inside this function. - */ + } catch (e) { + /* It's not only possible, but expected that an error will be thrown by eachMessage. + * This is especially true since the pattern of pause() followed by throwing an error + * is encouraged. To meet the API contract, we seek one offset backward at this point (which + * means seeking to the message offset). */ + this.seek({ + topic: m.topic, + partition: m.partition, + offset: m.offset, + }); } + + /* Force a immediate seek here. It's possible that there are no more messages to be passed to the user, + * but the user seeked in the call to eachMessage, or else we encountered the error catch block. + * In that case, the results of that seek will never be reflected unless we do this. */ + if (this.#checkPendingSeeks) + await this.#seekInternal(); + + /* TODO: another check we need to do here is to see how kafkaJS is handling + * commits. Are they commmitting after a message is _processed_? + * In that case we need to turn off librdkafka's auto-commit, and commit + * inside this function. + */ } } @@ -646,6 +675,7 @@ class Consumer { * all partitions for the given topic. If topic partition(s) are already paused * this method has no effect. * @param {{topic: string, partitions?: number[]}[]} topics + * @returns {Function} a function that can be called to resume the given topic partitions. */ pause(topics) { if (this.#state !== ConsumerState.CONNECTED) { @@ -653,6 +683,10 @@ class Consumer { } for (let topic of topics) { + if (typeof topic.topic !== 'string') { + throw new error.KafkaJSError('Topic must be a string.', { code: error.ErrorCodes.ERR__INVALID_ARG }); + } + if (!topic.partitions) { topic.partitions = this.#getAllAssignedPartition(topic.topic); } @@ -662,12 +696,31 @@ class Consumer { if (topics.length === 0) { return; } - this.#internalClient.pause(topics); + + topics.map(JSON.stringify).forEach(topicPartition => this.#pausedPartitions.add(topicPartition)); + + return () => this.resume(topics); } + /** + * Returns the list of paused topic partitions. + * @returns {{topic: string, partitions: number[]}[]} a list of paused topic partitions. + */ paused() { - notImplemented(); + const topicToPartitions = Array + .from(this.#pausedPartitions.values()) + .map(JSON.parse) + .reduce( + (acc, { topic, partition }) => { + if (!acc[topic]) { + acc[topic] = []; + } + acc[topic].push(partition); + return acc; + }, + {}); + return Array.from(Object.entries(topicToPartitions), ([topic, partitions]) => ({ topic, partitions })); } @@ -683,13 +736,22 @@ class Consumer { } for (let topic of topics) { + if (typeof topic.topic !== 'string') { + throw new error.KafkaJSError('Topic must be a string.', { code: error.ErrorCodes.ERR__INVALID_ARG }); + } + if (!topic.partitions) { topic.partitions = this.#getAllAssignedPartition(topic.topic); } } topics = this.#flattenTopicPartitions(topics); + if (topics.length === 0) { + return; + } this.#internalClient.resume(topics); + + topics.map(JSON.stringify).forEach(topicPartition => this.#pausedPartitions.delete(topicPartition)); } on(/* eventName, listener */) { diff --git a/test/promisified/consumer/pause.spec.js b/test/promisified/consumer/pause.spec.js new file mode 100644 index 00000000..6350f7e7 --- /dev/null +++ b/test/promisified/consumer/pause.spec.js @@ -0,0 +1,660 @@ +const { + secureRandom, + createTopic, + waitForMessages, + waitFor, + waitForConsumerToJoinGroup, + createConsumer, + createProducer, +} = require('../testhelpers'); + +describe('Consumer', () => { + let consumer; + let groupId, producer, topics; + + beforeEach(async () => { + topics = [`test-topic1-${secureRandom()}`, `test-topic2-${secureRandom()}`] + groupId = `consumer-group-id-${secureRandom()}` + + for (const topic of topics) { + await createTopic({ topic, partitions: 2 }) + } + + producer = createProducer({ + }); + + consumer = createConsumer({ + groupId, + maxWaitTimeInMs: 1, + maxBytesPerPartition: 180, + rdKafka: { + globalConfig: { + // 'debug': 'fetch', + }, + topicConfig: { + 'auto.offset.reset': 'earliest', + }, + } + }); + }) + + afterEach(async () => { + consumer && (await consumer.disconnect()) + producer && (await producer.disconnect()) + }) + + describe('when pausing', () => { + it('throws an error if the topic is invalid', async () => { + await consumer.connect(); + expect(() => consumer.pause([{ topic: null, partitions: [0] }])).toThrow('Topic must be a string'); + }); + + it('throws an error if Consumer#connect has not been called', () => { + expect(() => consumer.pause([{ topic: 'foo', partitions: [0] }])).toThrow('Pause can only be called while connected'); + }); + + it('pauses the appropriate topic/partition when pausing via the eachMessage callback', async () => { + await consumer.connect(); + await producer.connect(); + + /* Send 4 of the same messages to each topic, in order to partition 0, 0, 1, 0 of that topic. */ + const messages = [0, 0, 1, 0].map(partition => { + const key = secureRandom() + return { key: `key-${key}`, value: `value-${key}`, partition } + }) + + /* Send the first 2 messages to each topic. */ + for (const topic of topics) { + await producer.send({ topic, messages: messages.slice(0, 2) }); + } + await consumer.subscribe({ topics: topics }); + + let shouldPause = true; + let pauseMessageRecvd = false; + const messagesConsumed = []; + const resumeCallbacks = []; + consumer.run({ + eachMessage: async event => { + const { topic, message, pause } = event; + + const whichTopic = topics.indexOf(topic) + const whichMessage = messages.findIndex(m => String(m.key) === String(message.key)) + + /* In case we're at the 2nd message (idx = 1) for the first topic, pause the partition. + * It should be the 0th partition which gets paused. */ + if (shouldPause && whichTopic === 0 && whichMessage === 1) { + resumeCallbacks.push(pause()); + pauseMessageRecvd = true; + /* We throw an error to indicate to the runner that this message should be + * considered 'unprocessed'. */ + throw new Error('bailing out'); + } + messagesConsumed.push({ + topic: whichTopic, + message: whichMessage, + }); + }, + }); + + await waitForMessages(messagesConsumed, { number: 3 }); + /* Librdkafka provides no guarantee about message ordering beyond per-partition. + * Encountering 3 messages is no guarantee of that we did manage to pause. */ + await waitFor(() => pauseMessageRecvd, () => { }, { delay: 100 }); + const [pausedTopic] = topics; + expect(consumer.paused()).toEqual([{ topic: pausedTopic, partitions: [0] }]) + + for (const topic of topics) { + await producer.send({ topic, messages: messages.slice(2) }) + } + await waitForMessages(messagesConsumed, { number: 6, delay: 10 }) + + expect(messagesConsumed).toHaveLength(6) + expect(messagesConsumed).toContainEqual({ topic: 0, message: 0 }) // partition 0 + expect(messagesConsumed).toContainEqual({ topic: 0, message: 2 }) // partition 1 + + expect(messagesConsumed).toContainEqual({ topic: 1, message: 0 }) // partition 0 + expect(messagesConsumed).toContainEqual({ topic: 1, message: 1 }) // partition 0 + expect(messagesConsumed).toContainEqual({ topic: 1, message: 2 }) // partition 1 + expect(messagesConsumed).toContainEqual({ topic: 1, message: 3 }) // partition 0 + + shouldPause = false; + resumeCallbacks.forEach(resume => resume()); + + await waitForMessages(messagesConsumed, { number: 8 }) + + // these messages have to wait until the consumer has resumed + expect(messagesConsumed).toHaveLength(8) + expect(messagesConsumed).toContainEqual({ topic: 0, message: 1 }) // partition 0 + expect(messagesConsumed).toContainEqual({ topic: 0, message: 3 }) // partition 0 + }, 10000); + + it('avoids calling eachMessage again for paused topics/partitions when paused via consumer.pause', async () => { + await consumer.connect() + await producer.connect() + const messages = [0, 0, 1, 0].map(partition => { + const key = secureRandom() + return { key: `key-${key}`, value: `value-${key}`, partition } + }) + + for (const topic of topics) { + await producer.send({ topic, messages: messages.slice(0, 2) }) + } + await consumer.subscribe({ topics }) + + let shouldPause = true + const messagesConsumed = [] + consumer.run({ + eachMessage: async event => { + const { topic, message, partition } = event + + const whichTopic = topics.indexOf(topic) + const whichMessage = messages.findIndex(m => String(m.key) === String(message.key)) + + messagesConsumed.push({ + topic: whichTopic, + message: whichMessage, + }) + + // here, we pause after the first message (0) on the first topic (0) + if (shouldPause && whichTopic === 0 && whichMessage === 0) { + consumer.pause([{ topic, partitions: [partition] }]) + // we don't throw an exception here to ensure the loop calling us breaks on its own and doesn't call us again + } + }, + }) + + await waitForMessages(messagesConsumed, { number: 3 }) + const [pausedTopic] = topics + expect(consumer.paused()).toEqual([{ topic: pausedTopic, partitions: [0] }]) + + for (const topic of topics) { + await producer.send({ topic, messages: messages.slice(2) }) + } + await waitForMessages(messagesConsumed, { number: 6, delay: 10 }) + + expect(messagesConsumed).toHaveLength(6) + expect(messagesConsumed).toContainEqual({ topic: 0, message: 0 }) // partition 0 + expect(messagesConsumed).toContainEqual({ topic: 0, message: 2 }) // partition 1 + + expect(messagesConsumed).toContainEqual({ topic: 1, message: 0 }) // partition 0 + expect(messagesConsumed).toContainEqual({ topic: 1, message: 1 }) // partition 0 + expect(messagesConsumed).toContainEqual({ topic: 1, message: 2 }) // partition 1 + expect(messagesConsumed).toContainEqual({ topic: 1, message: 3 }) // partition 0 + + shouldPause = false + consumer.resume(consumer.paused()) + + await waitForMessages(messagesConsumed, { number: 8 }) + + // these messages have to wait until the consumer has resumed + expect(messagesConsumed).toHaveLength(8) + expect(messagesConsumed).toContainEqual({ topic: 0, message: 1 }) // partition 0 + expect(messagesConsumed).toContainEqual({ topic: 0, message: 3 }) // partition 0 + }, 10000); + + /* Skip until eachBatch is made available */ + it.skip('pauses when pausing via the eachBatch callback - skipped until eachBatch is made available', async () => { + await consumer.connect() + await producer.connect() + const originalMessages = [0, 0, 0, 1].map(partition => { + const key = secureRandom() + return { key: `key-${key}`, value: `value-${key}`, partition } + }) + + for (const topic of topics) { + await producer.send({ topic, messages: originalMessages }) + await consumer.subscribe({ topic }) + } + + let shouldPause = true + const messagesConsumed = [] + const resumeCallbacks = [] + consumer.run({ + eachBatch: async event => { + const { + batch: { topic, messages }, + pause, + resolveOffset, + commitOffsetsIfNecessary, + } = event + messages.every(message => { + const whichTopic = topics.indexOf(topic) + const whichMessage = originalMessages.findIndex( + m => String(m.key) === String(message.key) + ) + + if (shouldPause && whichTopic === 0 && whichMessage === 1) { + resumeCallbacks.push(pause()) + return false + } else if (shouldPause && whichTopic === 1 && whichMessage === 3) { + resumeCallbacks.push(pause()) + return false + } + messagesConsumed.push({ + topic: whichTopic, + message: whichMessage, + }) + resolveOffset(message.offset) + return true + }) + await commitOffsetsIfNecessary() + }, + eachBatchAutoResolve: false, + }) + await waitForConsumerToJoinGroup(consumer) + await waitForMessages(messagesConsumed, { number: 5 }) + expect(consumer.paused()).toContainEqual({ topic: topics[0], partitions: [0] }) + expect(consumer.paused()).toContainEqual({ topic: topics[1], partitions: [1] }) + shouldPause = false + resumeCallbacks.forEach(resume => resume()) + await waitForMessages(messagesConsumed, { number: 8 }) + expect(consumer.paused()).toEqual([]) + expect(messagesConsumed).toContainEqual({ topic: 0, message: 1 }) + expect(messagesConsumed).toContainEqual({ topic: 1, message: 3 }) + }); + + it('does not fetch messages for the paused topic', async () => { + await consumer.connect(); + await producer.connect(); + + const key1 = secureRandom(); + const message1 = { key: `key-${key1}`, value: `value-${key1}`, partition: 0 }; + const key2 = secureRandom(); + const message2 = { key: `key-${key2}`, value: `value-${key2}`, partition: 1 }; + + for (const topic of topics) { + await producer.send({ topic, messages: [message1] }); + } + await consumer.subscribe({ topics }); + + const messagesConsumed = []; + consumer.run({ eachMessage: async event => messagesConsumed.push(event) }); + await waitForMessages(messagesConsumed, { number: 2 }); + + expect(consumer.paused()).toEqual([]); + const [pausedTopic, activeTopic] = topics; + consumer.pause([{ topic: pausedTopic }]); + + for (const topic of topics) { + await producer.send({ topic, messages: [message2] }); + } + + const consumedMessages = await waitForMessages(messagesConsumed, { number: 3 }); + + expect(consumedMessages.filter(({ topic }) => topic === pausedTopic)).toEqual([ + expect.objectContaining({ + topic: pausedTopic, + partition: expect.any(Number), + message: expect.objectContaining({ offset: '0' }), + }), + ]); + + const byPartition = (a, b) => a.partition - b.partition + expect( + consumedMessages.filter(({ topic }) => topic === activeTopic).sort(byPartition) + ).toEqual([ + expect.objectContaining({ + topic: activeTopic, + partition: 0, + message: expect.objectContaining({ offset: '0' }), + }), + expect.objectContaining({ + topic: activeTopic, + partition: 1, + message: expect.objectContaining({ offset: '0' }), + }), + ]); + + expect(consumer.paused()).toEqual([ + { + topic: pausedTopic, + partitions: [0, 1], + }, + ]); + }, 10000); + + it('does not fetch messages for the paused partitions', async () => { + await consumer.connect(); + await producer.connect(); + + const [topic] = topics; + const partitions = [0, 1]; + + const messages = Array(1) + .fill() + .map(() => { + const value = secureRandom() + return { key: `key-${value}`, value: `value-${value}` } + }); + const forPartition = partition => message => ({ ...message, partition }); + + for (const partition of partitions) { + await producer.send({ topic, messages: messages.map(forPartition(partition)) }); + } + await consumer.subscribe({ topic }); + + const messagesConsumed = [] + consumer.run({ eachMessage: async event => messagesConsumed.push(event) }); + + await waitForMessages(messagesConsumed, { number: messages.length * partitions.length }); + + expect(consumer.paused()).toEqual([]); + const [pausedPartition, activePartition] = partitions; + consumer.pause([{ topic, partitions: [pausedPartition] }]); + + for (const partition of partitions) { + await producer.send({ topic, messages: messages.map(forPartition(partition)) }); + } + + const consumedMessages = await waitForMessages(messagesConsumed, { + number: messages.length * 3, + }); + + expect(consumedMessages.filter(({ partition }) => partition === pausedPartition)).toEqual( + messages.map((message, i) => + expect.objectContaining({ + topic, + partition: pausedPartition, + message: expect.objectContaining({ offset: `${i}` }), + }) + ) + ); + + expect(consumedMessages.filter(({ partition }) => partition !== pausedPartition)).toEqual( + messages.concat(messages).map((message, i) => + expect.objectContaining({ + topic, + partition: activePartition, + message: expect.objectContaining({ offset: `${i}` }), + }) + ) + ); + + expect(consumer.paused()).toEqual([ + { + topic, + partitions: [pausedPartition], + }, + ]); + }, 10000); + }); + + describe('when pausing and breaking the consumption', () => { + it('does not process messages when consumption from topic is paused', async () => { + const [topic] = topics; + const key1 = secureRandom(); + const message1 = { key: `key-${key1}`, value: `value-${key1}`, partition: 0 }; + const messagesConsumed = []; + let shouldThrow = true; + + await consumer.connect(); + await producer.connect(); + + await producer.send({ topic, messages: [message1] }); + await consumer.subscribe({ topic }); + + consumer.run({ + eachMessage: async event => { + messagesConsumed.push(event) + if (shouldThrow) { + consumer.pause([{ topic }]) + throw new Error('Should fail') + } + }, + }); + + const consumedMessagesTillError = [ + ...(await waitForMessages(messagesConsumed, { delay: 100 })), + ]; + + shouldThrow = false; + consumer.resume([{ topic }]); + + const consumedMessages = await waitForMessages(messagesConsumed, { number: 2 }) + + expect(consumedMessagesTillError).toHaveLength(1) + expect(consumedMessagesTillError).toEqual([ + expect.objectContaining({ + topic, + partition: expect.any(Number), + message: expect.objectContaining({ offset: '0' }), + }), + ]) + expect(consumedMessages).toHaveLength(2) + expect(consumedMessages).toEqual([ + expect.objectContaining({ + topic, + partition: expect.any(Number), + message: expect.objectContaining({ offset: '0' }), + }), + expect.objectContaining({ + topic, + partition: expect.any(Number), + message: expect.objectContaining({ offset: '0' }), + }), + ]) + }, 10000); + + it('does not process messages when consumption from topic-partition is paused', async () => { + const [topic] = topics; + const pausedPartition = 0; + const key1 = secureRandom(); + const message1 = { key: `key-${key1}`, value: `value-${key1}`, partition: 0 }; + const key2 = secureRandom(); + const message2 = { key: `key-${key2}`, value: `value-${key2}`, partition: 1 }; + const messagesConsumed = []; + let shouldThrow = true; + + await consumer.connect(); + await producer.connect(); + + await producer.send({ topic, messages: [message1, message2] }) + await consumer.subscribe({ topic }) + + consumer.run({ + eachMessage: async event => { + messagesConsumed.push(event) + if (shouldThrow && event.partition === pausedPartition) { + consumer.pause([{ topic, partitions: [pausedPartition] }]) + throw new Error('Should fail') + } + }, + }); + + const consumedMessagesTillError = [ + ...(await waitForMessages(messagesConsumed, { number: 2 })), + ]; + + shouldThrow = false; + consumer.resume([{ topic, partitions: [pausedPartition] }]); + + const consumedMessages = await waitForMessages(messagesConsumed, { number: 3 }); + + expect(consumedMessagesTillError).toHaveLength(2); + expect(consumedMessagesTillError).toEqual( + expect.arrayContaining([ + expect.objectContaining({ + topic, + partition: 0, + message: expect.objectContaining({ offset: '0' }), + }), + expect.objectContaining({ + topic, + partition: 1, + message: expect.objectContaining({ offset: '0' }), + }), + ]) + ); + expect(consumedMessages).toHaveLength(3); + expect(consumedMessages).toEqual( + expect.arrayContaining([ + expect.objectContaining({ + topic, + partition: 0, + message: expect.objectContaining({ offset: '0' }), + }), + expect.objectContaining({ + topic, + partition: 0, + message: expect.objectContaining({ offset: '0' }), + }), + expect.objectContaining({ + topic, + partition: 1, + message: expect.objectContaining({ offset: '0' }), + }), + ]) + ); + }, 10000); + }); + + describe('when all topics are paused', () => { + it('does not fetch messages', async () => { + consumer = createConsumer({ + groupId, + maxWaitTimeInMs: 100, + maxBytesPerPartition: 180, + }) + + await producer.connect() + await consumer.connect() + + const [topic1, topic2] = topics + await consumer.subscribe({ topics: [topic1, topic2] }); + + const eachMessage = jest.fn() + consumer.run({ eachMessage }) + + consumer.pause([{ topic: topic1 }, { topic: topic2 }]) + + const key1 = secureRandom() + const message1 = { key: `key-${key1}`, value: `value-${key1}`, partition: 0 } + + await producer.send({ topic: topic1, messages: [message1] }) + await producer.send({ topic: topic2, messages: [message1] }) + + expect(eachMessage).not.toHaveBeenCalled() + }) + }); + + describe('when resuming', () => { + it('throws an error if the topic is invalid', async () => { + await consumer.connect(); + expect(() => consumer.pause([{ topic: null, partitions: [0] }])).toThrow('Topic must be a string'); + }); + + it('throws an error if Consumer#connect has not been called', () => { + expect(() => consumer.resume([{ topic: 'foo', partitions: [0] }])).toThrow( + 'Resume can only be called while connected' + ); + }); + + it('resumes fetching from the specified topic', async () => { + await consumer.connect(); + await producer.connect(); + + const key = secureRandom(); + const message = { key: `key-${key}`, value: `value-${key}`, partition: 0 }; + + await consumer.subscribe({ topics }); + + const messagesConsumed = []; + consumer.run({ + eachMessage: async event => { + return messagesConsumed.push(event); + } + }); + await waitFor(() => consumer.assignment().length > 0, () => { }, { delay: 10 }); + const [pausedTopic, activeTopic] = topics; + consumer.pause([{ topic: pausedTopic }]); + + for (const topic of topics) { + await producer.send({ topic, messages: [message] }); + } + + await waitForMessages(messagesConsumed, { number: 1 }); + + consumer.resume([{ topic: pausedTopic }]); + + await expect(waitForMessages(messagesConsumed, { number: 2 })).resolves.toEqual([ + expect.objectContaining({ + topic: activeTopic, + partition: 0, + message: expect.objectContaining({ offset: '0' }), + }), + expect.objectContaining({ + topic: pausedTopic, + partition: 0, + message: expect.objectContaining({ offset: '0' }), + }), + ]); + + expect(consumer.paused()).toEqual([]); + }, 10000); + + it('resumes fetching from earlier paused partitions', async () => { + await consumer.connect(); + await producer.connect(); + + const [topic] = topics; + const partitions = [0, 1]; + + const messages = Array(1) + .fill() + .map(() => { + const value = secureRandom() + return { key: `key-${value}`, value: `value-${value}` } + }); + const forPartition = partition => message => ({ ...message, partition }); + + for (const partition of partitions) { + await producer.send({ topic, messages: messages.map(forPartition(partition)) }); + } + await consumer.subscribe({ topic }); + + const messagesConsumed = []; + consumer.run({ eachMessage: async event => messagesConsumed.push(event) }); + + await waitForMessages(messagesConsumed, { number: messages.length * partitions.length }); + + const [pausedPartition, activePartition] = partitions; + consumer.pause([{ topic, partitions: [pausedPartition] }]); + + for (const partition of partitions) { + await producer.send({ topic, messages: messages.map(forPartition(partition)) }); + } + + await waitForMessages(messagesConsumed, { + number: messages.length * 3, + }); + + consumer.resume([{ topic, partitions: [pausedPartition] }]); + + const consumedMessages = await waitForMessages(messagesConsumed, { + number: messages.length * 4, + }); + + expect(consumedMessages.filter(({ partition }) => partition === pausedPartition)).toEqual( + messages.concat(messages).map((message, i) => + expect.objectContaining({ + topic, + partition: pausedPartition, + message: expect.objectContaining({ offset: `${i}` }), + }) + ) + ) + + expect(consumedMessages.filter(({ partition }) => partition !== pausedPartition)).toEqual( + messages.concat(messages).map((message, i) => + expect.objectContaining({ + topic, + partition: activePartition, + message: expect.objectContaining({ offset: `${i}` }), + }) + ) + ) + + expect(consumer.paused()).toEqual([]) + }, 10000); + }); +}) diff --git a/test/promisified/testhelpers.js b/test/promisified/testhelpers.js index 632788f9..0a8b9362 100644 --- a/test/promisified/testhelpers.js +++ b/test/promisified/testhelpers.js @@ -40,17 +40,21 @@ async function waitForConsumerToJoinGroup(consumer) { return new Promise(resolve => setTimeout(resolve, 2500)); } -async function waitForMessages(messagesConsumed, { number } = {number: 0}) { +async function waitFor(check, resolveValue, { delay = 50 } = {}) { return new Promise(resolve => { const interval = setInterval(() => { - if (messagesConsumed.length >= number) { + if (check()) { clearInterval(interval); - resolve(messagesConsumed); + resolve(resolveValue()); } - }, 200); + }, delay); }); } +async function waitForMessages(messagesConsumed, { number = 1, delay } = {}) { + return waitFor(() => messagesConsumed.length >= number, () => messagesConsumed, { delay }); +} + module.exports = { createConsumer, createProducer, @@ -60,4 +64,5 @@ module.exports = { waitForMessages, createTopic, waitForConsumerToJoinGroup, + waitFor, } From 184e124561e381ee00940c6f605df10521b356dd Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 12 Dec 2023 15:17:20 +0530 Subject: [PATCH 12/26] Add test for consumer.run(). --- MIGRATION.md | 1 + examples/kafkajs/eos.js | 4 +- lib/kafkajs/_consumer.js | 43 +- lib/kafkajs/_producer.js | 10 +- .../consumer/consumeMessages.spec.js | 910 ++++++++++++++++++ test/promisified/consumer/pause.spec.js | 27 +- test/promisified/consumer/subscribe.spec.js | 6 +- test/promisified/testhelpers.js | 21 + 8 files changed, 997 insertions(+), 25 deletions(-) create mode 100644 test/promisified/consumer/consumeMessages.spec.js diff --git a/MIGRATION.md b/MIGRATION.md index ae2029fe..3c5e7f6d 100644 --- a/MIGRATION.md +++ b/MIGRATION.md @@ -178,6 +178,7 @@ * Error-handling for a failed `send` is stricter. While sending multiple messages, even if one of the messages fails, the method throws an error. * `sendBatch` is supported. However, the actual batching semantics are handled by librdkafka, and it just acts as a wrapper around `send` (See `send` for changes). +* A transactional producer (with a `transactionId`) set, can only send messages after calling `producer.transaction()`. ### Consumer diff --git a/examples/kafkajs/eos.js b/examples/kafkajs/eos.js index be06485e..d0aabcf3 100644 --- a/examples/kafkajs/eos.js +++ b/examples/kafkajs/eos.js @@ -66,7 +66,9 @@ async function eosStart() { { topic, partitions: [ - { partition, offset: message.offset }, + /* The message.offset indicates current offset, so we need to add 1 to it, since committed offset denotes + * the next offset to consume. */ + { partition, offset: message.offset + 1 }, ], } ], diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 98266ce1..3ddedd0d 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -449,6 +449,14 @@ class Consumer { while (this.#state === ConsumerState.CONNECTED) { const m = await this.#consumeSingle(); + + /* It's possible for the state to change as we await something. + * Unfortunately, we have no alternative but to break and drop the message, if it exists. + * TODO: fix this, possibly with a flag in disconnect that waits until the run loop winds down. + */ + if (this.#state !== ConsumerState.CONNECTED) + break; + if (!m) { continue; } @@ -470,28 +478,42 @@ class Consumer { /* Don't pass this message on to the user if this topic partition was seeked to. */ continue; } + + /* It's possible for the state to change as we await something. + * Unfortunately, we have no alternative but to break and drop the message. + * TODO: fix this, possibly with a flag in disconnect that waits until the run loop winds down. + */ + if (this.#state !== ConsumerState.CONNECTED) + break; } try { await config.eachMessage( this.#createPayload(m) ) + + /* It's possible for the state to change as we await something. + * Unfortunately, we have no alternative but to break without taking any action that the user might need. + */ + if (this.#state !== ConsumerState.CONNECTED) + break; } catch (e) { /* It's not only possible, but expected that an error will be thrown by eachMessage. * This is especially true since the pattern of pause() followed by throwing an error * is encouraged. To meet the API contract, we seek one offset backward at this point (which * means seeking to the message offset). */ - this.seek({ - topic: m.topic, - partition: m.partition, - offset: m.offset, - }); + if (this.#state === ConsumerState.CONNECTED) + this.seek({ + topic: m.topic, + partition: m.partition, + offset: m.offset, + }); } /* Force a immediate seek here. It's possible that there are no more messages to be passed to the user, * but the user seeked in the call to eachMessage, or else we encountered the error catch block. * In that case, the results of that seek will never be reflected unless we do this. */ - if (this.#checkPendingSeeks) + if (this.#checkPendingSeeks && this.#state === ConsumerState.CONNECTED) await this.#seekInternal(); /* TODO: another check we need to do here is to see how kafkaJS is handling @@ -499,6 +521,14 @@ class Consumer { * In that case we need to turn off librdkafka's auto-commit, and commit * inside this function. */ + + /* Yield for a bit to allow other scheduled tasks on the event loop to run. + * For instance, if disconnect() is called during/after we await eachMessage, and + * we don't await anything else after that, this loop will run despite needing to + * disconnect. + * It's better than any pending tasks be processed here, while we've processed one + * message completely, rather than between message processing. */ + await new Promise((resolve) => setTimeout(resolve, 0)); } } @@ -573,7 +603,6 @@ class Consumer { */ async #seekInternal(messageTopicPartition) { this.#checkPendingSeeks = false; - const assignment = this.assignment(); const offsetsToCommit = []; let invalidateMessage = false; diff --git a/lib/kafkajs/_producer.js b/lib/kafkajs/_producer.js index cf5ac62d..220dbb47 100644 --- a/lib/kafkajs/_producer.js +++ b/lib/kafkajs/_producer.js @@ -169,8 +169,8 @@ class Producer { throw new error.KafkaJSError(`Ready callback called in invalid state ${this.#state}`, { code: error.ErrorCodes.ERR__STATE }); } - const config = await this.#config(); - if (Object.hasOwn(config, 'transactional.id') && this.#state !== ProducerState.INITIALIZED_TRANSACTIONS) { + const {globalConfig} = await this.#config(); + if (Object.hasOwn(globalConfig, 'transactional.id') && this.#state !== ProducerState.INITIALIZED_TRANSACTIONS) { this.#state = ProducerState.INITIALIZING_TRANSACTIONS; this.#internalClient.initTransactions(5000 /* default: 5s */, this.#readyTransactions.bind(this)); return; @@ -450,7 +450,11 @@ class Producer { msgPromises.push(new Promise((resolve, reject) => { const opaque = { resolve, reject }; - this.#internalClient.produce(sendOptions.topic, msg.partition, msg.value, msg.key, msg.timestamp, opaque, msg.headers); + try { + this.#internalClient.produce(sendOptions.topic, msg.partition, msg.value, msg.key, msg.timestamp, opaque, msg.headers); + } catch(err) { + reject(err); + } })); } diff --git a/test/promisified/consumer/consumeMessages.spec.js b/test/promisified/consumer/consumeMessages.spec.js new file mode 100644 index 00000000..de3ebb88 --- /dev/null +++ b/test/promisified/consumer/consumeMessages.spec.js @@ -0,0 +1,910 @@ +jest.setTimeout(30000) + +const { ErrorCodes } = require('../../../lib').KafkaJS; + +const { + secureRandom, + createTopic, + waitFor, + createProducer, + createConsumer, + waitForMessages, + waitForConsumerToJoinGroup, + sleep, + generateMessages, +} = require('../testhelpers') + +describe('Consumer', () => { + let topicName, groupId, producer, consumer; + + beforeEach(async () => { + topicName = `test-topic-${secureRandom()}` + groupId = `consumer-group-id-${secureRandom()}` + + await createTopic({ topic: topicName }) + + producer = createProducer({}); + + consumer = createConsumer({ + groupId, + maxWaitTimeInMs: 100, + rdKafka: { + topicConfig: { + 'auto.offset.reset': 'earliest', + } + } + }); + }); + + afterEach(async () => { + consumer && (await consumer.disconnect()) + producer && (await producer.disconnect()) + }); + + it('consume messages', async () => { + await consumer.connect(); + await producer.connect(); + await consumer.subscribe({ topic: topicName }) + + const messagesConsumed = []; + consumer.run({ eachMessage: async event => messagesConsumed.push(event) }); + + const messages = Array(100) + .fill() + .map(() => { + const value = secureRandom() + return { key: `key-${value}`, value: `value-${value}` } + }) + + await producer.send({ topic: topicName, messages }) + await waitForMessages(messagesConsumed, { number: messages.length }) + + expect(messagesConsumed[0]).toEqual( + expect.objectContaining({ + topic: topicName, + partition: 0, + message: expect.objectContaining({ + key: Buffer.from(messages[0].key), + value: Buffer.from(messages[0].value), + offset: '0', + }), + }) + ) + + expect(messagesConsumed[messagesConsumed.length - 1]).toEqual( + expect.objectContaining({ + topic: topicName, + partition: 0, + message: expect.objectContaining({ + key: Buffer.from(messages[messages.length - 1].key), + value: Buffer.from(messages[messages.length - 1].value), + offset: '99', + }), + }) + ) + + // check if all offsets are present + expect(messagesConsumed.map(m => m.message.offset)).toEqual(messages.map((_, i) => `${i}`)) + }); + + /* Skip until concurrency support for eachMessage is added. */ + it.skip('consumes messages concurrently', async () => { + const partitionsConsumedConcurrently = 2 + topicName = `test-topic-${secureRandom()}` + await createTopic({ + topic: topicName, + partitions: partitionsConsumedConcurrently + 1, + }) + await consumer.connect() + await producer.connect() + await consumer.subscribe({ topic: topicName }) + + let inProgress = 0 + let hitConcurrencyLimit = false + consumer.on(consumer.events.START_BATCH_PROCESS, () => { + inProgress++ + expect(inProgress).toBeLessThanOrEqual(partitionsConsumedConcurrently) + hitConcurrencyLimit = hitConcurrencyLimit || inProgress === partitionsConsumedConcurrently + }) + consumer.on(consumer.events.END_BATCH_PROCESS, () => inProgress--) + + const messagesConsumed = [] + consumer.run({ + partitionsConsumedConcurrently, + eachMessage: async event => { + await sleep(1) + messagesConsumed.push(event) + }, + }) + + await waitForConsumerToJoinGroup(consumer) + + const messages = Array(100) + .fill() + .map(() => { + const value = secureRandom() + return { key: `key-${value}`, value: `value-${value}` } + }) + + await producer.send({ topic: topicName, messages }) + await waitForMessages(messagesConsumed, { number: messages.length }) + + expect(hitConcurrencyLimit).toBeTrue() + }); + + it('consume GZIP messages', async () => { + /* Discard and recreate producer with the compression set */ + producer = createProducer({ + rdKafka: { + globalConfig: { + 'compression.codec': 'gzip', + } + } + }); + + await consumer.connect(); + await producer.connect(); + await consumer.subscribe({ topic: topicName }); + + const messagesConsumed = []; + consumer.run({ eachMessage: async event => messagesConsumed.push(event) }); + + const key1 = secureRandom(); + const message1 = { key: `key-${key1}`, value: `value-${key1}` }; + const key2 = secureRandom(); + const message2 = { key: `key-${key2}`, value: `value-${key2}` }; + + await producer.send({ + topic: topicName, + messages: [message1, message2], + }); + + await expect(waitForMessages(messagesConsumed, { number: 2 })).resolves.toEqual([ + expect.objectContaining({ + topic: topicName, + partition: 0, + message: expect.objectContaining({ + key: Buffer.from(message1.key), + value: Buffer.from(message1.value), + offset: '0', + }), + }), + expect.objectContaining({ + topic: topicName, + partition: 0, + message: expect.objectContaining({ + key: Buffer.from(message2.key), + value: Buffer.from(message2.value), + offset: '1', + }), + }), + ]) + }); + + /* Skip as it uses eachBatch */ + it.skip('commits the last offsets processed before stopping', async () => { + jest.spyOn(cluster, 'refreshMetadataIfNecessary') + + await Promise.all([admin.connect(), consumer.connect(), producer.connect()]) + await consumer.subscribe({ topic: topicName }) + + const messagesConsumed = [] + consumer.run({ eachMessage: async event => messagesConsumed.push(event) }) + await waitForConsumerToJoinGroup(consumer) + + // stop the consumer right after processing the batch, the offsets should be + // committed in the end + consumer.on(consumer.events.END_BATCH_PROCESS, async () => { + await consumer.stop() + }) + + const messages = Array(100) + .fill() + .map(() => { + const value = secureRandom() + return { key: `key-${value}`, value: `value-${value}` } + }) + + await producer.send({ topic: topicName, messages }) + await waitForMessages(messagesConsumed, { number: messages.length }) + + expect(cluster.refreshMetadataIfNecessary).toHaveBeenCalled() + + expect(messagesConsumed[0]).toEqual( + expect.objectContaining({ + topic: topicName, + partition: 0, + message: expect.objectContaining({ + key: Buffer.from(messages[0].key), + value: Buffer.from(messages[0].value), + offset: '0', + }), + }) + ) + + expect(messagesConsumed[messagesConsumed.length - 1]).toEqual( + expect.objectContaining({ + topic: topicName, + partition: 0, + message: expect.objectContaining({ + key: Buffer.from(messages[messages.length - 1].key), + value: Buffer.from(messages[messages.length - 1].value), + offset: '99', + }), + }) + ) + + // check if all offsets are present + expect(messagesConsumed.map(m => m.message.offset)).toEqual(messages.map((_, i) => `${i}`)) + const response = await admin.fetchOffsets({ groupId, topics: [topicName] }) + const { partitions } = response.find(({ topic }) => topic === topicName) + const partition = partitions.find(({ partition }) => partition === 0) + expect(partition.offset).toEqual('100') // check if offsets were committed + }); + + it('stops consuming messages when running = false', async () => { + await consumer.connect(); + await producer.connect(); + await consumer.subscribe({ topic: topicName }); + + let calls = 0; + + consumer.run({ + eachMessage: async event => { + calls++; + await sleep(100); + }, + }); + + const key1 = secureRandom(); + const message1 = { key: `key-${key1}`, value: `value-${key1}` }; + const key2 = secureRandom(); + const message2 = { key: `key-${key2}`, value: `value-${key2}` }; + + await producer.send({ topic: topicName, messages: [message1, message2] }); + await waitFor(() => calls > 0, () => { }, 10); + await consumer.disconnect(); // don't give the consumer the chance to consume the 2nd message + + expect(calls).toEqual(1); + }); + + describe('discarding messages after seeking', () => { + it('stops consuming messages when fetched batch has gone stale', async () => { + consumer = createConsumer({ + groupId, + minBytes: 1024, + maxWaitTimeInMs: 500, + rdKafka: { + topicConfig: { + 'auto.offset.reset': 'earliest', + } + } + }); + + const messages = Array(10) + .fill() + .map(() => { + const value = secureRandom() + return { key: `key-${value}`, value: `value-${value}` } + }); + + await consumer.connect(); + await producer.connect(); + await producer.send({ topic: topicName, messages }); + await consumer.subscribe({ topic: topicName }); + + const offsetsConsumed = []; + + consumer.run({ + eachMessage: async ({ message }) => { + offsetsConsumed.push(message.offset) + + if (offsetsConsumed.length === 1) { + consumer.seek({ topic: topicName, partition: 0, offset: message.offset }); + } + }, + }) + + await waitFor(() => offsetsConsumed.length >= 2, () => { }, { delay: 50 }) + + expect(offsetsConsumed[0]).toEqual(offsetsConsumed[1]) + }); + + /* Skip as it uses eachBatch */ + it.skip('resolves a batch as stale when seek was called while processing it', async () => { + consumer = createConsumer({ + groupId, + // make sure we fetch a batch of messages + minBytes: 1024, + maxWaitTimeInMs: 500, + rdKafka: { + topicConfig: { + 'auto.offset.reset': 'earliest', + } + } + }) + + const messages = Array(10) + .fill() + .map(() => { + const value = secureRandom() + return { key: `key-${value}`, value: `value-${value}` } + }) + + await consumer.connect() + await producer.connect() + await producer.send({ topic: topicName, messages }) + await consumer.subscribe({ topic: topicName }) + + const offsetsConsumed = [] + + consumer.run({ + eachBatch: async ({ batch, isStale, heartbeat, resolveOffset }) => { + for (const message of batch.messages) { + if (isStale()) break + + offsetsConsumed.push(message.offset) + + if (offsetsConsumed.length === 1) { + consumer.seek({ topic: topicName, partition: 0, offset: message.offset }) + } + + resolveOffset(message.offset) + await heartbeat() + } + }, + }) + + await waitFor(() => offsetsConsumed.length >= 2, { delay: 50 }) + + expect(offsetsConsumed[0]).toEqual(offsetsConsumed[1]) + }); + + /* Skip as it uses eachBatch */ + it.skip('skips messages fetched while seek was called', async () => { + consumer = createConsumer({ + cluster: createCluster(), + groupId, + maxWaitTimeInMs: 1000, + logger: newLogger(), + }) + + const messages = Array(10) + .fill() + .map(() => { + const value = secureRandom() + return { key: `key-${value}`, value: `value-${value}` } + }) + await producer.connect() + await producer.send({ topic: topicName, messages }) + + await consumer.connect() + + await consumer.subscribe({ topic: topicName }) + + const offsetsConsumed = [] + + const eachBatch = async ({ batch, heartbeat }) => { + for (const message of batch.messages) { + offsetsConsumed.push(message.offset) + } + + await heartbeat() + } + + consumer.run({ + eachBatch, + }) + + await waitForConsumerToJoinGroup(consumer) + + await waitFor(() => offsetsConsumed.length === messages.length, { delay: 50 }) + await waitForNextEvent(consumer, consumer.events.FETCH_START) + + const seekedOffset = offsetsConsumed[Math.floor(messages.length / 2)] + consumer.seek({ topic: topicName, partition: 0, offset: seekedOffset }) + await producer.send({ topic: topicName, messages }) // trigger completion of fetch + + await waitFor(() => offsetsConsumed.length > messages.length, { delay: 50 }) + + expect(offsetsConsumed[messages.length]).toEqual(seekedOffset) + }); + }); + + /* Skip as it uses eachBatch */ + it.skip('discards messages received when pausing while fetch is in-flight', async () => { + consumer = createConsumer({ + cluster: createCluster(), + groupId, + maxWaitTimeInMs: 200, + logger: newLogger(), + }) + + const messages = Array(10) + .fill() + .map(() => { + const value = secureRandom() + return { key: `key-${value}`, value: `value-${value}` } + }) + await producer.connect() + await producer.send({ topic: topicName, messages }) + + await consumer.connect() + + await consumer.subscribe({ topic: topicName }) + + const offsetsConsumed = [] + + const eachBatch = async ({ batch, heartbeat }) => { + for (const message of batch.messages) { + offsetsConsumed.push(message.offset) + } + + await heartbeat() + } + + consumer.run({ + eachBatch, + }) + + await waitForConsumerToJoinGroup(consumer) + await waitFor(() => offsetsConsumed.length === messages.length, { delay: 50 }) + await waitForNextEvent(consumer, consumer.events.FETCH_START) + + consumer.pause([{ topic: topicName }]) + await producer.send({ topic: topicName, messages }) // trigger completion of fetch + + await waitForNextEvent(consumer, consumer.events.FETCH) + + expect(offsetsConsumed.length).toEqual(messages.length) + }); + + describe('transactions', () => { + it('accepts messages from an idempotent producer', async () => { + producer = createProducer({ + idempotent: true, + maxInFlightRequests: 1, + }) + + consumer = createConsumer({ + groupId, + maxWaitTimeInMs: 100, + rdKafka: { + topicConfig: { + 'auto.offset.reset': 'earliest', + } + } + }); + + await consumer.connect(); + await producer.connect(); + await consumer.subscribe({ topic: topicName }); + + const messagesConsumed = [] + const idempotentMessages = generateMessages({ prefix: 'idempotent' }) + + consumer.run({ + eachMessage: async event => messagesConsumed.push(event), + }) + + await producer.sendBatch({ + topicMessages: [{ topic: topicName, messages: idempotentMessages }], + }) + + const number = idempotentMessages.length; + await waitForMessages(messagesConsumed, { + number, + }); + + expect(messagesConsumed).toHaveLength(idempotentMessages.length) + expect(messagesConsumed[0].message.value.toString()).toMatch(/value-idempotent-0/) + expect(messagesConsumed[99].message.value.toString()).toMatch(/value-idempotent-99/) + }); + + it('accepts messages from committed transactions', async () => { + producer = createProducer({ + transactionalId: `transactional-id-${secureRandom()}`, + maxInFlightRequests: 1, + }); + + consumer = createConsumer({ + groupId, + maxWaitTimeInMs: 100, + rdKafka: { + globalConfig: { + // debug: 'cgrp,topic', + }, + topicConfig: { + 'auto.offset.reset': 'earliest', + } + } + }); + + await consumer.connect(); + await producer.connect(); + await consumer.subscribe({ topic: topicName }); + + const messagesConsumed = []; + + const messages1 = generateMessages({ prefix: 'txn1' }); + const messages2 = generateMessages({ prefix: 'txn2' }); + const nontransactionalMessages1 = generateMessages({ prefix: 'nontransactional1', number: 1 }); + + consumer.run({ + eachMessage: async event => messagesConsumed.push(event), + }); + + // We cannot send non-transaction messages. + await expect(producer.sendBatch({ + topicMessages: [{ topic: topicName, messages: nontransactionalMessages1 }], + })).rejects.toHaveProperty('code', ErrorCodes.ERR__STATE); + + // We can run a transaction + const txn1 = await producer.transaction(); + await txn1.sendBatch({ + topicMessages: [{ topic: topicName, messages: messages1 }], + }); + await txn1.commit(); + + // We can immediately run another transaction + const txn2 = await producer.transaction(); + await txn2.sendBatch({ + topicMessages: [{ topic: topicName, messages: messages2 }], + }); + await txn2.commit(); + + const numMessages = + messages1.length + messages2.length; + + await waitForMessages(messagesConsumed, { + number: numMessages, + }) + + expect(messagesConsumed[0].message.value.toString()).toMatch(/value-txn1-0/) + expect(messagesConsumed[numMessages - 1].message.value.toString()).toMatch(/value-txn2-99/) + }); + + it('does not receive aborted messages', async () => { + producer = createProducer({ + transactionalId: `transactional-id-${secureRandom()}`, + maxInFlightRequests: 1, + }); + + consumer = createConsumer({ + groupId, + maxWaitTimeInMs: 100, + rdKafka: { + topicConfig: { + 'auto.offset.reset': 'earliest', + } + } + }); + + await consumer.connect(); + await producer.connect(); + await consumer.subscribe({ topic: topicName }); + + const messagesConsumed = [] + + const abortedMessages1 = generateMessages({ prefix: 'aborted-txn-1' }); + const abortedMessages2 = generateMessages({ prefix: 'aborted-txn-2' }); + const committedMessages = generateMessages({ prefix: 'committed-txn', number: 10 }); + + consumer.run({ + eachMessage: async event => messagesConsumed.push(event), + }); + + const abortedTxn1 = await producer.transaction(); + await abortedTxn1.sendBatch({ + topicMessages: [{ topic: topicName, messages: abortedMessages1 }], + }); + await abortedTxn1.abort(); + + const abortedTxn2 = await producer.transaction(); + await abortedTxn2.sendBatch({ + topicMessages: [{ topic: topicName, messages: abortedMessages2 }], + }); + await abortedTxn2.abort(); + + const committedTxn = await producer.transaction(); + await committedTxn.sendBatch({ + topicMessages: [{ topic: topicName, messages: committedMessages }], + }); + await committedTxn.commit(); + + const number = committedMessages.length + await waitForMessages(messagesConsumed, { + number, + }); + + expect(messagesConsumed).toHaveLength(number); + expect(messagesConsumed[0].message.value.toString()).toMatch(/value-committed-txn-0/); + expect(messagesConsumed[number - 1].message.value.toString()).toMatch(/value-committed-txn-9/); + }); + + it( + 'receives aborted messages for an isolation level of READ_UNCOMMITTED', + async () => { + producer = createProducer({ + transactionalId: `transactional-id-${secureRandom()}`, + maxInFlightRequests: 1, + }) + + consumer = createConsumer({ + groupId, + maxWaitTimeInMs: 100, + readUncommitted: true, + rdKafka: { + topicConfig: { + 'auto.offset.reset': 'earliest', + } + } + }) + + await consumer.connect(); + await producer.connect(); + await consumer.subscribe({ topic: topicName }); + + const messagesConsumed = []; + + const abortedMessages = generateMessages({ prefix: 'aborted-txn1' }); + + consumer.run({ + eachMessage: async event => messagesConsumed.push(event), + }); + + const abortedTxn1 = await producer.transaction(); + await abortedTxn1.sendBatch({ + topicMessages: [{ topic: topicName, messages: abortedMessages }], + }); + await abortedTxn1.abort(); + + const number = abortedMessages.length; + await waitForMessages(messagesConsumed, { + number, + }); + + expect(messagesConsumed).toHaveLength(abortedMessages.length); + expect(messagesConsumed[0].message.value.toString()).toMatch(/value-aborted-txn1-0/); + expect(messagesConsumed[messagesConsumed.length - 1].message.value.toString()).toMatch( + /value-aborted-txn1-99/ + ); + } + ); + + it( + 'respects offsets sent by a committed transaction ("consume-transform-produce" flow)', + async () => { + // Seed the topic with some messages. We don't need a tx producer for this. + await producer.connect(); + + const partition = 0; + const messages = generateMessages().map(message => ({ + ...message, + partition, + })); + + await producer.send({ + topic: topicName, + messages, + }) + + await producer.disconnect(); + + producer = createProducer({ + transactionalId: `transactional-id-${secureRandom()}`, + maxInFlightRequests: 1, + }) + + consumer = createConsumer({ + groupId, + maxWaitTimeInMs: 100, + rdKafka: { + globalConfig: { + 'enable.auto.commit': false, + }, + topicConfig: { + 'auto.offset.reset': 'earliest', + } + } + }); + + await consumer.connect(); + await producer.connect(); + await consumer.subscribe({ topic: topicName }); + + // 1. Run consumer with "autoCommit=false" + + let messagesConsumed = []; + // This stores the latest offsets consumed for each partition, when we received the ith message. + let uncommittedOffsetsPerMessage = []; + let latestOffsetsPerPartition = {}; + + const eachMessage = async ({ topic, partition, message }) => { + messagesConsumed.push(message) + /* The message.offset indicates current offset, so we need to add 1 to it, since committed offset denotes + * the next offset to consume. */ + latestOffsetsPerPartition[partition] = Number(message.offset) + 1; + uncommittedOffsetsPerMessage.push(Object.assign({}, latestOffsetsPerPartition)); + }; + + consumer.run({ + eachMessage, + }) + + // Consume pre-produced messages. + + const number = messages.length; + await waitForMessages(messagesConsumed, { + number, + }) + + expect(messagesConsumed[0].value.toString()).toMatch(/value-0/) + expect(messagesConsumed[99].value.toString()).toMatch(/value-99/) + expect(uncommittedOffsetsPerMessage).toHaveLength(messagesConsumed.length) + + // 3. Send offsets in a transaction and commit + const txnToCommit = await producer.transaction(); + let offsetsToCommit = uncommittedOffsetsPerMessage[97]; + let topicPartitionOffsets = { topic: topicName, partitions: [] }; + for (const partition in offsetsToCommit) { + topicPartitionOffsets.partitions.push({ partition, offset: offsetsToCommit[partition] }); + } + + await txnToCommit.sendOffsets({ + consumer, + topics: [topicPartitionOffsets], + }); + await txnToCommit.commit(); + + // Restart consumer - we cannot stop it, so we recreate it. + await consumer.disconnect(); + + consumer = createConsumer({ + groupId, + maxWaitTimeInMs: 100, + rdKafka: { + globalConfig: { + 'enable.auto.commit': false, + }, + topicConfig: { + 'auto.offset.reset': 'earliest', + } + } + }); + + await consumer.connect(); + await consumer.subscribe({ topic: topicName }); + + messagesConsumed = [] + uncommittedOffsetsPerMessage = [] + + consumer.run({ eachMessage }) + + // Assert we only consume the messages that were after the sent offset + await waitForMessages(messagesConsumed, { + number: 2, + }) + + expect(messagesConsumed).toHaveLength(2); + expect(messagesConsumed[0].value.toString()).toMatch(/value-98/); + expect(messagesConsumed[1].value.toString()).toMatch(/value-99/); + } + ); + + it( + 'does not respect offsets sent by an aborted transaction ("consume-transform-produce" flow)', + async () => { + // Seed the topic with some messages. We don't need a tx producer for this. + await producer.connect(); + + const partition = 0; + const messages = generateMessages().map(message => ({ + ...message, + partition, + })); + + await producer.send({ + topic: topicName, + messages, + }) + + await producer.disconnect(); + + producer = createProducer({ + transactionalId: `transactional-id-${secureRandom()}`, + maxInFlightRequests: 1, + }) + + consumer = createConsumer({ + groupId, + maxWaitTimeInMs: 100, + rdKafka: { + topicConfig: { + 'auto.offset.reset': 'earliest', + }, + globalConfig: { + 'enable.auto.commit': false, + }, + } + }); + + await consumer.connect(); + await producer.connect(); + await consumer.subscribe({ topic: topicName }); + + // 1. Run consumer with "autoCommit=false" + + let messagesConsumed = []; + // This stores the latest offsets consumed for each partition, when we received the ith message. + let uncommittedOffsetsPerMessage = []; + let latestOffsetsPerPartition = {}; + + const eachMessage = async ({ topic, partition, message }) => { + messagesConsumed.push(message) + /* The message.offset indicates current offset, so we need to add 1 to it, since committed offset denotes + * the next offset to consume. */ + latestOffsetsPerPartition[partition] = Number(message.offset) + 1; + uncommittedOffsetsPerMessage.push(Object.assign({}, latestOffsetsPerPartition)); + }; + + consumer.run({ + eachMessage, + }) + + // Consume produced messages. + await waitForMessages(messagesConsumed, { number: messages.length }); + + // Restart consumer - we cannot stop it, so we recreate it. + await consumer.disconnect(); + + consumer = createConsumer({ + groupId, + maxWaitTimeInMs: 100, + rdKafka: { + globalConfig: { + 'enable.auto.commit': false, + }, + topicConfig: { + 'auto.offset.reset': 'earliest', + } + } + }); + + await consumer.connect(); + await consumer.subscribe({ topic: topicName }); + + expect(messagesConsumed[0].value.toString()).toMatch(/value-0/); + expect(messagesConsumed[99].value.toString()).toMatch(/value-99/); + expect(uncommittedOffsetsPerMessage).toHaveLength(messagesConsumed.length); + + // 3. Send offsets in a transaction and commit + const txnToAbort = await producer.transaction(); + let offsetsToCommit = uncommittedOffsetsPerMessage[97]; + let topicPartitionOffsets = { topic: topicName, partitions: [] }; + for (const partition in offsetsToCommit) { + topicPartitionOffsets.partitions.push({ partition, offset: offsetsToCommit[partition] }); + } + + await txnToAbort.sendOffsets({ + consumer, + topics: [topicPartitionOffsets], + }); + await txnToAbort.abort() + + // Restart consumer + messagesConsumed = [] + uncommittedOffsetsPerMessage = [] + + consumer.run({ + eachMessage + }) + + await waitForMessages(messagesConsumed, { number: 1 }); + expect(messagesConsumed[0].value.toString()).toMatch(/value-0/) + await waitForMessages(messagesConsumed, { number: messages.length }); + expect(messagesConsumed[messagesConsumed.length - 1].value.toString()).toMatch(/value-99/) + } + ); + }); +}); diff --git a/test/promisified/consumer/pause.spec.js b/test/promisified/consumer/pause.spec.js index 6350f7e7..dbe54e90 100644 --- a/test/promisified/consumer/pause.spec.js +++ b/test/promisified/consumer/pause.spec.js @@ -1,3 +1,5 @@ +jest.setTimeout(30000); + const { secureRandom, createTopic, @@ -514,26 +516,27 @@ describe('Consumer', () => { groupId, maxWaitTimeInMs: 100, maxBytesPerPartition: 180, - }) + }); - await producer.connect() - await consumer.connect() + await producer.connect(); + await consumer.connect(); - const [topic1, topic2] = topics + const [topic1, topic2] = topics; await consumer.subscribe({ topics: [topic1, topic2] }); - const eachMessage = jest.fn() - consumer.run({ eachMessage }) + const eachMessage = jest.fn(); + consumer.run({ eachMessage }); - consumer.pause([{ topic: topic1 }, { topic: topic2 }]) + await waitFor(() => consumer.assignment().length > 0, () => { }, { delay: 10 }); + consumer.pause([{ topic: topic1 }, { topic: topic2 }]); - const key1 = secureRandom() - const message1 = { key: `key-${key1}`, value: `value-${key1}`, partition: 0 } + const key1 = secureRandom(); + const message1 = { key: `key-${key1}`, value: `value-${key1}`, partition: 0 }; - await producer.send({ topic: topic1, messages: [message1] }) - await producer.send({ topic: topic2, messages: [message1] }) + await producer.send({ topic: topic1, messages: [message1] }); + await producer.send({ topic: topic2, messages: [message1] }); - expect(eachMessage).not.toHaveBeenCalled() + expect(eachMessage).not.toHaveBeenCalled(); }) }); diff --git a/test/promisified/consumer/subscribe.spec.js b/test/promisified/consumer/subscribe.spec.js index bbbcd483..7c6f8456 100644 --- a/test/promisified/consumer/subscribe.spec.js +++ b/test/promisified/consumer/subscribe.spec.js @@ -1,3 +1,5 @@ +jest.setTimeout(30000); + const { Kafka, ErrorCodes } = require('../../../lib').KafkaJS; const { secureRandom, createTopic, @@ -69,7 +71,7 @@ describe('Consumer', () => { expect(messagesConsumed.map(m => m.message.value.toString())).toEqual( expect.arrayContaining(['drink', 'your', 'ovaltine']) ); - }) + }, 10000); }) describe('Deprecated "topic" interface', () => { @@ -149,7 +151,7 @@ describe('Consumer', () => { 'value-br', 'value-se', ]); - }); + }, 10000); }); }); }); diff --git a/test/promisified/testhelpers.js b/test/promisified/testhelpers.js index 0a8b9362..47bf18fc 100644 --- a/test/promisified/testhelpers.js +++ b/test/promisified/testhelpers.js @@ -55,6 +55,25 @@ async function waitForMessages(messagesConsumed, { number = 1, delay } = {}) { return waitFor(() => messagesConsumed.length >= number, () => messagesConsumed, { delay }); } +async function sleep(ms) { + return new Promise(resolve => setTimeout(resolve, ms)); +} + +const generateMessages = options => { + const { prefix, number = 100 } = options || {} + const prefixOrEmpty = prefix ? `-${prefix}` : '' + + return Array(number) + .fill() + .map((v, i) => { + const value = secureRandom() + return { + key: `key${prefixOrEmpty}-${i}-${value}`, + value: `value${prefixOrEmpty}-${i}-${value}`, + } + }) + } + module.exports = { createConsumer, createProducer, @@ -65,4 +84,6 @@ module.exports = { createTopic, waitForConsumerToJoinGroup, waitFor, + sleep, + generateMessages, } From daa572b526d7c5957237dbbe1dabe711a6bdb956 Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 12 Dec 2023 17:08:56 +0530 Subject: [PATCH 13/26] Make subscriptions replaceable with flag --- lib/kafkajs/_consumer.js | 9 ++++++++- test/promisified/consumer/pause.spec.js | 4 ++-- test/promisified/consumer/subscribe.spec.js | 4 ++-- types/kafkajs.d.ts | 4 ++-- 4 files changed, 14 insertions(+), 7 deletions(-) diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index 3ddedd0d..e5fcdb30 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -73,6 +73,12 @@ class Consumer { */ #pausedPartitions = new Set(); + /** + * Contains a list of stored topics/regexes that the user has subscribed to. + * @type {(string|RegExp)[]} + */ + #storedSubscriptions = []; + /** * @constructor * @param {import("../../types/kafkajs").ConsumerConfig} kJSConfig @@ -425,7 +431,8 @@ class Consumer { } }); - this.#internalClient.subscribe(topics); + this.#storedSubscriptions = subscription.replace ? topics : this.#storedSubscriptions.concat(topics); + this.#internalClient.subscribe(this.#storedSubscriptions); } async stop() { diff --git a/test/promisified/consumer/pause.spec.js b/test/promisified/consumer/pause.spec.js index dbe54e90..b563ce0b 100644 --- a/test/promisified/consumer/pause.spec.js +++ b/test/promisified/consumer/pause.spec.js @@ -68,8 +68,8 @@ describe('Consumer', () => { /* Send the first 2 messages to each topic. */ for (const topic of topics) { await producer.send({ topic, messages: messages.slice(0, 2) }); + await consumer.subscribe({ topic }); } - await consumer.subscribe({ topics: topics }); let shouldPause = true; let pauseMessageRecvd = false; @@ -141,7 +141,7 @@ describe('Consumer', () => { for (const topic of topics) { await producer.send({ topic, messages: messages.slice(0, 2) }) } - await consumer.subscribe({ topics }) + await consumer.subscribe({ topics, replace: true }); let shouldPause = true const messagesConsumed = [] diff --git a/test/promisified/consumer/subscribe.spec.js b/test/promisified/consumer/subscribe.spec.js index 7c6f8456..30493c82 100644 --- a/test/promisified/consumer/subscribe.spec.js +++ b/test/promisified/consumer/subscribe.spec.js @@ -71,7 +71,7 @@ describe('Consumer', () => { expect(messagesConsumed.map(m => m.message.value.toString())).toEqual( expect.arrayContaining(['drink', 'your', 'ovaltine']) ); - }, 10000); + }); }) describe('Deprecated "topic" interface', () => { @@ -151,7 +151,7 @@ describe('Consumer', () => { 'value-br', 'value-se', ]); - }, 10000); + }); }); }); }); diff --git a/types/kafkajs.d.ts b/types/kafkajs.d.ts index 59aa5efc..30697952 100644 --- a/types/kafkajs.d.ts +++ b/types/kafkajs.d.ts @@ -376,9 +376,9 @@ export type EachBatchHandler = (payload: EachBatchPayload) => Promise export type EachMessageHandler = (payload: EachMessagePayload) => Promise -export type ConsumerSubscribeTopic = { topic: string | RegExp; fromBeginning?: boolean } +export type ConsumerSubscribeTopic = { topic: string | RegExp; fromBeginning?: boolean, replace?: boolean } -export type ConsumerSubscribeTopics = { topics: (string | RegExp)[]; fromBeginning?: boolean } +export type ConsumerSubscribeTopics = { topics: (string | RegExp)[]; fromBeginning?: boolean, replace?: boolean } export type ConsumerRunConfig = { autoCommit?: boolean From 11aa5e4a8dc73f23a1eb3393a9920ac2bf2dee10 Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 12 Dec 2023 18:35:51 +0530 Subject: [PATCH 14/26] Add producer tests and changes --- MIGRATION.md | 4 +- lib/kafkajs/_consumer.js | 8 +- .../producer/concurrentTransaction.spec.js | 49 ++++ .../producer/idempotentProduceMessage.spec.js | 211 ++++++++++++++++++ .../producer/producingToInvalidTopic.spec.js | 30 +++ 5 files changed, 300 insertions(+), 2 deletions(-) create mode 100644 test/promisified/producer/concurrentTransaction.spec.js create mode 100644 test/promisified/producer/idempotentProduceMessage.spec.js create mode 100644 test/promisified/producer/producingToInvalidTopic.spec.js diff --git a/MIGRATION.md b/MIGRATION.md index 3c5e7f6d..be3cfbb8 100644 --- a/MIGRATION.md +++ b/MIGRATION.md @@ -220,9 +220,11 @@ #### Semantic and Per-Method Changes - * Changes to subscribe: + * Changes to `subscribe`: * Regex flags are ignored while passing a topic subscription (like 'i' or 'g'). * Subscribe must be called after `connect`. + * An optional parameter, `replace` is provided. If set to true, the current subscription is replaced with the new one. If set to false, the new subscription is added to the current one. + The default value is false. * While passing a list of topics to `subscribe`, the `fromBeginning` property is not supported. Instead, the property `auto.offset.reset` needs to be used. Before: ```javascript diff --git a/lib/kafkajs/_consumer.js b/lib/kafkajs/_consumer.js index e5fcdb30..769ef3c3 100644 --- a/lib/kafkajs/_consumer.js +++ b/lib/kafkajs/_consumer.js @@ -440,7 +440,7 @@ class Consumer { } /** - * Starts consumer polling. + * Starts consumer polling. This method returns immediately. * @param {import("../../types/kafkajs").ConsumerRunConfig} config */ async run(config) { @@ -454,6 +454,12 @@ class Consumer { { code: error.ErrorCodes.ERR__NOT_IMPLEMENTED }); } + /* We deliberately don't await this. */ + this.#runInternal(config); + } + + /* Internal polling loop. It accepts the same config object that `run` accepts. */ + async #runInternal(config) { while (this.#state === ConsumerState.CONNECTED) { const m = await this.#consumeSingle(); diff --git a/test/promisified/producer/concurrentTransaction.spec.js b/test/promisified/producer/concurrentTransaction.spec.js new file mode 100644 index 00000000..09ca0175 --- /dev/null +++ b/test/promisified/producer/concurrentTransaction.spec.js @@ -0,0 +1,49 @@ +const { + secureRandom, + createProducer, + createTopic, +} = require('../testhelpers') + +describe('Producer > Transactional producer', () => { + let producer1, producer2, topicName, transactionalId, message; + + const newProducer = () => + createProducer({ + idempotent: true, + transactionalId, + transactionTimeout: 1000, + }); + + beforeEach(async () => { + topicName = `test-topic-${secureRandom()}`; + transactionalId = `transactional-id-${secureRandom()}`; + message = { key: `key-${secureRandom()}`, value: `value-${secureRandom()}` }; + + await createTopic({ topic: topicName }); + }) + + afterEach(async () => { + producer1 && (await producer1.disconnect()); + producer2 && (await producer2.disconnect()); + }) + + describe('when there is an ongoing transaction on connect', () => { + it('retries initProducerId to cancel the ongoing transaction', + async () => { + // Producer 1 will create a transaction and "crash", it will never commit or abort the connection + producer1 = newProducer(); + await producer1.connect(); + const transaction1 = await producer1.transaction(); + await transaction1.send({ topic: topicName, messages: [message] }); + + // Producer 2 starts with the same transactional id to cause the concurrent transactions error + producer2 = newProducer(); + await producer2.connect(); + let transaction2; + await expect(producer2.transaction().then(t => (transaction2 = t))).resolves.toBeTruthy(); + await transaction2.send({ topic: topicName, messages: [message] }); + await transaction2.commit(); + } + ) + }) +}) diff --git a/test/promisified/producer/idempotentProduceMessage.spec.js b/test/promisified/producer/idempotentProduceMessage.spec.js new file mode 100644 index 00000000..3ef6b355 --- /dev/null +++ b/test/promisified/producer/idempotentProduceMessage.spec.js @@ -0,0 +1,211 @@ +jest.setTimeout(10000) + +const { + secureRandom, + createTopic, + waitForMessages, + createProducer, + createConsumer, +} = require('../testhelpers'); +const { KafkaJSError } = require('../../../lib').KafkaJS; + +describe('Producer > Idempotent producer', () => { + let producer, consumer, topicName, cluster, messages; + + beforeAll(async () => { + messages = Array(4) + .fill() + .map((_, i) => { + const value = secureRandom() + return { key: `key-${value}`, value: `${i}` } + }) + }) + + beforeEach(async () => { + topicName = `test-topic-${secureRandom()}`; + producer = createProducer({ + idempotent: true, + }) + consumer = createConsumer({ + groupId: `consumer-group-id-${secureRandom()}`, + maxWaitTimeInMs: 0, + rdKafka: { + topicConfig: { + 'auto.offset.reset': 'earliest', + }, + } + }) + await createTopic({ topic: topicName, partitions: 1 }); + await Promise.all([producer.connect(), consumer.connect()]); + await consumer.subscribe({ topic: topicName }); + }); + + afterEach( + async () => + await Promise.all([ + producer && (await producer.disconnect()), + consumer && (await consumer.disconnect()), + ]) + ); + + it('sequential produce() calls > all messages are written to the partition once, in order', async () => { + const messagesConsumed = []; + + for (const m of messages) { + await producer.send({ topic: topicName, messages: [m] }) + } + + await consumer.run({ eachMessage: async message => messagesConsumed.push(message) }); + await waitForMessages(messagesConsumed, { number: messages.length }); + + messagesConsumed.forEach(({ message: { value } }, i) => + expect(value.toString()).toEqual(`${i}`) + ); + }); + + /* Skip as we don't have the mock broker available */ + it.skip('sequential produce() calls > where produce() throws a retriable error, all messages are written to the partition once, in order', async () => { + for (const nodeId of [0, 1, 2]) { + const broker = await cluster.findBroker({ nodeId }) + + const brokerProduce = jest.spyOn(broker, 'produce') + brokerProduce.mockImplementationOnce(() => { + throw new KafkaJSError('retriable error') + }) + } + + const messagesConsumed = [] + + for (const m of messages) { + await producer.send({ acks: -1, topic: topicName, messages: [m] }) + } + + await consumer.run({ eachMessage: async message => messagesConsumed.push(message) }) + + await waitForMessages(messagesConsumed, { number: messages.length }) + + messagesConsumed.forEach(({ message: { value } }, i) => + expect(value.toString()).toEqual(`${i}`) + ) + }); + + /* Skip as we don't have the mock broker available */ + it.skip('sequential produce() calls > where produce() throws a retriable error after the message is written to the log, all messages are written to the partition once, in order', async () => { + for (const nodeId of [0, 1, 2]) { + const broker = await cluster.findBroker({ nodeId }) + const originalCall = broker.produce.bind(broker) + const brokerProduce = jest.spyOn(broker, 'produce') + brokerProduce.mockImplementationOnce() + brokerProduce.mockImplementationOnce() + brokerProduce.mockImplementationOnce(async (...args) => { + await originalCall(...args) + throw new KafkaJSError('retriable error') + }) + } + + const messagesConsumed = [] + + for (const m of messages) { + await producer.send({ acks: -1, topic: topicName, messages: [m] }) + } + + await consumer.run({ eachMessage: async message => messagesConsumed.push(message) }) + + await waitForMessages(messagesConsumed, { number: messages.length }) + + messagesConsumed.forEach(({ message: { value } }, i) => + expect(value.toString()).toEqual(`${i}`) + ) + }) + + it('concurrent produce() calls > all messages are written to the partition once', async () => { + const messagesConsumed = [] + + await Promise.all( + messages.map(m => producer.send({ topic: topicName, messages: [m] })) + ) + + await consumer.run({ eachMessage: async message => messagesConsumed.push(message) }) + + await waitForMessages(messagesConsumed, { number: messages.length }) + expect(messagesConsumed).toHaveLength(messages.length) + }); + + /* Skip as we don't have the mock broker available */ + it.skip('concurrent produce() calls > where produce() throws a retriable error on the first call, all messages are written to the partition once', async () => { + for (const nodeId of [0, 1, 2]) { + const broker = await cluster.findBroker({ nodeId }) + + const brokerProduce = jest.spyOn(broker, 'produce') + brokerProduce.mockImplementationOnce(async () => { + throw new KafkaJSError('retriable error') + }) + } + + await Promise.allSettled( + messages.map(m => producer.send({ acks: -1, topic: topicName, messages: [m] })) + ) + + const messagesConsumed = [] + await consumer.run({ eachMessage: async message => messagesConsumed.push(message) }) + + await waitForMessages(messagesConsumed, { number: messages.length }) + + expect(arrayUnique(messagesConsumed.map(({ message: { value } }) => value))).toHaveLength( + messages.length + ) + }) + + /* Skip as we don't have the mock broker available */ + it.skip('concurrent produce() calls > where produce() throws a retriable error on 2nd call, all messages are written to the partition once', async () => { + for (const nodeId of [0, 1, 2]) { + const broker = await cluster.findBroker({ nodeId }) + + const brokerProduce = jest.spyOn(broker, 'produce') + brokerProduce.mockImplementationOnce() + brokerProduce.mockImplementationOnce(async () => { + throw new KafkaJSError('retriable error') + }) + } + + await Promise.allSettled( + messages.map(m => producer.send({ acks: -1, topic: topicName, messages: [m] })) + ) + + const messagesConsumed = [] + await consumer.run({ eachMessage: async message => messagesConsumed.push(message) }) + + await waitForMessages(messagesConsumed, { number: messages.length }) + + expect(arrayUnique(messagesConsumed.map(({ message: { value } }) => value))).toHaveLength( + messages.length + ) + }) + + /* Skip as we don't have the mock broker available */ + it.skip('concurrent produce() calls > where produce() throws a retriable error after the message is written to the log, all messages are written to the partition once', async () => { + for (const nodeId of [0, 1, 2]) { + const broker = await cluster.findBroker({ nodeId }) + const originalCall = broker.produce.bind(broker) + const brokerProduce = jest.spyOn(broker, 'produce') + brokerProduce.mockImplementationOnce(async (...args) => { + await originalCall(...args) + throw new KafkaJSError('retriable error') + }) + } + + const messagesConsumed = [] + + await Promise.all( + messages.map(m => producer.send({ acks: -1, topic: topicName, messages: [m] })) + ) + + await consumer.run({ eachMessage: async message => messagesConsumed.push(message) }) + + await waitForMessages(messagesConsumed, { number: messages.length }) + + expect(arrayUnique(messagesConsumed.map(({ message: { value } }) => value))).toHaveLength( + messages.length + ) + }) +}) diff --git a/test/promisified/producer/producingToInvalidTopic.spec.js b/test/promisified/producer/producingToInvalidTopic.spec.js new file mode 100644 index 00000000..ad624661 --- /dev/null +++ b/test/promisified/producer/producingToInvalidTopic.spec.js @@ -0,0 +1,30 @@ +const { createTopic, createProducer, secureRandom } = require('../testhelpers'); +const { ErrorCodes } = require('../../../lib').KafkaJS; + +describe('Producer > Producing to invalid topics', () => { + let producer, topicName; + + beforeEach(async () => { + topicName = `test-topic-${secureRandom()}` + + producer = createProducer({ + }) + await producer.connect(); + await createTopic({ topic: topicName }); + }) + + afterEach(async () => { + producer && (await producer.disconnect()) + }) + + it('it rejects when producing to an invalid topic name, but is able to subsequently produce to a valid topic', async () => { + const message = { key: `key-${secureRandom()}`, value: `value-${secureRandom()}` }; + const invalidTopicName = `${topicName}-abc)(*&^%`; + await expect(producer.send({ topic: invalidTopicName, messages: [message] })).rejects.toHaveProperty( + 'code', + ErrorCodes.ERR_TOPIC_EXCEPTION, + ); + + await expect(producer.send({ topic: topicName, messages: [message] })).resolves.toBeTruthy(); + }); +}) From 0b43e52c6a6e250f3259325c690e9895f4250f2e Mon Sep 17 00:00:00 2001 From: Milind L Date: Thu, 30 Nov 2023 14:21:09 +0530 Subject: [PATCH 15/26] Add semaphore block to project (#5) --- service.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/service.yml b/service.yml index 9dc1749a..760002d2 100644 --- a/service.yml +++ b/service.yml @@ -7,3 +7,5 @@ github: enable: true codeowners: enable: true +semaphore: + enable: true From 4493422d901589c57144b0a353308ba451180abf Mon Sep 17 00:00:00 2001 From: Confluent Jenkins Bot Date: Thu, 30 Nov 2023 02:16:10 -0800 Subject: [PATCH 16/26] chore: update repo by service bot (#6) * chore: update repo semaphore project * chore: update repo semaphore config --- .semaphore/project.yml | 44 ++++++++++++++++++++++++++++ .semaphore/semaphore.yml | 62 ++++++++++++++++++++++++++++++++++++++++ 2 files changed, 106 insertions(+) create mode 100644 .semaphore/project.yml create mode 100644 .semaphore/semaphore.yml diff --git a/.semaphore/project.yml b/.semaphore/project.yml new file mode 100644 index 00000000..e20c3839 --- /dev/null +++ b/.semaphore/project.yml @@ -0,0 +1,44 @@ +# This file is managed by ServiceBot plugin - Semaphore. The content in this file is created using a common +# template and configurations in service.yml. +# Modifications in this file will be overwritten by generated content in the nightly run. +# For more information, please refer to the page: +# https://confluentinc.atlassian.net/wiki/spaces/Foundations/pages/2871296194/Add+SemaphoreCI +apiVersion: v1alpha +kind: Project +metadata: + name: confluent-kafka-js + description: "" +spec: + visibility: private + repository: + url: git@github.com:confluentinc/confluent-kafka-js.git + run_on: + - branches + - tags + - pull_requests + pipeline_file: .semaphore/semaphore.yml + integration_type: github_app + status: + pipeline_files: + - path: .semaphore/semaphore.yml + level: pipeline + whitelist: + branches: + - master + - main + - /^v\d+\.\d+\.x$/ + - /^gh-readonly-queue.*/ + custom_permissions: true + debug_permissions: + - empty + - default_branch + - non_default_branch + - pull_request + - forked_pull_request + - tag + attach_permissions: + - default_branch + - non_default_branch + - pull_request + - forked_pull_request + - tag diff --git a/.semaphore/semaphore.yml b/.semaphore/semaphore.yml new file mode 100644 index 00000000..044c8f09 --- /dev/null +++ b/.semaphore/semaphore.yml @@ -0,0 +1,62 @@ +# This file is managed by ServiceBot plugin - Semaphore. The content in this file is created using a common +# template and configurations in service.yml. +# Any modifications made to version, name, agent, and global_job_config will be overwritten by the generated +# content in nightly runs. Code changes made to the remaining sections will not be affected. +# For more information, please refer to the page: +# https://confluentinc.atlassian.net/wiki/spaces/Foundations/pages/2871296194/Add+SemaphoreCI +version: v1.0 +name: build-test-release +agent: + machine: + type: s1-prod-ubuntu20-04-amd64-1 + +auto_cancel: + running: + when: "branch != 'master'" + +execution_time_limit: + hours: 1 + +queue: + - when: "branch != 'master'" + processing: parallel + +global_job_config: + prologue: + commands: + - checkout + - make show-args + - . vault-setup + - . vault-sem-get-secret cpd_gcloud + - . vault-sem-get-secret ci-reporting + - . vault-sem-get-secret v1/ci/kv/service-foundations/cc-mk-include + - make init-ci + epilogue: + always: + commands: + - make epilogue-ci + +blocks: + - name: "Build, Test, Release" + run: + # don't run the build or unit tests on non-functional changes... + when: "change_in('/', {exclude: ['/.deployed-versions/', '.github/']})" + task: + # You can customize your CI job here +# env_vars: +# # custom env_vars +# prologue: +# commands: +# # custom vault secrets +# # custom prologue commands + jobs: + - name: "Build, Test, Release" + commands: + - make build + - make test + - make release-ci + epilogue: + always: + commands: + - make epilogue-ci + - make testbreak-after From 5470c709e2c844cc7d35980104104e4e37f74108 Mon Sep 17 00:00:00 2001 From: Milind L Date: Thu, 30 Nov 2023 20:10:10 +0530 Subject: [PATCH 17/26] Set up basic semaphore --- .eslintrc.js | 26 ++ .semaphore/semaphore.yml | 48 +-- package-lock.json | 786 ++++++++++++++++++++++++++++++++++++++- package.json | 1 + 4 files changed, 819 insertions(+), 42 deletions(-) create mode 100644 .eslintrc.js diff --git a/.eslintrc.js b/.eslintrc.js new file mode 100644 index 00000000..120e6804 --- /dev/null +++ b/.eslintrc.js @@ -0,0 +1,26 @@ +module.exports = { + "env": { + "browser": true, + "commonjs": true, + "es2021": true + }, + "extends": "eslint:recommended", + "overrides": [ + { + "env": { + "node": true + }, + "files": [ + ".eslintrc.{js,cjs}" + ], + "parserOptions": { + "sourceType": "script" + } + } + ], + "parserOptions": { + "ecmaVersion": "latest" + }, + "rules": { + } +} diff --git a/.semaphore/semaphore.yml b/.semaphore/semaphore.yml index 044c8f09..08d3ad9b 100644 --- a/.semaphore/semaphore.yml +++ b/.semaphore/semaphore.yml @@ -25,38 +25,30 @@ global_job_config: prologue: commands: - checkout - - make show-args - - . vault-setup - - . vault-sem-get-secret cpd_gcloud - - . vault-sem-get-secret ci-reporting - - . vault-sem-get-secret v1/ci/kv/service-foundations/cc-mk-include - - make init-ci - epilogue: - always: - commands: - - make epilogue-ci blocks: - - name: "Build, Test, Release" - run: - # don't run the build or unit tests on non-functional changes... - when: "change_in('/', {exclude: ['/.deployed-versions/', '.github/']})" + - name: "Linux amd64" + dependencies: [ ] task: - # You can customize your CI job here -# env_vars: -# # custom env_vars -# prologue: -# commands: -# # custom vault secrets -# # custom prologue commands + agent: + machine: + type: s1-prod-ubuntu20-04-amd64-2 + prologue: + commands: + - sem-version node 18.19.0 + - node --version + - npm --version + - npx --version + - sudo apt-get update -q + - git submodule update --init --recursive jobs: - - name: "Build, Test, Release" + - name: "Build from source and test" commands: - - make build + - sudo apt install -y libcurl4-openssl-dev libcrypto++-dev libssl-dev libzstd-dev + - npm install # this will actually not build anything, but rather, fetch things using node-pre-gyp - so change this later. - make test - - make release-ci - epilogue: - always: + - name: "ESLint" commands: - - make epilogue-ci - - make testbreak-after + - npm install --only=dev + - npx eslint lib/kafkajs + diff --git a/package-lock.json b/package-lock.json index 53717a2d..689ecaca 100644 --- a/package-lock.json +++ b/package-lock.json @@ -17,6 +17,7 @@ "devDependencies": { "@types/node": "^20.4.5", "bluebird": "^3.5.3", + "eslint": "8.54.0", "jest": "^29.7.0", "jsdoc": "^3.4.0", "jshint": "^2.10.1", @@ -29,6 +30,15 @@ "node": ">=14.0.0" } }, + "node_modules/@aashutoshrathi/word-wrap": { + "version": "1.2.6", + "resolved": "https://registry.npmjs.org/@aashutoshrathi/word-wrap/-/word-wrap-1.2.6.tgz", + "integrity": "sha512-1Yjs2SvM8TflER/OD3cOjhWWOZb58A2t7wpE2S9XfBYTiIl+XFhQG2bjy4Pu1I+EAlCNUzRDYDdFwFYUKvXcIA==", + "dev": true, + "engines": { + "node": ">=0.10.0" + } + }, "node_modules/@ampproject/remapping": { "version": "2.2.1", "resolved": "https://registry.npmjs.org/@ampproject/remapping/-/remapping-2.2.1.tgz", @@ -682,12 +692,146 @@ "integrity": "sha512-0hYQ8SB4Db5zvZB4axdMHGwEaQjkZzFjQiN9LVYvIFB2nSUHW9tYpxWriPrWDASIxiaXax83REcLxuSdnGPZtw==", "dev": true }, + "node_modules/@eslint-community/eslint-utils": { + "version": "4.4.0", + "resolved": "https://registry.npmjs.org/@eslint-community/eslint-utils/-/eslint-utils-4.4.0.tgz", + "integrity": "sha512-1/sA4dwrzBAyeUoQ6oxahHKmrZvsnLCg4RfxW3ZFGGmQkSNQPFNLV9CUEFQP1x9EYXHTo5p6xdhZM1Ne9p/AfA==", + "dev": true, + "dependencies": { + "eslint-visitor-keys": "^3.3.0" + }, + "engines": { + "node": "^12.22.0 || ^14.17.0 || >=16.0.0" + }, + "peerDependencies": { + "eslint": "^6.0.0 || ^7.0.0 || >=8.0.0" + } + }, + "node_modules/@eslint-community/regexpp": { + "version": "4.10.0", + "resolved": "https://registry.npmjs.org/@eslint-community/regexpp/-/regexpp-4.10.0.tgz", + "integrity": "sha512-Cu96Sd2By9mCNTx2iyKOmq10v22jUVQv0lQnlGNy16oE9589yE+QADPbrMGCkA51cKZSg3Pu/aTJVTGfL/qjUA==", + "dev": true, + "engines": { + "node": "^12.0.0 || ^14.0.0 || >=16.0.0" + } + }, + "node_modules/@eslint/eslintrc": { + "version": "2.1.4", + "resolved": "https://registry.npmjs.org/@eslint/eslintrc/-/eslintrc-2.1.4.tgz", + "integrity": "sha512-269Z39MS6wVJtsoUl10L60WdkhJVdPG24Q4eZTH3nnF6lpvSShEK3wQjDX9JRWAUPvPh7COouPpU9IrqaZFvtQ==", + "dev": true, + "dependencies": { + "ajv": "^6.12.4", + "debug": "^4.3.2", + "espree": "^9.6.0", + "globals": "^13.19.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" + }, + "engines": { + "node": "^12.22.0 || ^14.17.0 || >=16.0.0" + }, + "funding": { + "url": "https://opencollective.com/eslint" + } + }, + "node_modules/@eslint/eslintrc/node_modules/argparse": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/argparse/-/argparse-2.0.1.tgz", + "integrity": "sha512-8+9WqebbFzpX9OR+Wa6O29asIogeRMzcGtAINdpMHHyAg10f05aSFVBbcEqGf/PXw1EjAZ+q2/bEBg3DvurK3Q==", + "dev": true + }, + "node_modules/@eslint/eslintrc/node_modules/globals": { + "version": "13.24.0", + "resolved": "https://registry.npmjs.org/globals/-/globals-13.24.0.tgz", + "integrity": "sha512-AhO5QUcj8llrbG09iWhPU2B204J1xnPeL8kQmVorSsy+Sjj1sk8gIyh6cUocGmH4L0UuhAJy+hJMRA4mgA4mFQ==", + "dev": true, + "dependencies": { + "type-fest": "^0.20.2" + }, + "engines": { + "node": ">=8" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/@eslint/eslintrc/node_modules/js-yaml": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/js-yaml/-/js-yaml-4.1.0.tgz", + "integrity": "sha512-wpxZs9NoxZaJESJGIZTyDEaYpl0FKSA+FB9aJiyemKhMwkxQg63h4T1KJgUGHpTqPDNRcmmYLugrRjJlBtWvRA==", + "dev": true, + "dependencies": { + "argparse": "^2.0.1" + }, + "bin": { + "js-yaml": "bin/js-yaml.js" + } + }, + "node_modules/@eslint/eslintrc/node_modules/type-fest": { + "version": "0.20.2", + "resolved": "https://registry.npmjs.org/type-fest/-/type-fest-0.20.2.tgz", + "integrity": "sha512-Ne+eE4r0/iWnpAxD852z3A+N0Bt5RN//NjJwRd2VFHEmrywxf5vsZlh4R6lixl6B+wz/8d+maTSAkN1FIkI3LQ==", + "dev": true, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/@eslint/js": { + "version": "8.54.0", + "resolved": "https://registry.npmjs.org/@eslint/js/-/js-8.54.0.tgz", + "integrity": "sha512-ut5V+D+fOoWPgGGNj83GGjnntO39xDy6DWxO0wb7Jp3DcMX0TfIqdzHF85VTQkerdyGmuuMD9AKAo5KiNlf/AQ==", + "dev": true, + "engines": { + "node": "^12.22.0 || ^14.17.0 || >=16.0.0" + } + }, "node_modules/@gar/promisify": { "version": "1.1.3", "resolved": "https://registry.npmjs.org/@gar/promisify/-/promisify-1.1.3.tgz", "integrity": "sha512-k2Ty1JcVojjJFwrg/ThKi2ujJ7XNLYaFGNB/bWT9wGR+oSMJHMa5w+CUq6p/pVrKeNNgA7pCqEcjSnHVoqJQFw==", "dev": true }, + "node_modules/@humanwhocodes/config-array": { + "version": "0.11.13", + "resolved": "https://registry.npmjs.org/@humanwhocodes/config-array/-/config-array-0.11.13.tgz", + "integrity": "sha512-JSBDMiDKSzQVngfRjOdFXgFfklaXI4K9nLF49Auh21lmBWRLIK3+xTErTWD4KU54pb6coM6ESE7Awz/FNU3zgQ==", + "dev": true, + "dependencies": { + "@humanwhocodes/object-schema": "^2.0.1", + "debug": "^4.1.1", + "minimatch": "^3.0.5" + }, + "engines": { + "node": ">=10.10.0" + } + }, + "node_modules/@humanwhocodes/module-importer": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/@humanwhocodes/module-importer/-/module-importer-1.0.1.tgz", + "integrity": "sha512-bxveV4V8v5Yb4ncFTT3rPSgZBOpCkjfK0y4oVVVJwIuDVBRMDXrPyXRL988i5ap9m9bnyEEjWfm5WkBmtffLfA==", + "dev": true, + "engines": { + "node": ">=12.22" + }, + "funding": { + "type": "github", + "url": "https://github.com/sponsors/nzakas" + } + }, + "node_modules/@humanwhocodes/object-schema": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/@humanwhocodes/object-schema/-/object-schema-2.0.1.tgz", + "integrity": "sha512-dvuCeX5fC9dXgJn9t+X5atfmgQAzUOWqS1254Gh0m6i8wKd10ebXkfNKiRK+1GWi/yTvvLDHpoxLr0xxxeslWw==", + "dev": true + }, "node_modules/@istanbuljs/load-nyc-config": { "version": "1.1.0", "resolved": "https://registry.npmjs.org/@istanbuljs/load-nyc-config/-/load-nyc-config-1.1.0.tgz", @@ -1058,6 +1202,41 @@ "node-pre-gyp": "bin/node-pre-gyp" } }, + "node_modules/@nodelib/fs.scandir": { + "version": "2.1.5", + "resolved": "https://registry.npmjs.org/@nodelib/fs.scandir/-/fs.scandir-2.1.5.tgz", + "integrity": "sha512-vq24Bq3ym5HEQm2NKCr3yXDwjc7vTsEThRDnkp2DK9p1uqLR+DHurm/NOTo0KG7HYHU7eppKZj3MyqYuMBf62g==", + "dev": true, + "dependencies": { + "@nodelib/fs.stat": "2.0.5", + "run-parallel": "^1.1.9" + }, + "engines": { + "node": ">= 8" + } + }, + "node_modules/@nodelib/fs.stat": { + "version": "2.0.5", + "resolved": "https://registry.npmjs.org/@nodelib/fs.stat/-/fs.stat-2.0.5.tgz", + "integrity": "sha512-RkhPPp2zrqDAQA/2jNhnztcPAlv64XdhIp7a7454A5ovI7Bukxgt7MX7udwAu3zg1DcpPU0rz3VV1SeaqvY4+A==", + "dev": true, + "engines": { + "node": ">= 8" + } + }, + "node_modules/@nodelib/fs.walk": { + "version": "1.2.8", + "resolved": "https://registry.npmjs.org/@nodelib/fs.walk/-/fs.walk-1.2.8.tgz", + "integrity": "sha512-oGB+UxlgWcgQkgwo8GcEGwemoTFt3FIO9ababBmaGwXIoBKZ+GTy0pP185beGg7Llih/NSHSV2XAs1lnznocSg==", + "dev": true, + "dependencies": { + "@nodelib/fs.scandir": "2.1.5", + "fastq": "^1.6.0" + }, + "engines": { + "node": ">= 8" + } + }, "node_modules/@npmcli/fs": { "version": "2.1.2", "resolved": "https://registry.npmjs.org/@npmcli/fs/-/fs-2.1.2.tgz", @@ -1244,11 +1423,38 @@ "integrity": "sha512-I4q9QU9MQv4oEOz4tAHJtNz1cwuLxn2F3xcc2iV5WdqLPpUnj30aUuxt1mAxYTG+oe8CZMV/+6rU4S4gRDzqtQ==", "dev": true }, + "node_modules/@ungap/structured-clone": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/@ungap/structured-clone/-/structured-clone-1.2.0.tgz", + "integrity": "sha512-zuVdFrMJiuCDQUMCzQaD6KL28MjnqqN8XnAqiEq9PNm/hCPTSGfrXCOfwj1ow4LFb/tNymJPwsNbVePc1xFqrQ==", + "dev": true + }, "node_modules/abbrev": { "version": "1.1.1", "resolved": "https://registry.npmjs.org/abbrev/-/abbrev-1.1.1.tgz", "integrity": "sha512-nne9/IiQ/hzIhY6pdDnbBtz7DjPTKrY00P/zvPSm5pOFkl6xuGrGnXn/VtTNNfNtAfZ9/1RtehkszU9qcTii0Q==" }, + "node_modules/acorn": { + "version": "8.11.2", + "resolved": "https://registry.npmjs.org/acorn/-/acorn-8.11.2.tgz", + "integrity": "sha512-nc0Axzp/0FILLEVsm4fNwLCwMttvhEI263QtVPQcbpfZZ3ts0hLsZGOpE6czNlid7CJ9MlyH8reXkpsf3YUY4w==", + "dev": true, + "bin": { + "acorn": "bin/acorn" + }, + "engines": { + "node": ">=0.4.0" + } + }, + "node_modules/acorn-jsx": { + "version": "5.3.2", + "resolved": "https://registry.npmjs.org/acorn-jsx/-/acorn-jsx-5.3.2.tgz", + "integrity": "sha512-rq9s+JNhf0IChjtDXxllJ7g41oZk5SlXtp0LHwyA5cejwn7vKmKp4pPri6YEePv2PU65sAsegbXtIinmDFDXgQ==", + "dev": true, + "peerDependencies": { + "acorn": "^6.0.0 || ^7.0.0 || ^8.0.0" + } + }, "node_modules/agent-base": { "version": "6.0.2", "resolved": "https://registry.npmjs.org/agent-base/-/agent-base-6.0.2.tgz", @@ -1285,6 +1491,22 @@ "node": ">=8" } }, + "node_modules/ajv": { + "version": "6.12.6", + "resolved": "https://registry.npmjs.org/ajv/-/ajv-6.12.6.tgz", + "integrity": "sha512-j3fVLgvTo527anyYyJOGTYJbG+vnnQYvE0m5mmkc1TK+nxAppkCLMIL0aZ4dblVCNoGShhm+kzE4ZUykBoMg4g==", + "dev": true, + "dependencies": { + "fast-deep-equal": "^3.1.1", + "fast-json-stable-stringify": "^2.0.0", + "json-schema-traverse": "^0.4.1", + "uri-js": "^4.2.2" + }, + "funding": { + "type": "github", + "url": "https://github.com/sponsors/epoberezkin" + } + }, "node_modules/ansi-colors": { "version": "4.1.1", "resolved": "https://registry.npmjs.org/ansi-colors/-/ansi-colors-4.1.1.tgz", @@ -2011,6 +2233,12 @@ } } }, + "node_modules/deep-is": { + "version": "0.1.4", + "resolved": "https://registry.npmjs.org/deep-is/-/deep-is-0.1.4.tgz", + "integrity": "sha512-oIPzksmTg4/MriiaYGO+okXDT7ztn/w3Eptv/+gSIdMdKsJo0u4CfYNFJPy+4SKMuCqGw2wxnA+URMg3t8a/bQ==", + "dev": true + }, "node_modules/deepmerge": { "version": "4.3.1", "resolved": "https://registry.npmjs.org/deepmerge/-/deepmerge-4.3.1.tgz", @@ -2060,6 +2288,18 @@ "node": "^14.15.0 || ^16.10.0 || >=18.0.0" } }, + "node_modules/doctrine": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/doctrine/-/doctrine-3.0.0.tgz", + "integrity": "sha512-yS+Q5i3hBf7GBkd4KG8a7eBNNWNGLTaEwwYWUijIYM7zrlYDM0BFXHjjPWlWZ1Rg7UaddZeIDmi9jF3HmqiQ2w==", + "dev": true, + "dependencies": { + "esutils": "^2.0.2" + }, + "engines": { + "node": ">=6.0.0" + } + }, "node_modules/dom-serializer": { "version": "0.2.2", "resolved": "https://registry.npmjs.org/dom-serializer/-/dom-serializer-0.2.2.tgz", @@ -2196,6 +2436,221 @@ "node": ">=8" } }, + "node_modules/eslint": { + "version": "8.54.0", + "resolved": "https://registry.npmjs.org/eslint/-/eslint-8.54.0.tgz", + "integrity": "sha512-NY0DfAkM8BIZDVl6PgSa1ttZbx3xHgJzSNJKYcQglem6CppHyMhRIQkBVSSMaSRnLhig3jsDbEzOjwCVt4AmmA==", + "dev": true, + "dependencies": { + "@eslint-community/eslint-utils": "^4.2.0", + "@eslint-community/regexpp": "^4.6.1", + "@eslint/eslintrc": "^2.1.3", + "@eslint/js": "8.54.0", + "@humanwhocodes/config-array": "^0.11.13", + "@humanwhocodes/module-importer": "^1.0.1", + "@nodelib/fs.walk": "^1.2.8", + "@ungap/structured-clone": "^1.2.0", + "ajv": "^6.12.4", + "chalk": "^4.0.0", + "cross-spawn": "^7.0.2", + "debug": "^4.3.2", + "doctrine": "^3.0.0", + "escape-string-regexp": "^4.0.0", + "eslint-scope": "^7.2.2", + "eslint-visitor-keys": "^3.4.3", + "espree": "^9.6.1", + "esquery": "^1.4.2", + "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", + "graphemer": "^1.4.0", + "ignore": "^5.2.0", + "imurmurhash": "^0.1.4", + "is-glob": "^4.0.0", + "is-path-inside": "^3.0.3", + "js-yaml": "^4.1.0", + "json-stable-stringify-without-jsonify": "^1.0.1", + "levn": "^0.4.1", + "lodash.merge": "^4.6.2", + "minimatch": "^3.1.2", + "natural-compare": "^1.4.0", + "optionator": "^0.9.3", + "strip-ansi": "^6.0.1", + "text-table": "^0.2.0" + }, + "bin": { + "eslint": "bin/eslint.js" + }, + "engines": { + "node": "^12.22.0 || ^14.17.0 || >=16.0.0" + }, + "funding": { + "url": "https://opencollective.com/eslint" + } + }, + "node_modules/eslint-scope": { + "version": "7.2.2", + "resolved": "https://registry.npmjs.org/eslint-scope/-/eslint-scope-7.2.2.tgz", + "integrity": "sha512-dOt21O7lTMhDM+X9mB4GX+DZrZtCUJPL/wlcTqxyrx5IvO0IYtILdtrQGQp+8n5S0gwSVmOf9NQrjMOgfQZlIg==", + "dev": true, + "dependencies": { + "esrecurse": "^4.3.0", + "estraverse": "^5.2.0" + }, + "engines": { + "node": "^12.22.0 || ^14.17.0 || >=16.0.0" + }, + "funding": { + "url": "https://opencollective.com/eslint" + } + }, + "node_modules/eslint-visitor-keys": { + "version": "3.4.3", + "resolved": "https://registry.npmjs.org/eslint-visitor-keys/-/eslint-visitor-keys-3.4.3.tgz", + "integrity": "sha512-wpc+LXeiyiisxPlEkUzU6svyS1frIO3Mgxj1fdy7Pm8Ygzguax2N3Fa/D/ag1WqbOprdI+uY6wMUl8/a2G+iag==", + "dev": true, + "engines": { + "node": "^12.22.0 || ^14.17.0 || >=16.0.0" + }, + "funding": { + "url": "https://opencollective.com/eslint" + } + }, + "node_modules/eslint/node_modules/argparse": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/argparse/-/argparse-2.0.1.tgz", + "integrity": "sha512-8+9WqebbFzpX9OR+Wa6O29asIogeRMzcGtAINdpMHHyAg10f05aSFVBbcEqGf/PXw1EjAZ+q2/bEBg3DvurK3Q==", + "dev": true + }, + "node_modules/eslint/node_modules/escape-string-regexp": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/escape-string-regexp/-/escape-string-regexp-4.0.0.tgz", + "integrity": "sha512-TtpcNJ3XAzx3Gq8sWRzJaVajRs0uVxA2YAkdb1jm2YkPz4G6egUFAyA3n5vtEIZefPk5Wa4UXbKuS5fKkJWdgA==", + "dev": true, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/eslint/node_modules/find-up": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/find-up/-/find-up-5.0.0.tgz", + "integrity": "sha512-78/PXT1wlLLDgTzDs7sjq9hzz0vXD+zn+7wypEe4fXQxCmdmqfGsEPQxmiCSQI3ajFV91bVSsvNtrJRiW6nGng==", + "dev": true, + "dependencies": { + "locate-path": "^6.0.0", + "path-exists": "^4.0.0" + }, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/eslint/node_modules/glob-parent": { + "version": "6.0.2", + "resolved": "https://registry.npmjs.org/glob-parent/-/glob-parent-6.0.2.tgz", + "integrity": "sha512-XxwI8EOhVQgWp6iDL+3b0r86f4d6AX6zSU55HfB4ydCEuXLXc5FcYeOu+nnGftS4TEju/11rt4KJPTMgbfmv4A==", + "dev": true, + "dependencies": { + "is-glob": "^4.0.3" + }, + "engines": { + "node": ">=10.13.0" + } + }, + "node_modules/eslint/node_modules/globals": { + "version": "13.24.0", + "resolved": "https://registry.npmjs.org/globals/-/globals-13.24.0.tgz", + "integrity": "sha512-AhO5QUcj8llrbG09iWhPU2B204J1xnPeL8kQmVorSsy+Sjj1sk8gIyh6cUocGmH4L0UuhAJy+hJMRA4mgA4mFQ==", + "dev": true, + "dependencies": { + "type-fest": "^0.20.2" + }, + "engines": { + "node": ">=8" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/eslint/node_modules/js-yaml": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/js-yaml/-/js-yaml-4.1.0.tgz", + "integrity": "sha512-wpxZs9NoxZaJESJGIZTyDEaYpl0FKSA+FB9aJiyemKhMwkxQg63h4T1KJgUGHpTqPDNRcmmYLugrRjJlBtWvRA==", + "dev": true, + "dependencies": { + "argparse": "^2.0.1" + }, + "bin": { + "js-yaml": "bin/js-yaml.js" + } + }, + "node_modules/eslint/node_modules/locate-path": { + "version": "6.0.0", + "resolved": "https://registry.npmjs.org/locate-path/-/locate-path-6.0.0.tgz", + "integrity": "sha512-iPZK6eYjbxRu3uB4/WZ3EsEIMJFMqAoopl3R+zuq0UjcAm/MO6KCweDgPfP3elTztoKP3KtnVHxTn2NHBSDVUw==", + "dev": true, + "dependencies": { + "p-locate": "^5.0.0" + }, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/eslint/node_modules/p-locate": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/p-locate/-/p-locate-5.0.0.tgz", + "integrity": "sha512-LaNjtRWUBY++zB5nE/NwcaoMylSPk+S+ZHNB1TzdbMJMny6dynpAGt7X/tl/QYq3TIeE6nxHppbo2LGymrG5Pw==", + "dev": true, + "dependencies": { + "p-limit": "^3.0.2" + }, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/eslint/node_modules/type-fest": { + "version": "0.20.2", + "resolved": "https://registry.npmjs.org/type-fest/-/type-fest-0.20.2.tgz", + "integrity": "sha512-Ne+eE4r0/iWnpAxD852z3A+N0Bt5RN//NjJwRd2VFHEmrywxf5vsZlh4R6lixl6B+wz/8d+maTSAkN1FIkI3LQ==", + "dev": true, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/espree": { + "version": "9.6.1", + "resolved": "https://registry.npmjs.org/espree/-/espree-9.6.1.tgz", + "integrity": "sha512-oruZaFkjorTpF32kDSI5/75ViwGeZginGGy2NoOSg3Q9bnwlnmDm4HLnkl0RE3n+njDXR037aY1+x58Z/zFdwQ==", + "dev": true, + "dependencies": { + "acorn": "^8.9.0", + "acorn-jsx": "^5.3.2", + "eslint-visitor-keys": "^3.4.1" + }, + "engines": { + "node": "^12.22.0 || ^14.17.0 || >=16.0.0" + }, + "funding": { + "url": "https://opencollective.com/eslint" + } + }, "node_modules/esprima": { "version": "4.0.1", "resolved": "https://registry.npmjs.org/esprima/-/esprima-4.0.1.tgz", @@ -2209,6 +2664,48 @@ "node": ">=4" } }, + "node_modules/esquery": { + "version": "1.5.0", + "resolved": "https://registry.npmjs.org/esquery/-/esquery-1.5.0.tgz", + "integrity": "sha512-YQLXUplAwJgCydQ78IMJywZCceoqk1oH01OERdSAJc/7U2AylwjhSCLDEtqwg811idIS/9fIU5GjG73IgjKMVg==", + "dev": true, + "dependencies": { + "estraverse": "^5.1.0" + }, + "engines": { + "node": ">=0.10" + } + }, + "node_modules/esrecurse": { + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/esrecurse/-/esrecurse-4.3.0.tgz", + "integrity": "sha512-KmfKL3b6G+RXvP8N1vr3Tq1kL/oCFgn2NYXEtqP8/L3pKapUA4G8cFVaoF3SU323CD4XypR/ffioHmkti6/Tag==", + "dev": true, + "dependencies": { + "estraverse": "^5.2.0" + }, + "engines": { + "node": ">=4.0" + } + }, + "node_modules/estraverse": { + "version": "5.3.0", + "resolved": "https://registry.npmjs.org/estraverse/-/estraverse-5.3.0.tgz", + "integrity": "sha512-MMdARuVEQziNTeJD8DgMqmhwR11BRQ/cBP+pLtYdSTnf3MIO8fFeiINEbX36ZdNlfU/7A9f3gUw49B3oQsvwBA==", + "dev": true, + "engines": { + "node": ">=4.0" + } + }, + "node_modules/esutils": { + "version": "2.0.3", + "resolved": "https://registry.npmjs.org/esutils/-/esutils-2.0.3.tgz", + "integrity": "sha512-kVscqXk4OCp68SZ0dkgEKVi6/8ij300KBWTJq32P/dYeWTSwK41WyTxalN1eRmA5Z9UU/LX9D7FWSmV9SAYx6g==", + "dev": true, + "engines": { + "node": ">=0.10.0" + } + }, "node_modules/execa": { "version": "5.1.1", "resolved": "https://registry.npmjs.org/execa/-/execa-5.1.1.tgz", @@ -2263,12 +2760,33 @@ "integrity": "sha512-dX7e/LHVJ6W3DE1MHWi9S1EYzDESENfLrYohG2G++ovZrYOkm4Knwa0mc1cn84xJOR4KEU0WSchhLbd0UklbHw==", "dev": true }, + "node_modules/fast-deep-equal": { + "version": "3.1.3", + "resolved": "https://registry.npmjs.org/fast-deep-equal/-/fast-deep-equal-3.1.3.tgz", + "integrity": "sha512-f3qQ9oQy9j2AhBe/H9VC91wLmKBCCU/gDOnKNAYG5hswO7BLKj09Hc5HYNz9cGI++xlpDCIgDaitVs03ATR84Q==", + "dev": true + }, "node_modules/fast-json-stable-stringify": { "version": "2.1.0", "resolved": "https://registry.npmjs.org/fast-json-stable-stringify/-/fast-json-stable-stringify-2.1.0.tgz", "integrity": "sha512-lhd/wF+Lk98HZoTCtlVraHtfh5XYijIjalXck7saUtuanSDyLMxnHhSXEDJqHxD7msR8D0uCmqlkwjCV8xvwHw==", "dev": true }, + "node_modules/fast-levenshtein": { + "version": "2.0.6", + "resolved": "https://registry.npmjs.org/fast-levenshtein/-/fast-levenshtein-2.0.6.tgz", + "integrity": "sha512-DCXu6Ifhqcks7TZKY3Hxp3y6qphY5SJZmrWMDrKcERSOXWQdMhU9Ig/PYrzyw/ul9jOIyh0N4M0tbC5hodg8dw==", + "dev": true + }, + "node_modules/fastq": { + "version": "1.15.0", + "resolved": "https://registry.npmjs.org/fastq/-/fastq-1.15.0.tgz", + "integrity": "sha512-wBrocU2LCXXa+lWBt8RoIRD89Fi8OdABODa/kEnyeyjS5aZO5/GNvI5sEINADqP/h8M29UHTHUb53sUu5Ihqdw==", + "dev": true, + "dependencies": { + "reusify": "^1.0.4" + } + }, "node_modules/fb-watchman": { "version": "2.0.2", "resolved": "https://registry.npmjs.org/fb-watchman/-/fb-watchman-2.0.2.tgz", @@ -2278,6 +2796,18 @@ "bser": "2.1.1" } }, + "node_modules/file-entry-cache": { + "version": "6.0.1", + "resolved": "https://registry.npmjs.org/file-entry-cache/-/file-entry-cache-6.0.1.tgz", + "integrity": "sha512-7Gps/XWymbLk2QLYK4NzpMOrYjMhdIxXuIvy2QBsLE6ljuodKvdkWs/cpyJJ3CVIVpH0Oi1Hvg1ovbMzLdFBBg==", + "dev": true, + "dependencies": { + "flat-cache": "^3.0.4" + }, + "engines": { + "node": "^10.12.0 || >=12.0.0" + } + }, "node_modules/file-uri-to-path": { "version": "1.0.0", "resolved": "https://registry.npmjs.org/file-uri-to-path/-/file-uri-to-path-1.0.0.tgz", @@ -2317,6 +2847,26 @@ "flat": "cli.js" } }, + "node_modules/flat-cache": { + "version": "3.2.0", + "resolved": "https://registry.npmjs.org/flat-cache/-/flat-cache-3.2.0.tgz", + "integrity": "sha512-CYcENa+FtcUKLmhhqyctpclsq7QF38pKjZHsGNiSQF5r4FtoKDWabFDl3hzaEQMvT1LHEysw5twgLvpYYb4vbw==", + "dev": true, + "dependencies": { + "flatted": "^3.2.9", + "keyv": "^4.5.3", + "rimraf": "^3.0.2" + }, + "engines": { + "node": "^10.12.0 || >=12.0.0" + } + }, + "node_modules/flatted": { + "version": "3.2.9", + "resolved": "https://registry.npmjs.org/flatted/-/flatted-3.2.9.tgz", + "integrity": "sha512-36yxDn5H7OFZQla0/jFJmbIKTdZAQHngCedGxiMmpNfEZM0sdEeT+WczLQrjK6D7o2aiyLYDnkw0R3JK0Qv1RQ==", + "dev": true + }, "node_modules/fs-minipass": { "version": "2.1.0", "resolved": "https://registry.npmjs.org/fs-minipass/-/fs-minipass-2.1.0.tgz", @@ -2333,20 +2883,6 @@ "resolved": "https://registry.npmjs.org/fs.realpath/-/fs.realpath-1.0.0.tgz", "integrity": "sha512-OO0pH2lK6a0hZnAdau5ItzHPI6pUlvI7jMVnxUQRtw4owF2wk8lOSabtGDCTP4Ggrg2MbGnWO9X8K1t4+fGMDw==" }, - "node_modules/fsevents": { - "version": "2.3.3", - "resolved": "https://registry.npmjs.org/fsevents/-/fsevents-2.3.3.tgz", - "integrity": "sha512-5xoDfX+fL7faATnagmWPpbFtwh/R77WmMMqqHGS65C3vvB0YHrgF+B1YmZ3441tMj5n63k0212XNoJwzlhffQw==", - "dev": true, - "hasInstallScript": true, - "optional": true, - "os": [ - "darwin" - ], - "engines": { - "node": "^8.16.0 || ^10.6.0 || >=11.0.0" - } - }, "node_modules/function-bind": { "version": "1.1.2", "resolved": "https://registry.npmjs.org/function-bind/-/function-bind-1.1.2.tgz", @@ -2460,6 +2996,12 @@ "integrity": "sha512-RbJ5/jmFcNNCcDV5o9eTnBLJ/HszWV0P73bc+Ff4nS/rJj+YaS6IGyiOL0VoBYX+l1Wrl3k63h/KrH+nhJ0XvQ==", "dev": true }, + "node_modules/graphemer": { + "version": "1.4.0", + "resolved": "https://registry.npmjs.org/graphemer/-/graphemer-1.4.0.tgz", + "integrity": "sha512-EtKwoO6kxCL9WO5xipiHTZlSzBm7WLT627TqC/uVRd0HKmq8NXyebnNYxDoBi7wt8eTWrUrKXCOVaFq9x1kgag==", + "dev": true + }, "node_modules/has-flag": { "version": "4.0.0", "resolved": "https://registry.npmjs.org/has-flag/-/has-flag-4.0.0.tgz", @@ -2576,6 +3118,40 @@ "node": ">=0.10.0" } }, + "node_modules/ignore": { + "version": "5.3.0", + "resolved": "https://registry.npmjs.org/ignore/-/ignore-5.3.0.tgz", + "integrity": "sha512-g7dmpshy+gD7mh88OC9NwSGTKoc3kyLAZQRU1mt53Aw/vnvfXnbC+F/7F7QoYVKbV+KNvJx8wArewKy1vXMtlg==", + "dev": true, + "engines": { + "node": ">= 4" + } + }, + "node_modules/import-fresh": { + "version": "3.3.0", + "resolved": "https://registry.npmjs.org/import-fresh/-/import-fresh-3.3.0.tgz", + "integrity": "sha512-veYYhQa+D1QBKznvhUHxb8faxlrwUnxseDAbAp457E0wLNio2bOSKnjYDhMj+YiAq61xrMGhQk9iXVk5FzgQMw==", + "dev": true, + "dependencies": { + "parent-module": "^1.0.0", + "resolve-from": "^4.0.0" + }, + "engines": { + "node": ">=6" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/import-fresh/node_modules/resolve-from": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/resolve-from/-/resolve-from-4.0.0.tgz", + "integrity": "sha512-pb/MYmXstAkysRFx8piNI1tGFNQIFA3vkE3Gq4EuA1dF6gHp/+vgZqsCGJapvy8N3Q+4o7FwvquPJcnZ7RYy4g==", + "dev": true, + "engines": { + "node": ">=4" + } + }, "node_modules/import-local": { "version": "3.1.0", "resolved": "https://registry.npmjs.org/import-local/-/import-local-3.1.0.tgz", @@ -2722,6 +3298,15 @@ "node": ">=0.12.0" } }, + "node_modules/is-path-inside": { + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/is-path-inside/-/is-path-inside-3.0.3.tgz", + "integrity": "sha512-Fd4gABb+ycGAmKou8eMftCupSir5lRxqf4aD/vd0cD2qc4HL07OjCeuHMr8Ro4CoMaeCKDB0/ECBOVWjTwUvPQ==", + "dev": true, + "engines": { + "node": ">=8" + } + }, "node_modules/is-plain-obj": { "version": "2.1.0", "resolved": "https://registry.npmjs.org/is-plain-obj/-/is-plain-obj-2.1.0.tgz", @@ -3514,12 +4099,30 @@ "node": ">=0.8.0" } }, + "node_modules/json-buffer": { + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/json-buffer/-/json-buffer-3.0.1.tgz", + "integrity": "sha512-4bV5BfR2mqfQTJm+V5tPPdf+ZpuhiIvTuAB5g8kcrXOZpTT/QwwVRWBywX1ozr6lEuPdbHxwaJlm9G6mI2sfSQ==", + "dev": true + }, "node_modules/json-parse-even-better-errors": { "version": "2.3.1", "resolved": "https://registry.npmjs.org/json-parse-even-better-errors/-/json-parse-even-better-errors-2.3.1.tgz", "integrity": "sha512-xyFwyhro/JEof6Ghe2iz2NcXoj2sloNsWr/XsERDK/oiPCfaNhl5ONfp+jQdAZRQQ0IJWNzH9zIZF7li91kh2w==", "dev": true }, + "node_modules/json-schema-traverse": { + "version": "0.4.1", + "resolved": "https://registry.npmjs.org/json-schema-traverse/-/json-schema-traverse-0.4.1.tgz", + "integrity": "sha512-xbbCH5dCYU5T8LcEhhuh7HJ88HXuW3qsI3Y0zOZFKfZEHcpWiHU/Jxzk629Brsab/mMiHQti9wMP+845RPe3Vg==", + "dev": true + }, + "node_modules/json-stable-stringify-without-jsonify": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/json-stable-stringify-without-jsonify/-/json-stable-stringify-without-jsonify-1.0.1.tgz", + "integrity": "sha512-Bdboy+l7tA3OGW6FjyFHWkP5LuByj1Tk33Ljyq0axyzdk9//JSi2u3fP1QSmd1KNwq6VOKYGlAu87CisVir6Pw==", + "dev": true + }, "node_modules/json5": { "version": "2.2.3", "resolved": "https://registry.npmjs.org/json5/-/json5-2.2.3.tgz", @@ -3532,6 +4135,15 @@ "node": ">=6" } }, + "node_modules/keyv": { + "version": "4.5.4", + "resolved": "https://registry.npmjs.org/keyv/-/keyv-4.5.4.tgz", + "integrity": "sha512-oxVHkHR/EJf2CNXnWxRLW6mg7JyCCUcG0DtEGmL2ctUo1PNTin1PUil+r/+4r5MpVgC/fn1kjsx7mjSujKqIpw==", + "dev": true, + "dependencies": { + "json-buffer": "3.0.1" + } + }, "node_modules/klaw": { "version": "3.0.0", "resolved": "https://registry.npmjs.org/klaw/-/klaw-3.0.0.tgz", @@ -3559,6 +4171,19 @@ "node": ">=6" } }, + "node_modules/levn": { + "version": "0.4.1", + "resolved": "https://registry.npmjs.org/levn/-/levn-0.4.1.tgz", + "integrity": "sha512-+bT2uH4E5LGE7h/n3evcS/sQlJXCpIp6ym8OWJ5eV6+67Dsql/LaaT7qJBAt2rzfoa/5QBGBhxDix1dMt2kQKQ==", + "dev": true, + "dependencies": { + "prelude-ls": "^1.2.1", + "type-check": "~0.4.0" + }, + "engines": { + "node": ">= 0.8.0" + } + }, "node_modules/lines-and-columns": { "version": "1.2.4", "resolved": "https://registry.npmjs.org/lines-and-columns/-/lines-and-columns-1.2.4.tgz", @@ -3592,6 +4217,12 @@ "integrity": "sha512-v2kDEe57lecTulaDIuNTPy3Ry4gLGJ6Z1O3vE1krgXZNrsQ+LFTGHVxVjcXPs17LhbZVGedAJv8XZ1tvj5FvSg==", "dev": true }, + "node_modules/lodash.merge": { + "version": "4.6.2", + "resolved": "https://registry.npmjs.org/lodash.merge/-/lodash.merge-4.6.2.tgz", + "integrity": "sha512-0KpjqXRVvrYyCsX1swR/XTK0va6VQkQM6MNo7PqW77ByjAhoARA8EfrP1N4+KlKj8YS0ZUCtRT/YUuhyYDujIQ==", + "dev": true + }, "node_modules/log-symbols": { "version": "4.1.0", "resolved": "https://registry.npmjs.org/log-symbols/-/log-symbols-4.1.0.tgz", @@ -4365,6 +4996,23 @@ "url": "https://github.com/sponsors/sindresorhus" } }, + "node_modules/optionator": { + "version": "0.9.3", + "resolved": "https://registry.npmjs.org/optionator/-/optionator-0.9.3.tgz", + "integrity": "sha512-JjCoypp+jKn1ttEFExxhetCKeJt9zhAgAve5FXHixTvFDW/5aEktX9bufBKLRRMdU7bNtpLfcGu94B3cdEJgjg==", + "dev": true, + "dependencies": { + "@aashutoshrathi/word-wrap": "^1.2.3", + "deep-is": "^0.1.3", + "fast-levenshtein": "^2.0.6", + "levn": "^0.4.1", + "prelude-ls": "^1.2.1", + "type-check": "^0.4.0" + }, + "engines": { + "node": ">= 0.8.0" + } + }, "node_modules/p-limit": { "version": "3.1.0", "resolved": "https://registry.npmjs.org/p-limit/-/p-limit-3.1.0.tgz", @@ -4431,6 +5079,18 @@ "node": ">=6" } }, + "node_modules/parent-module": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/parent-module/-/parent-module-1.0.1.tgz", + "integrity": "sha512-GQ2EWRpQV8/o+Aw8YqtfZZPfNRWZYkbidE9k5rpl/hC3vtHHBfGm2Ifi6qWV+coDGkrUKZAxE3Lot5kcsRlh+g==", + "dev": true, + "dependencies": { + "callsites": "^3.0.0" + }, + "engines": { + "node": ">=6" + } + }, "node_modules/parse-json": { "version": "5.2.0", "resolved": "https://registry.npmjs.org/parse-json/-/parse-json-5.2.0.tgz", @@ -4520,6 +5180,15 @@ "node": ">=8" } }, + "node_modules/prelude-ls": { + "version": "1.2.1", + "resolved": "https://registry.npmjs.org/prelude-ls/-/prelude-ls-1.2.1.tgz", + "integrity": "sha512-vkcDPrRZo1QZLbn5RLGPpg/WmIQ65qoWWhcGKf/b5eplkkarX0m9z8ppCat4mlOqUsWpyNuYgO3VRyrYHSzX5g==", + "dev": true, + "engines": { + "node": ">= 0.8.0" + } + }, "node_modules/pretty-format": { "version": "29.7.0", "resolved": "https://registry.npmjs.org/pretty-format/-/pretty-format-29.7.0.tgz", @@ -4578,6 +5247,15 @@ "node": ">= 6" } }, + "node_modules/punycode": { + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/punycode/-/punycode-2.3.1.tgz", + "integrity": "sha512-vYt7UD1U9Wg6138shLtLOvdAu+8DsC/ilFtEVHcH+wydcSpNE20AfSOduf6MkRFahL5FY7X1oU7nKVZFtfq8Fg==", + "dev": true, + "engines": { + "node": ">=6" + } + }, "node_modules/pure-rand": { "version": "6.0.4", "resolved": "https://registry.npmjs.org/pure-rand/-/pure-rand-6.0.4.tgz", @@ -4594,6 +5272,26 @@ } ] }, + "node_modules/queue-microtask": { + "version": "1.2.3", + "resolved": "https://registry.npmjs.org/queue-microtask/-/queue-microtask-1.2.3.tgz", + "integrity": "sha512-NuaNSa6flKT5JaSYQzJok04JzTL1CA6aGhv5rfLW3PgqA+M2ChpZQnAC8h8i4ZFkBS8X5RqkDBHA7r4hej3K9A==", + "dev": true, + "funding": [ + { + "type": "github", + "url": "https://github.com/sponsors/feross" + }, + { + "type": "patreon", + "url": "https://www.patreon.com/feross" + }, + { + "type": "consulting", + "url": "https://feross.org/support" + } + ] + }, "node_modules/randombytes": { "version": "2.1.0", "resolved": "https://registry.npmjs.org/randombytes/-/randombytes-2.1.0.tgz", @@ -4707,6 +5405,16 @@ "node": ">= 4" } }, + "node_modules/reusify": { + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/reusify/-/reusify-1.0.4.tgz", + "integrity": "sha512-U9nH88a3fc/ekCF1l0/UP1IosiuIjyTh7hBvXVMHYgVcfGvt897Xguj2UOLDeI5BG2m7/uwyaLVT6fbtCwTyzw==", + "dev": true, + "engines": { + "iojs": ">=1.0.0", + "node": ">=0.10.0" + } + }, "node_modules/rimraf": { "version": "3.0.2", "resolved": "https://registry.npmjs.org/rimraf/-/rimraf-3.0.2.tgz", @@ -4721,6 +5429,29 @@ "url": "https://github.com/sponsors/isaacs" } }, + "node_modules/run-parallel": { + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/run-parallel/-/run-parallel-1.2.0.tgz", + "integrity": "sha512-5l4VyZR86LZ/lDxZTR6jqL8AFE2S0IFLMP26AbjsLVADxHdhB/c0GUsH+y39UfCi3dzz8OlQuPmnaJOMoDHQBA==", + "dev": true, + "funding": [ + { + "type": "github", + "url": "https://github.com/sponsors/feross" + }, + { + "type": "patreon", + "url": "https://www.patreon.com/feross" + }, + { + "type": "consulting", + "url": "https://feross.org/support" + } + ], + "dependencies": { + "queue-microtask": "^1.2.2" + } + }, "node_modules/safe-buffer": { "version": "5.2.1", "resolved": "https://registry.npmjs.org/safe-buffer/-/safe-buffer-5.2.1.tgz", @@ -5064,6 +5795,12 @@ "node": ">=8" } }, + "node_modules/text-table": { + "version": "0.2.0", + "resolved": "https://registry.npmjs.org/text-table/-/text-table-0.2.0.tgz", + "integrity": "sha512-N+8UisAXDGk8PFXP4HAzVR9nbfmVJ3zYLAWiTIoqC5v5isinhr+r5uaO8+7r3BMfuNIufIsA7RdpVgacC2cSpw==", + "dev": true + }, "node_modules/tmpl": { "version": "1.0.5", "resolved": "https://registry.npmjs.org/tmpl/-/tmpl-1.0.5.tgz", @@ -5102,6 +5839,18 @@ "resolved": "https://registry.npmjs.org/tr46/-/tr46-0.0.3.tgz", "integrity": "sha512-N3WMsuqV66lT30CrXNbEjx4GEwlow3v6rr4mCcv6prnfwhS01rkgyFdjPNBYd9br7LpXV1+Emh01fHnq2Gdgrw==" }, + "node_modules/type-check": { + "version": "0.4.0", + "resolved": "https://registry.npmjs.org/type-check/-/type-check-0.4.0.tgz", + "integrity": "sha512-XleUoc9uwGXqjWwXaUTZAmzMcFZ5858QA2vvx1Ur5xIcixXIP+8LnFDgRplU30us6teqdlskFfu+ae4K79Ooew==", + "dev": true, + "dependencies": { + "prelude-ls": "^1.2.1" + }, + "engines": { + "node": ">= 0.8.0" + } + }, "node_modules/type-detect": { "version": "4.0.8", "resolved": "https://registry.npmjs.org/type-detect/-/type-detect-4.0.8.tgz", @@ -5208,6 +5957,15 @@ "browserslist": ">= 4.21.0" } }, + "node_modules/uri-js": { + "version": "4.4.1", + "resolved": "https://registry.npmjs.org/uri-js/-/uri-js-4.4.1.tgz", + "integrity": "sha512-7rKUyy33Q1yc98pQ1DAmLtwX109F7TIfWlW1Ydo8Wl1ii1SeHieeh0HHfPeL2fMXK6z0s8ecKs9frCuLJvndBg==", + "dev": true, + "dependencies": { + "punycode": "^2.1.0" + } + }, "node_modules/util-deprecate": { "version": "1.0.2", "resolved": "https://registry.npmjs.org/util-deprecate/-/util-deprecate-1.0.2.tgz", diff --git a/package.json b/package.json index 45d1ec63..ca112fa9 100644 --- a/package.json +++ b/package.json @@ -43,6 +43,7 @@ "@types/node": "^20.4.5", "bluebird": "^3.5.3", "jest": "^29.7.0", + "eslint": "8.54.0", "jsdoc": "^3.4.0", "jshint": "^2.10.1", "mocha": "^10.2.0", From 9b3bdd55808642d6022a6ffb407f3fd306ad3288 Mon Sep 17 00:00:00 2001 From: Milind L Date: Thu, 30 Nov 2023 22:16:05 +0530 Subject: [PATCH 18/26] macOS - basic changes --- .semaphore/semaphore.yml | 42 +++++++++++++++++++++++++++++----------- 1 file changed, 31 insertions(+), 11 deletions(-) diff --git a/.semaphore/semaphore.yml b/.semaphore/semaphore.yml index 08d3ad9b..d14ffa01 100644 --- a/.semaphore/semaphore.yml +++ b/.semaphore/semaphore.yml @@ -27,28 +27,48 @@ global_job_config: - checkout blocks: - - name: "Linux amd64" - dependencies: [ ] + # - name: "Linux amd64" + # dependencies: [ ] + # task: + # agent: + # machine: + # type: s1-prod-ubuntu20-04-amd64-2 + # prologue: + # commands: + # - sem-version node 18.19.0 + # - node --version + # - npm --version + # - npx --version + # - sudo apt-get update -q + # - git submodule update --init --recursive + # jobs: + # - name: "Build from source and test" + # commands: + # - sudo apt install -y libcurl4-openssl-dev libcrypto++-dev libssl-dev libzstd-dev + # - npm install # this will actually not build anything if we have a release, but rather, fetch things using node-pre-gyp - so change this later. + # - make test + # - name: "ESLint" + # commands: + # - npm install --only=dev + # - npx eslint lib/kafkajs + + - name: 'OSX arm64/m1' + dependencies: [] task: agent: machine: - type: s1-prod-ubuntu20-04-amd64-2 + type: s1-prod-macos-arm64 prologue: commands: - sem-version node 18.19.0 - node --version - npm --version - npx --version - - sudo apt-get update -q - git submodule update --init --recursive jobs: - - name: "Build from source and test" + - name: 'Build from source and test' commands: - - sudo apt install -y libcurl4-openssl-dev libcrypto++-dev libssl-dev libzstd-dev - - npm install # this will actually not build anything, but rather, fetch things using node-pre-gyp - so change this later. + - npm install # this will actually not build anything if we have a release, but rather, fetch things using node-pre-gyp - so change this later. - make test - - name: "ESLint" - commands: - - npm install --only=dev - - npx eslint lib/kafkajs + From af6e124b05e3fe1d5fdeb31368f505efe86354e6 Mon Sep 17 00:00:00 2001 From: Milind L Date: Fri, 1 Dec 2023 14:54:26 +0530 Subject: [PATCH 19/26] Try adding packaging flow --- .semaphore/semaphore.yml | 84 ++++++++++++++++++++++++++-------------- package.json | 2 +- 2 files changed, 56 insertions(+), 30 deletions(-) diff --git a/.semaphore/semaphore.yml b/.semaphore/semaphore.yml index d14ffa01..9aa850df 100644 --- a/.semaphore/semaphore.yml +++ b/.semaphore/semaphore.yml @@ -26,49 +26,75 @@ global_job_config: commands: - checkout +# blocks: +# - name: "Linux amd64: Build and test" +# dependencies: [ ] +# task: +# agent: +# machine: +# type: s1-prod-ubuntu20-04-amd64-2 +# prologue: +# commands: +# - sem-version node 18.19.0 +# - node --version +# - npm --version +# - npx --version +# - sudo apt-get update -q +# - git submodule update --init --recursive +# - sudo apt install -y libcurl4-openssl-dev libcrypto++-dev libssl-dev libzstd-dev +# - npm install # this will actually not build anything if we have a release, but rather, fetch things using node-pre-gyp - so change this later. +# jobs: +# - name: "Test" +# commands: +# - make test +# - name: "ESLint" +# commands: +# - npx eslint lib/kafkajs + blocks: - # - name: "Linux amd64" - # dependencies: [ ] + - name: "Linux amd64: Release" + dependencies: [ ] + run: + when: "tag =~ '^v[0-9]\\.'" + task: + agent: + machine: + type: s1-prod-ubuntu20-04-amd64-2 + prologue: + commands: + - sem-version node 18.19.0 + - node --version + - npm --version + - npx --version + - sudo apt-get update -q + - git submodule update --init --recursive + - sudo apt install -y libcurl4-openssl-dev libcrypto++-dev libssl-dev libzstd-dev + jobs: + - name: "Release" + commands: + - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v108-linux-x64.tar.gz" + - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. + - npx node-pre-gyp package + - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} + - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${ARTIFACT_KEY}" + + # - name: 'OSX arm64/m1' + # dependencies: [] # task: # agent: # machine: - # type: s1-prod-ubuntu20-04-amd64-2 + # type: s1-prod-macos-arm64 # prologue: # commands: # - sem-version node 18.19.0 # - node --version # - npm --version # - npx --version - # - sudo apt-get update -q # - git submodule update --init --recursive # jobs: - # - name: "Build from source and test" + # - name: 'Build from source and test' # commands: - # - sudo apt install -y libcurl4-openssl-dev libcrypto++-dev libssl-dev libzstd-dev # - npm install # this will actually not build anything if we have a release, but rather, fetch things using node-pre-gyp - so change this later. # - make test - # - name: "ESLint" - # commands: - # - npm install --only=dev - # - npx eslint lib/kafkajs - - - name: 'OSX arm64/m1' - dependencies: [] - task: - agent: - machine: - type: s1-prod-macos-arm64 - prologue: - commands: - - sem-version node 18.19.0 - - node --version - - npm --version - - npx --version - - git submodule update --init --recursive - jobs: - - name: 'Build from source and test' - commands: - - npm install # this will actually not build anything if we have a release, but rather, fetch things using node-pre-gyp - so change this later. - - make test diff --git a/package.json b/package.json index ca112fa9..e94a3815 100644 --- a/package.json +++ b/package.json @@ -1,6 +1,6 @@ { "name": "confluent-kafka-js", - "version": "v2.18.0", + "version": "v0.1.0-devel", "description": "Node.js bindings for librdkafka", "librdkafka": "2.3.0", "main": "lib/index.js", From 256cea7cead1456709d3239497842c55a592b65d Mon Sep 17 00:00:00 2001 From: Milind L Date: Tue, 5 Dec 2023 15:46:05 +0530 Subject: [PATCH 20/26] Everything, all at once --- .semaphore/semaphore.yml | 221 +++++++++++++++++++++++++++++---------- deps/librdkafka | 2 +- package-lock.json | 4 +- package.json | 4 +- util/configure.js | 7 +- 5 files changed, 176 insertions(+), 62 deletions(-) diff --git a/.semaphore/semaphore.yml b/.semaphore/semaphore.yml index 9aa850df..d45f8386 100644 --- a/.semaphore/semaphore.yml +++ b/.semaphore/semaphore.yml @@ -25,33 +25,58 @@ global_job_config: prologue: commands: - checkout + - git submodule update --init --recursive + - cd deps/librdkafka + - git fetch origin + - git checkout dev_libdir_fix_for_zstd_zlib + - cd ../../ + - export MKL_DEBUG=1 + -# blocks: -# - name: "Linux amd64: Build and test" -# dependencies: [ ] -# task: -# agent: -# machine: -# type: s1-prod-ubuntu20-04-amd64-2 -# prologue: -# commands: -# - sem-version node 18.19.0 -# - node --version -# - npm --version -# - npx --version -# - sudo apt-get update -q -# - git submodule update --init --recursive -# - sudo apt install -y libcurl4-openssl-dev libcrypto++-dev libssl-dev libzstd-dev -# - npm install # this will actually not build anything if we have a release, but rather, fetch things using node-pre-gyp - so change this later. -# jobs: -# - name: "Test" -# commands: -# - make test -# - name: "ESLint" -# commands: -# - npx eslint lib/kafkajs blocks: + - name: "Linux arm64: Build and test" + dependencies: [ ] + task: + agent: + machine: + type: s1-prod-ubuntu20-04-arm64-1 + jobs: + - name: "Build from source and test" + commands: + - npm install # this will actually not build anything if we have a release, but rather, fetch things using node-pre-gyp - so change this later. + - make test + + - name: 'macOS arm64/m1: Build and test' + dependencies: [] + task: + agent: + machine: + type: s1-prod-macos-arm64 + jobs: + - name: 'Build from source and test' + commands: + - npm install # this will actually not build anything if we have a release, but rather, fetch things using node-pre-gyp - so change this later. + - make test + + - name: "Linux amd64: Build, test, lint" + dependencies: [ ] + task: + agent: + machine: + type: s1-prod-ubuntu20-04-amd64-2 + prologue: + commands: + - npm install # this will actually not build anything if we have a release, but rather, fetch things using node-pre-gyp - so change this later. + jobs: + - name: "Test" + commands: + - make test + - name: "ESLint" + commands: + - npx eslint lib/kafkajs + + - name: "Linux amd64: Release" dependencies: [ ] run: @@ -60,41 +85,125 @@ blocks: agent: machine: type: s1-prod-ubuntu20-04-amd64-2 - prologue: - commands: - - sem-version node 18.19.0 - - node --version - - npm --version - - npx --version - - sudo apt-get update -q - - git submodule update --init --recursive - - sudo apt install -y libcurl4-openssl-dev libcrypto++-dev libssl-dev libzstd-dev + env_vars: + - name: ARCHITECTURE + value: "x64" + - name: PLATFORM + value: "linux" + - name: LIBC + value: "glibc" jobs: - - name: "Release" + - name: "Release: LTS:18" commands: - - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v108-linux-x64.tar.gz" + - sem-version node 18.19.0 + - export NODE_ABI=108 + - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - npx node-pre-gyp package - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} - - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${ARTIFACT_KEY}" - - # - name: 'OSX arm64/m1' - # dependencies: [] - # task: - # agent: - # machine: - # type: s1-prod-macos-arm64 - # prologue: - # commands: - # - sem-version node 18.19.0 - # - node --version - # - npm --version - # - npx --version - # - git submodule update --init --recursive - # jobs: - # - name: 'Build from source and test' - # commands: - # - npm install # this will actually not build anything if we have a release, but rather, fetch things using node-pre-gyp - so change this later. - # - make test - + - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" + - name: "Release: LTS:20" + commands: + - sem-version node 20.10.0 + - export NODE_ABI=115 + - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. + - npx node-pre-gyp package + - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} + - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" + - name: "Release: latest: 21" + commands: + - sem-version node 21.4.0 + - export NODE_ABI=120 + - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. + - npx node-pre-gyp package + - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} + - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" + - name: "Linux arm64: Release" + dependencies: [ ] + run: + when: "tag =~ '^v[0-9]\\.'" + task: + agent: + machine: + type: s1-prod-ubuntu20-04-arm64-1 + env_vars: + - name: ARCHITECTURE + value: "arm64" + - name: PLATFORM + value: "linux" + - name: LIBC + value: "glibc" + jobs: + - name: "Release: LTS:18" + commands: + - sem-version node 18.19.0 + - export NODE_ABI=108 + - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. + - npx node-pre-gyp package + - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} + - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" + - name: "Release: LTS:20" + commands: + - sem-version node 20.10.0 + - export NODE_ABI=115 + - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. + - npx node-pre-gyp package + - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} + - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" + - name: "Release: latest: 21" + commands: + - sem-version node 21.4.0 + - export NODE_ABI=120 + - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. + - npx node-pre-gyp package + - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} + - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" + - name: "macOS arm64/m1: Release" + dependencies: [ ] + run: + when: "tag =~ '^v[0-9]\\.'" + task: + agent: + machine: + type: s1-prod-macos-arm64 + env_vars: + - name: ARCHITECTURE + value: "arm64" + - name: PLATFORM + value: "darwin" + - name: LIBC + value: "unknown" + jobs: + - name: "Release: LTS:18" + commands: + - sem-version node 18.19.0 + - export NODE_ABI=108 + - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. + - npx node-pre-gyp package + - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} + - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" + - name: "Release: LTS:20" + commands: + - sem-version node 20.10.0 + - export NODE_ABI=115 + - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. + - npx node-pre-gyp package + - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} + - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" + - name: "Release: latest: 21" + commands: + - sem-version node 21.4.0 + - export NODE_ABI=120 + - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. + - npx node-pre-gyp package + - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} + - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" \ No newline at end of file diff --git a/deps/librdkafka b/deps/librdkafka index 95a542c8..cc58c2cc 160000 --- a/deps/librdkafka +++ b/deps/librdkafka @@ -1 +1 @@ -Subproject commit 95a542c87c61d2c45b445f91c73dd5442eb04f3c +Subproject commit cc58c2cc9aa651e71fcaa25d69766b4fdd6de977 diff --git a/package-lock.json b/package-lock.json index 689ecaca..0ef2c3a9 100644 --- a/package-lock.json +++ b/package-lock.json @@ -1,12 +1,12 @@ { "name": "confluent-kafka-js", - "version": "v2.18.0", + "version": "v0.1.1-devel", "lockfileVersion": 3, "requires": true, "packages": { "": { "name": "confluent-kafka-js", - "version": "v2.18.0", + "version": "v0.1.1-devel", "hasInstallScript": true, "license": "MIT", "dependencies": { diff --git a/package.json b/package.json index e94a3815..252e948d 100644 --- a/package.json +++ b/package.json @@ -1,6 +1,6 @@ { "name": "confluent-kafka-js", - "version": "v0.1.0-devel", + "version": "v0.1.2-devel", "description": "Node.js bindings for librdkafka", "librdkafka": "2.3.0", "main": "lib/index.js", @@ -16,7 +16,7 @@ "binary": { "module_name": "node-librdkafka", "module_path": "./build/{configuration}/", - "package_name": "{module_name}-v{version}-{node_abi}-{platform}-{arch}.tar.gz", + "package_name": "{module_name}-v{version}-{node_abi}-{platform}-{libc}-{arch}.tar.gz", "host": "https://github.com/confluentinc/confluent-kafka-js/releases/download/", "remote_path": "v{version}" }, diff --git a/util/configure.js b/util/configure.js index 040c8a0b..a47fea97 100644 --- a/util/configure.js +++ b/util/configure.js @@ -7,6 +7,10 @@ var path = require('path'); var baseDir = path.resolve(__dirname, '../'); var releaseDir = path.join(baseDir, 'build', 'deps'); +// var command = './configure --install-deps --source-deps-only --disable-lz4-ext --enable-static --enable-strip --disable-gssapi'; +// if (!process.env.IS_ON_CI) { +var command = './configure --install-deps --source-deps-only --disable-lz4-ext --enable-static --enable-strip --disable-gssapi --prefix=' + releaseDir + ' --libdir=' + releaseDir; +// } var isWin = /^win/.test(process.platform); @@ -19,7 +23,8 @@ if (isWin) { var childProcess = require('child_process'); try { - childProcess.execSync('./configure --install-deps --source-deps-only --disable-lz4-ext --enable-static --enable-strip --disable-gssapi --prefix=' + releaseDir + ' --libdir=' + releaseDir, { + process.stderr.write("Running: " + command + 'on working directory = ' + baseDir + '\n'); + childProcess.execSync(command, { cwd: baseDir, stdio: [0,1,2] }); From 96add1bf372f4eb058ffae9cf26564b7ec5c426b Mon Sep 17 00:00:00 2001 From: Milind L Date: Wed, 13 Dec 2023 17:57:29 +0530 Subject: [PATCH 21/26] Bump version --- .semaphore/semaphore.yml | 302 +++++++++++++++++++-------------------- package.json | 2 +- util/configure.js | 2 +- 3 files changed, 153 insertions(+), 153 deletions(-) diff --git a/.semaphore/semaphore.yml b/.semaphore/semaphore.yml index 0d07da03..4cc1d131 100644 --- a/.semaphore/semaphore.yml +++ b/.semaphore/semaphore.yml @@ -35,29 +35,29 @@ global_job_config: blocks: - - name: "Linux arm64: Build and test" - dependencies: [ ] - task: - agent: - machine: - type: s1-prod-ubuntu20-04-arm64-1 - jobs: - - name: "Build from source and test" - commands: - - npm install # this will actually not build anything if we have a release, but rather, fetch things using node-pre-gyp - so change this later. - - make test + # - name: "Linux arm64: Build and test" + # dependencies: [ ] + # task: + # agent: + # machine: + # type: s1-prod-ubuntu20-04-arm64-1 + # jobs: + # - name: "Build from source and test" + # commands: + # - npm install # this will actually not build anything if we have a release, but rather, fetch things using node-pre-gyp - so change this later. + # - make test - - name: 'macOS arm64/m1: Build and test' - dependencies: [] - task: - agent: - machine: - type: s1-prod-macos-arm64 - jobs: - - name: 'Build from source and test' - commands: - - npm install # this will actually not build anything if we have a release, but rather, fetch things using node-pre-gyp - so change this later. - - make test + # - name: 'macOS arm64/m1: Build and test' + # dependencies: [] + # task: + # agent: + # machine: + # type: s1-prod-macos-arm64 + # jobs: + # - name: 'Build from source and test' + # commands: + # - npm install # this will actually not build anything if we have a release, but rather, fetch things using node-pre-gyp - so change this later. + # - make test - name: "Linux amd64: Build, test, lint" dependencies: [ ] @@ -77,133 +77,133 @@ blocks: - npx eslint lib/kafkajs - - name: "Linux amd64: Release" - dependencies: [ ] - run: - when: "tag =~ '^v[0-9]\\.'" - task: - agent: - machine: - type: s1-prod-ubuntu20-04-amd64-2 - env_vars: - - name: ARCHITECTURE - value: "x64" - - name: PLATFORM - value: "linux" - - name: LIBC - value: "glibc" - jobs: - - name: "Release: LTS:18" - commands: - - sem-version node 18.19.0 - - export NODE_ABI=108 - - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - - npx node-pre-gyp package - - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} - - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - - name: "Release: LTS:20" - commands: - - sem-version node 20.10.0 - - export NODE_ABI=115 - - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - - npx node-pre-gyp package - - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} - - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - - name: "Release: latest: 21" - commands: - - sem-version node 21.4.0 - - export NODE_ABI=120 - - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - - npx node-pre-gyp package - - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} - - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" + # - name: "Linux amd64: Release" + # dependencies: [ ] + # run: + # when: "tag =~ '^v[0-9]\\.'" + # task: + # agent: + # machine: + # type: s1-prod-ubuntu20-04-amd64-2 + # env_vars: + # - name: ARCHITECTURE + # value: "x64" + # - name: PLATFORM + # value: "linux" + # - name: LIBC + # value: "glibc" + # jobs: + # - name: "Release: LTS:18" + # commands: + # - sem-version node 18.19.0 + # - export NODE_ABI=108 + # - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + # - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. + # - npx node-pre-gyp package + # - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} + # - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" + # - name: "Release: LTS:20" + # commands: + # - sem-version node 20.10.0 + # - export NODE_ABI=115 + # - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + # - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. + # - npx node-pre-gyp package + # - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} + # - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" + # - name: "Release: latest: 21" + # commands: + # - sem-version node 21.4.0 + # - export NODE_ABI=120 + # - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + # - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. + # - npx node-pre-gyp package + # - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} + # - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - - name: "Linux arm64: Release" - dependencies: [ ] - run: - when: "tag =~ '^v[0-9]\\.'" - task: - agent: - machine: - type: s1-prod-ubuntu20-04-arm64-1 - env_vars: - - name: ARCHITECTURE - value: "arm64" - - name: PLATFORM - value: "linux" - - name: LIBC - value: "glibc" - jobs: - - name: "Release: LTS:18" - commands: - - sem-version node 18.19.0 - - export NODE_ABI=108 - - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - - npx node-pre-gyp package - - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} - - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - - name: "Release: LTS:20" - commands: - - sem-version node 20.10.0 - - export NODE_ABI=115 - - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - - npx node-pre-gyp package - - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} - - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - - name: "Release: latest: 21" - commands: - - sem-version node 21.4.0 - - export NODE_ABI=120 - - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - - npx node-pre-gyp package - - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} - - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - - name: "macOS arm64/m1: Release" - dependencies: [ ] - run: - when: "tag =~ '^v[0-9]\\.'" - task: - agent: - machine: - type: s1-prod-macos-arm64 - env_vars: - - name: ARCHITECTURE - value: "arm64" - - name: PLATFORM - value: "darwin" - - name: LIBC - value: "unknown" - jobs: - - name: "Release: LTS:18" - commands: - - sem-version node 18.19.0 - - export NODE_ABI=108 - - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - - npx node-pre-gyp package - - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} - - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - - name: "Release: LTS:20" - commands: - - sem-version node 20.10.0 - - export NODE_ABI=115 - - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - - npx node-pre-gyp package - - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} - - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - - name: "Release: latest: 21" - commands: - - sem-version node 21.4.0 - - export NODE_ABI=120 - - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - - npx node-pre-gyp package - - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} - - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" + # - name: "Linux arm64: Release" + # dependencies: [ ] + # run: + # when: "tag =~ '^v[0-9]\\.'" + # task: + # agent: + # machine: + # type: s1-prod-ubuntu20-04-arm64-1 + # env_vars: + # - name: ARCHITECTURE + # value: "arm64" + # - name: PLATFORM + # value: "linux" + # - name: LIBC + # value: "glibc" + # jobs: + # - name: "Release: LTS:18" + # commands: + # - sem-version node 18.19.0 + # - export NODE_ABI=108 + # - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + # - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. + # - npx node-pre-gyp package + # - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} + # - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" + # - name: "Release: LTS:20" + # commands: + # - sem-version node 20.10.0 + # - export NODE_ABI=115 + # - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + # - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. + # - npx node-pre-gyp package + # - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} + # - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" + # - name: "Release: latest: 21" + # commands: + # - sem-version node 21.4.0 + # - export NODE_ABI=120 + # - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + # - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. + # - npx node-pre-gyp package + # - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} + # - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" + # - name: "macOS arm64/m1: Release" + # dependencies: [ ] + # run: + # when: "tag =~ '^v[0-9]\\.'" + # task: + # agent: + # machine: + # type: s1-prod-macos-arm64 + # env_vars: + # - name: ARCHITECTURE + # value: "arm64" + # - name: PLATFORM + # value: "darwin" + # - name: LIBC + # value: "unknown" + # jobs: + # - name: "Release: LTS:18" + # commands: + # - sem-version node 18.19.0 + # - export NODE_ABI=108 + # - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + # - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. + # - npx node-pre-gyp package + # - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} + # - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" + # - name: "Release: LTS:20" + # commands: + # - sem-version node 20.10.0 + # - export NODE_ABI=115 + # - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + # - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. + # - npx node-pre-gyp package + # - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} + # - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" + # - name: "Release: latest: 21" + # commands: + # - sem-version node 21.4.0 + # - export NODE_ABI=120 + # - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + # - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. + # - npx node-pre-gyp package + # - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} + # - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" diff --git a/package.json b/package.json index 6773d31d..8265bf00 100644 --- a/package.json +++ b/package.json @@ -1,6 +1,6 @@ { "name": "confluent-kafka-js", - "version": "v0.1.2-devel", + "version": "v0.1.3-devel", "description": "Node.js bindings for librdkafka", "librdkafka": "2.3.0", "main": "lib/index.js", diff --git a/util/configure.js b/util/configure.js index dfedc223..2f25b5ba 100644 --- a/util/configure.js +++ b/util/configure.js @@ -18,7 +18,7 @@ if (isWin) { var childProcess = require('child_process'); try { - let opts = '--install-deps --source-deps-only --disable-lz4-ext --enable-static --enable-strip --disable-gssapi'; + let opts = '--install-deps --source-deps-only --disable-lz4-ext --enable-static --enable-strip --disable-gssapi --disable-ssl --disable-curl'; if (process.env['CKJS_LINKING'] === 'dynamic') opts = ''; childProcess.execSync(`./configure ${opts} --prefix=${releaseDir} --libdir=${releaseDir}`, { From a92ae5833e56a83374874d82ea28373be2b522e4 Mon Sep 17 00:00:00 2001 From: Milind L Date: Thu, 14 Dec 2023 10:14:34 +0530 Subject: [PATCH 22/26] Remove -j flag --- .semaphore/semaphore.yml | 302 +++++++++++++++++++-------------------- deps/librdkafka | 2 +- util/configure.js | 2 +- 3 files changed, 153 insertions(+), 153 deletions(-) diff --git a/.semaphore/semaphore.yml b/.semaphore/semaphore.yml index 4cc1d131..8eca7fe0 100644 --- a/.semaphore/semaphore.yml +++ b/.semaphore/semaphore.yml @@ -35,29 +35,29 @@ global_job_config: blocks: - # - name: "Linux arm64: Build and test" - # dependencies: [ ] - # task: - # agent: - # machine: - # type: s1-prod-ubuntu20-04-arm64-1 - # jobs: - # - name: "Build from source and test" - # commands: - # - npm install # this will actually not build anything if we have a release, but rather, fetch things using node-pre-gyp - so change this later. - # - make test + - name: "Linux arm64: Build and test" + dependencies: [ ] + task: + agent: + machine: + type: s1-prod-ubuntu20-04-arm64-1 + jobs: + - name: "Build from source and test" + commands: + - npm install # this will actually not build anything if we have a release, but rather, fetch things using node-pre-gyp - so change this later. + - make test - # - name: 'macOS arm64/m1: Build and test' - # dependencies: [] - # task: - # agent: - # machine: - # type: s1-prod-macos-arm64 - # jobs: - # - name: 'Build from source and test' - # commands: - # - npm install # this will actually not build anything if we have a release, but rather, fetch things using node-pre-gyp - so change this later. - # - make test + - name: 'macOS arm64/m1: Build and test' + dependencies: [] + task: + agent: + machine: + type: s1-prod-macos-arm64 + jobs: + - name: 'Build from source and test' + commands: + - npm install # this will actually not build anything if we have a release, but rather, fetch things using node-pre-gyp - so change this later. + - make test - name: "Linux amd64: Build, test, lint" dependencies: [ ] @@ -77,133 +77,133 @@ blocks: - npx eslint lib/kafkajs - # - name: "Linux amd64: Release" - # dependencies: [ ] - # run: - # when: "tag =~ '^v[0-9]\\.'" - # task: - # agent: - # machine: - # type: s1-prod-ubuntu20-04-amd64-2 - # env_vars: - # - name: ARCHITECTURE - # value: "x64" - # - name: PLATFORM - # value: "linux" - # - name: LIBC - # value: "glibc" - # jobs: - # - name: "Release: LTS:18" - # commands: - # - sem-version node 18.19.0 - # - export NODE_ABI=108 - # - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - # - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - # - npx node-pre-gyp package - # - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} - # - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - # - name: "Release: LTS:20" - # commands: - # - sem-version node 20.10.0 - # - export NODE_ABI=115 - # - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - # - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - # - npx node-pre-gyp package - # - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} - # - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - # - name: "Release: latest: 21" - # commands: - # - sem-version node 21.4.0 - # - export NODE_ABI=120 - # - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - # - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - # - npx node-pre-gyp package - # - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} - # - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" + - name: "Linux amd64: Release" + dependencies: [ ] + run: + when: "tag =~ '^v[0-9]\\.'" + task: + agent: + machine: + type: s1-prod-ubuntu20-04-amd64-2 + env_vars: + - name: ARCHITECTURE + value: "x64" + - name: PLATFORM + value: "linux" + - name: LIBC + value: "glibc" + jobs: + - name: "Release: LTS:18" + commands: + - sem-version node 18.19.0 + - export NODE_ABI=108 + - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. + - npx node-pre-gyp package + - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} + - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" + - name: "Release: LTS:20" + commands: + - sem-version node 20.10.0 + - export NODE_ABI=115 + - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. + - npx node-pre-gyp package + - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} + - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" + - name: "Release: latest: 21" + commands: + - sem-version node 21.4.0 + - export NODE_ABI=120 + - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. + - npx node-pre-gyp package + - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} + - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - # - name: "Linux arm64: Release" - # dependencies: [ ] - # run: - # when: "tag =~ '^v[0-9]\\.'" - # task: - # agent: - # machine: - # type: s1-prod-ubuntu20-04-arm64-1 - # env_vars: - # - name: ARCHITECTURE - # value: "arm64" - # - name: PLATFORM - # value: "linux" - # - name: LIBC - # value: "glibc" - # jobs: - # - name: "Release: LTS:18" - # commands: - # - sem-version node 18.19.0 - # - export NODE_ABI=108 - # - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - # - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - # - npx node-pre-gyp package - # - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} - # - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - # - name: "Release: LTS:20" - # commands: - # - sem-version node 20.10.0 - # - export NODE_ABI=115 - # - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - # - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - # - npx node-pre-gyp package - # - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} - # - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - # - name: "Release: latest: 21" - # commands: - # - sem-version node 21.4.0 - # - export NODE_ABI=120 - # - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - # - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - # - npx node-pre-gyp package - # - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} - # - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - # - name: "macOS arm64/m1: Release" - # dependencies: [ ] - # run: - # when: "tag =~ '^v[0-9]\\.'" - # task: - # agent: - # machine: - # type: s1-prod-macos-arm64 - # env_vars: - # - name: ARCHITECTURE - # value: "arm64" - # - name: PLATFORM - # value: "darwin" - # - name: LIBC - # value: "unknown" - # jobs: - # - name: "Release: LTS:18" - # commands: - # - sem-version node 18.19.0 - # - export NODE_ABI=108 - # - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - # - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - # - npx node-pre-gyp package - # - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} - # - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - # - name: "Release: LTS:20" - # commands: - # - sem-version node 20.10.0 - # - export NODE_ABI=115 - # - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - # - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - # - npx node-pre-gyp package - # - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} - # - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" - # - name: "Release: latest: 21" - # commands: - # - sem-version node 21.4.0 - # - export NODE_ABI=120 - # - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - # - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - # - npx node-pre-gyp package - # - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} - # - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" + - name: "Linux arm64: Release" + dependencies: [ ] + run: + when: "tag =~ '^v[0-9]\\.'" + task: + agent: + machine: + type: s1-prod-ubuntu20-04-arm64-1 + env_vars: + - name: ARCHITECTURE + value: "arm64" + - name: PLATFORM + value: "linux" + - name: LIBC + value: "glibc" + jobs: + - name: "Release: LTS:18" + commands: + - sem-version node 18.19.0 + - export NODE_ABI=108 + - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. + - npx node-pre-gyp package + - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} + - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" + - name: "Release: LTS:20" + commands: + - sem-version node 20.10.0 + - export NODE_ABI=115 + - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. + - npx node-pre-gyp package + - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} + - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" + - name: "Release: latest: 21" + commands: + - sem-version node 21.4.0 + - export NODE_ABI=120 + - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. + - npx node-pre-gyp package + - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} + - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" + - name: "macOS arm64/m1: Release" + dependencies: [ ] + run: + when: "tag =~ '^v[0-9]\\.'" + task: + agent: + machine: + type: s1-prod-macos-arm64 + env_vars: + - name: ARCHITECTURE + value: "arm64" + - name: PLATFORM + value: "darwin" + - name: LIBC + value: "unknown" + jobs: + - name: "Release: LTS:18" + commands: + - sem-version node 18.19.0 + - export NODE_ABI=108 + - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. + - npx node-pre-gyp package + - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} + - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" + - name: "Release: LTS:20" + commands: + - sem-version node 20.10.0 + - export NODE_ABI=115 + - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. + - npx node-pre-gyp package + - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} + - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" + - name: "Release: latest: 21" + commands: + - sem-version node 21.4.0 + - export NODE_ABI=120 + - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. + - npx node-pre-gyp package + - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} + - artifact push project "build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" --destination "releases/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY}" diff --git a/deps/librdkafka b/deps/librdkafka index cc58c2cc..840ab459 160000 --- a/deps/librdkafka +++ b/deps/librdkafka @@ -1 +1 @@ -Subproject commit cc58c2cc9aa651e71fcaa25d69766b4fdd6de977 +Subproject commit 840ab45990d29914fbd5aa3649ba1c102340f1f8 diff --git a/util/configure.js b/util/configure.js index 2f25b5ba..dfedc223 100644 --- a/util/configure.js +++ b/util/configure.js @@ -18,7 +18,7 @@ if (isWin) { var childProcess = require('child_process'); try { - let opts = '--install-deps --source-deps-only --disable-lz4-ext --enable-static --enable-strip --disable-gssapi --disable-ssl --disable-curl'; + let opts = '--install-deps --source-deps-only --disable-lz4-ext --enable-static --enable-strip --disable-gssapi'; if (process.env['CKJS_LINKING'] === 'dynamic') opts = ''; childProcess.execSync(`./configure ${opts} --prefix=${releaseDir} --libdir=${releaseDir}`, { From 625f3b19d9f8e95c40e61b082a8ebaba6c34581e Mon Sep 17 00:00:00 2001 From: Milind L Date: Thu, 14 Dec 2023 10:38:13 +0530 Subject: [PATCH 23/26] Revert module name changes --- .semaphore/semaphore.yml | 18 +++++++++--------- binding.gyp | 2 +- deps/librdkafka.gyp | 2 +- librdkafka.js | 2 +- package.json | 2 +- test/binding.spec.js | 2 +- test/consumer.spec.js | 2 +- 7 files changed, 15 insertions(+), 15 deletions(-) diff --git a/.semaphore/semaphore.yml b/.semaphore/semaphore.yml index 8eca7fe0..0d07da03 100644 --- a/.semaphore/semaphore.yml +++ b/.semaphore/semaphore.yml @@ -97,7 +97,7 @@ blocks: commands: - sem-version node 18.19.0 - export NODE_ABI=108 - - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - npx node-pre-gyp package - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} @@ -106,7 +106,7 @@ blocks: commands: - sem-version node 20.10.0 - export NODE_ABI=115 - - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - npx node-pre-gyp package - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} @@ -115,7 +115,7 @@ blocks: commands: - sem-version node 21.4.0 - export NODE_ABI=120 - - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - npx node-pre-gyp package - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} @@ -141,7 +141,7 @@ blocks: commands: - sem-version node 18.19.0 - export NODE_ABI=108 - - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - npx node-pre-gyp package - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} @@ -150,7 +150,7 @@ blocks: commands: - sem-version node 20.10.0 - export NODE_ABI=115 - - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - npx node-pre-gyp package - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} @@ -159,7 +159,7 @@ blocks: commands: - sem-version node 21.4.0 - export NODE_ABI=120 - - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - npx node-pre-gyp package - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} @@ -184,7 +184,7 @@ blocks: commands: - sem-version node 18.19.0 - export NODE_ABI=108 - - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - npx node-pre-gyp package - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} @@ -193,7 +193,7 @@ blocks: commands: - sem-version node 20.10.0 - export NODE_ABI=115 - - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - npx node-pre-gyp package - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} @@ -202,7 +202,7 @@ blocks: commands: - sem-version node 21.4.0 - export NODE_ABI=120 - - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - npx node-pre-gyp package - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} diff --git a/binding.gyp b/binding.gyp index 9ec16668..9b38d4bb 100644 --- a/binding.gyp +++ b/binding.gyp @@ -7,7 +7,7 @@ }, "targets": [ { - "target_name": "confluent-kafka-js", + "target_name": "node-librdkafka", 'sources': [ 'src/binding.cc', 'src/callbacks.cc', diff --git a/deps/librdkafka.gyp b/deps/librdkafka.gyp index 3dc6ff48..6093d7b6 100644 --- a/deps/librdkafka.gyp +++ b/deps/librdkafka.gyp @@ -26,7 +26,7 @@ "librdkafka/config.h", ], "action": [ - "make", "-j5", "-C", "librdkafka", "libs", "install" + "make", "-C", "librdkafka", "libs", "install" ], "conditions": [ [ diff --git a/librdkafka.js b/librdkafka.js index 35e4e149..4a8b7200 100644 --- a/librdkafka.js +++ b/librdkafka.js @@ -7,6 +7,6 @@ * of the MIT license. See the LICENSE.txt file for details. */ -var kafka = require('bindings')('confluent-kafka-js'); +var kafka = require('bindings')('node-librdkafka'); module.exports = kafka; diff --git a/package.json b/package.json index 8265bf00..8ac23643 100644 --- a/package.json +++ b/package.json @@ -14,7 +14,7 @@ "test:types": "tsc -p ." }, "binary": { - "module_name": "confluent-kafka-js", + "module_name": "node-librdkafka", "module_path": "./build/{configuration}/", "package_name": "{module_name}-v{version}-{node_abi}-{platform}-{libc}-{arch}.tar.gz", "host": "https://github.com/confluentinc/confluent-kafka-js/releases/download/", diff --git a/test/binding.spec.js b/test/binding.spec.js index 466f3fe6..739c708f 100644 --- a/test/binding.spec.js +++ b/test/binding.spec.js @@ -7,7 +7,7 @@ * of the MIT license. See the LICENSE.txt file for details. */ -var addon = require('bindings')('confluent-kafka-js'); +var addon = require('bindings')('node-librdkafka'); var t = require('assert'); var consumerConfig = { diff --git a/test/consumer.spec.js b/test/consumer.spec.js index 45b1b17e..38d01f2e 100644 --- a/test/consumer.spec.js +++ b/test/consumer.spec.js @@ -7,7 +7,7 @@ * of the MIT license. See the LICENSE.txt file for details. */ -var addon = require('bindings')('confluent-kafka-js'); +var addon = require('bindings')('node-librdkafka'); var t = require('assert'); var client; From 9d91a32ae5d19e01dc9bff94aef6630cccd94d1f Mon Sep 17 00:00:00 2001 From: Milind L Date: Thu, 14 Dec 2023 13:39:03 +0530 Subject: [PATCH 24/26] Change gypfiles --- .semaphore/semaphore.yml | 18 ++++++++--------- binding.gyp | 42 +++++++++++++++++++++------------------- deps/librdkafka.gyp | 41 +++++++++++++++++++++++++++------------ librdkafka.js | 2 +- package.json | 2 +- test/binding.spec.js | 2 +- test/consumer.spec.js | 2 +- 7 files changed, 64 insertions(+), 45 deletions(-) diff --git a/.semaphore/semaphore.yml b/.semaphore/semaphore.yml index 0d07da03..8eca7fe0 100644 --- a/.semaphore/semaphore.yml +++ b/.semaphore/semaphore.yml @@ -97,7 +97,7 @@ blocks: commands: - sem-version node 18.19.0 - export NODE_ABI=108 - - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - npx node-pre-gyp package - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} @@ -106,7 +106,7 @@ blocks: commands: - sem-version node 20.10.0 - export NODE_ABI=115 - - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - npx node-pre-gyp package - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} @@ -115,7 +115,7 @@ blocks: commands: - sem-version node 21.4.0 - export NODE_ABI=120 - - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - npx node-pre-gyp package - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} @@ -141,7 +141,7 @@ blocks: commands: - sem-version node 18.19.0 - export NODE_ABI=108 - - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - npx node-pre-gyp package - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} @@ -150,7 +150,7 @@ blocks: commands: - sem-version node 20.10.0 - export NODE_ABI=115 - - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - npx node-pre-gyp package - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} @@ -159,7 +159,7 @@ blocks: commands: - sem-version node 21.4.0 - export NODE_ABI=120 - - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - npx node-pre-gyp package - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} @@ -184,7 +184,7 @@ blocks: commands: - sem-version node 18.19.0 - export NODE_ABI=108 - - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - npx node-pre-gyp package - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} @@ -193,7 +193,7 @@ blocks: commands: - sem-version node 20.10.0 - export NODE_ABI=115 - - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - npx node-pre-gyp package - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} @@ -202,7 +202,7 @@ blocks: commands: - sem-version node 21.4.0 - export NODE_ABI=120 - - export ARTIFACT_KEY="node-librdkafka-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" + - export ARTIFACT_KEY="confluent-kafka-js-${SEMAPHORE_GIT_TAG_NAME}-node-v${NODE_ABI}-${PLATFORM}-${LIBC}-${ARCHITECTURE}.tar.gz" - npm install # node-pre-gyp will fallback to build here, because new tag implies no release yet. - npx node-pre-gyp package - ls build/stage/${SEMAPHORE_GIT_TAG_NAME}/${ARTIFACT_KEY} diff --git a/binding.gyp b/binding.gyp index 9b38d4bb..258ff021 100644 --- a/binding.gyp +++ b/binding.gyp @@ -7,7 +7,7 @@ }, "targets": [ { - "target_name": "node-librdkafka", + "target_name": "confluent-kafka-js", 'sources': [ 'src/binding.cc', 'src/callbacks.cc', @@ -86,17 +86,28 @@ ], 'conditions': [ [ - ['OS=="linux"', 'CKJS_LINKING="dynamic"'], + 'CKJS_LINKING=="dynamic"', { - "libraries": [ - "../build/deps/librdkafka.so", - "../build/deps/librdkafka++.so", - "-Wl,-rpath='$$ORIGIN/../deps'", - ], - } - ], - [ - ['OS=="linux"', 'CKJS_LINKING!="dynamic"'], + "conditions": [ + [ + 'OS=="mac"', + { + "libraries": [ + "../build/deps/librdkafka.dylib", + "../build/deps/librdkafka++.dylib", + "-Wl,-rpath='$$ORIGIN/../deps'", + ], + }, + { + "libraries": [ + "../build/deps/librdkafka.so", + "../build/deps/librdkafka++.so", + "-Wl,-rpath='$$ORIGIN/../deps'", + ], + }, + ] + ] + }, { "libraries": [ "../build/deps/librdkafka-static.a", @@ -105,15 +116,6 @@ ], } ], - [ - 'OS=="mac"', - { - "libraries": [ - "../build/deps/librdkafka.dylib", - "../build/deps/librdkafka++.dylib", - ], - } - ] ], }, # Else link against globally installed rdkafka and use diff --git a/deps/librdkafka.gyp b/deps/librdkafka.gyp index 6093d7b6..4673ee1c 100644 --- a/deps/librdkafka.gyp +++ b/deps/librdkafka.gyp @@ -1,4 +1,7 @@ { + "variables": { + "CKJS_LINKING%": " Date: Thu, 14 Dec 2023 13:53:54 +0530 Subject: [PATCH 25/26] Just remove rpath --- binding.gyp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/binding.gyp b/binding.gyp index 258ff021..d018323b 100644 --- a/binding.gyp +++ b/binding.gyp @@ -95,14 +95,12 @@ "libraries": [ "../build/deps/librdkafka.dylib", "../build/deps/librdkafka++.dylib", - "-Wl,-rpath='$$ORIGIN/../deps'", ], }, { "libraries": [ "../build/deps/librdkafka.so", "../build/deps/librdkafka++.so", - "-Wl,-rpath='$$ORIGIN/../deps'", ], }, ] @@ -112,7 +110,7 @@ "libraries": [ "../build/deps/librdkafka-static.a", "../build/deps/librdkafka++.a", - "-Wl,-rpath='$$ORIGIN/../deps'", + "-Wl,-rpath,'$$ORIGIN/../deps'", ], } ], From fd2b46d6ad1cf01a3e2609651e98a1881de9701c Mon Sep 17 00:00:00 2001 From: Milind L Date: Thu, 14 Dec 2023 14:23:43 +0530 Subject: [PATCH 26/26] Change path, temporarily --- package-lock.json | 669 ++++++++++++++++++++++------------------------ package.json | 2 +- 2 files changed, 314 insertions(+), 357 deletions(-) diff --git a/package-lock.json b/package-lock.json index 0ef2c3a9..337bebea 100644 --- a/package-lock.json +++ b/package-lock.json @@ -1,12 +1,12 @@ { "name": "confluent-kafka-js", - "version": "v0.1.1-devel", + "version": "v0.1.3-devel", "lockfileVersion": 3, "requires": true, "packages": { "": { "name": "confluent-kafka-js", - "version": "v0.1.1-devel", + "version": "v0.1.3-devel", "hasInstallScript": true, "license": "MIT", "dependencies": { @@ -146,21 +146,21 @@ } }, "node_modules/@babel/core": { - "version": "7.23.5", - "resolved": "https://registry.npmjs.org/@babel/core/-/core-7.23.5.tgz", - "integrity": "sha512-Cwc2XjUrG4ilcfOw4wBAK+enbdgwAcAJCfGUItPBKR7Mjw4aEfAFYrLxeRp4jWgtNIKn3n2AlBOfwwafl+42/g==", + "version": "7.23.6", + "resolved": "https://registry.npmjs.org/@babel/core/-/core-7.23.6.tgz", + "integrity": "sha512-FxpRyGjrMJXh7X3wGLGhNDCRiwpWEF74sKjTLDJSG5Kyvow3QZaG0Adbqzi9ZrVjTWpsX+2cxWXD71NMg93kdw==", "dev": true, "dependencies": { "@ampproject/remapping": "^2.2.0", "@babel/code-frame": "^7.23.5", - "@babel/generator": "^7.23.5", - "@babel/helper-compilation-targets": "^7.22.15", + "@babel/generator": "^7.23.6", + "@babel/helper-compilation-targets": "^7.23.6", "@babel/helper-module-transforms": "^7.23.3", - "@babel/helpers": "^7.23.5", - "@babel/parser": "^7.23.5", + "@babel/helpers": "^7.23.6", + "@babel/parser": "^7.23.6", "@babel/template": "^7.22.15", - "@babel/traverse": "^7.23.5", - "@babel/types": "^7.23.5", + "@babel/traverse": "^7.23.6", + "@babel/types": "^7.23.6", "convert-source-map": "^2.0.0", "debug": "^4.1.0", "gensync": "^1.0.0-beta.2", @@ -185,12 +185,12 @@ } }, "node_modules/@babel/generator": { - "version": "7.23.5", - "resolved": "https://registry.npmjs.org/@babel/generator/-/generator-7.23.5.tgz", - "integrity": "sha512-BPssCHrBD+0YrxviOa3QzpqwhNIXKEtOa2jQrm4FlmkC2apYgRnQcmPWiGZDlGxiNtltnUFolMe8497Esry+jA==", + "version": "7.23.6", + "resolved": "https://registry.npmjs.org/@babel/generator/-/generator-7.23.6.tgz", + "integrity": "sha512-qrSfCYxYQB5owCmGLbl8XRpX1ytXlpueOb0N0UmQwA073KZxejgQTzAmJezxvpwQD9uGtK2shHdi55QT+MbjIw==", "dev": true, "dependencies": { - "@babel/types": "^7.23.5", + "@babel/types": "^7.23.6", "@jridgewell/gen-mapping": "^0.3.2", "@jridgewell/trace-mapping": "^0.3.17", "jsesc": "^2.5.1" @@ -200,14 +200,14 @@ } }, "node_modules/@babel/helper-compilation-targets": { - "version": "7.22.15", - "resolved": "https://registry.npmjs.org/@babel/helper-compilation-targets/-/helper-compilation-targets-7.22.15.tgz", - "integrity": "sha512-y6EEzULok0Qvz8yyLkCvVX+02ic+By2UdOhylwUOvOn9dvYc9mKICJuuU1n1XBI02YWsNsnrY1kc6DVbjcXbtw==", + "version": "7.23.6", + "resolved": "https://registry.npmjs.org/@babel/helper-compilation-targets/-/helper-compilation-targets-7.23.6.tgz", + "integrity": "sha512-9JB548GZoQVmzrFgp8o7KxdgkTGm6xs9DW0o/Pim72UDjzr5ObUQ6ZzYPqA+g9OTS2bBQoctLJrky0RDCAWRgQ==", "dev": true, "dependencies": { - "@babel/compat-data": "^7.22.9", - "@babel/helper-validator-option": "^7.22.15", - "browserslist": "^4.21.9", + "@babel/compat-data": "^7.23.5", + "@babel/helper-validator-option": "^7.23.5", + "browserslist": "^4.22.2", "lru-cache": "^5.1.1", "semver": "^6.3.1" }, @@ -350,14 +350,14 @@ } }, "node_modules/@babel/helpers": { - "version": "7.23.5", - "resolved": "https://registry.npmjs.org/@babel/helpers/-/helpers-7.23.5.tgz", - "integrity": "sha512-oO7us8FzTEsG3U6ag9MfdF1iA/7Z6dz+MtFhifZk8C8o453rGJFFWUP1t+ULM9TUIAzC9uxXEiXjOiVMyd7QPg==", + "version": "7.23.6", + "resolved": "https://registry.npmjs.org/@babel/helpers/-/helpers-7.23.6.tgz", + "integrity": "sha512-wCfsbN4nBidDRhpDhvcKlzHWCTlgJYUUdSJfzXb2NuBssDSIjc3xcb+znA7l+zYsFljAcGM0aFkN40cR3lXiGA==", "dev": true, "dependencies": { "@babel/template": "^7.22.15", - "@babel/traverse": "^7.23.5", - "@babel/types": "^7.23.5" + "@babel/traverse": "^7.23.6", + "@babel/types": "^7.23.6" }, "engines": { "node": ">=6.9.0" @@ -449,9 +449,9 @@ } }, "node_modules/@babel/parser": { - "version": "7.23.5", - "resolved": "https://registry.npmjs.org/@babel/parser/-/parser-7.23.5.tgz", - "integrity": "sha512-hOOqoiNXrmGdFbhgCzu6GiURxUgM27Xwd/aPuu8RfHEZPBzL1Z54okAHAQjXfcQNwvrlkAmAp4SlRTZ45vlthQ==", + "version": "7.23.6", + "resolved": "https://registry.npmjs.org/@babel/parser/-/parser-7.23.6.tgz", + "integrity": "sha512-Z2uID7YJ7oNvAI20O9X0bblw7Qqs8Q2hFy0R9tAfnfLkp5MW0UH9eUvnDSnFwKZ0AvgS1ucqR4KzvVHgnke1VQ==", "dev": true, "bin": { "parser": "bin/babel-parser.js" @@ -652,30 +652,39 @@ } }, "node_modules/@babel/traverse": { - "version": "7.23.5", - "resolved": "https://registry.npmjs.org/@babel/traverse/-/traverse-7.23.5.tgz", - "integrity": "sha512-czx7Xy5a6sapWWRx61m1Ke1Ra4vczu1mCTtJam5zRTBOonfdJ+S/B6HYmGYu3fJtr8GGET3si6IhgWVBhJ/m8w==", + "version": "7.23.6", + "resolved": "https://registry.npmjs.org/@babel/traverse/-/traverse-7.23.6.tgz", + "integrity": "sha512-czastdK1e8YByZqezMPFiZ8ahwVMh/ESl9vPgvgdB9AmFMGP5jfpFax74AQgl5zj4XHzqeYAg2l8PuUeRS1MgQ==", "dev": true, "dependencies": { "@babel/code-frame": "^7.23.5", - "@babel/generator": "^7.23.5", + "@babel/generator": "^7.23.6", "@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.5", - "@babel/types": "^7.23.5", - "debug": "^4.1.0", + "@babel/parser": "^7.23.6", + "@babel/types": "^7.23.6", + "debug": "^4.3.1", "globals": "^11.1.0" }, "engines": { "node": ">=6.9.0" } }, + "node_modules/@babel/traverse/node_modules/globals": { + "version": "11.12.0", + "resolved": "https://registry.npmjs.org/globals/-/globals-11.12.0.tgz", + "integrity": "sha512-WOBp/EEGUiIsJSp7wcv/y6MO+lV9UoncWqxuFfm8eBwzWNgyfBd6Gz+IeKQ9jCmyhoH99g15M3T+QaVHFjizVA==", + "dev": true, + "engines": { + "node": ">=4" + } + }, "node_modules/@babel/types": { - "version": "7.23.5", - "resolved": "https://registry.npmjs.org/@babel/types/-/types-7.23.5.tgz", - "integrity": "sha512-ON5kSOJwVO6xXVRTvOI0eOnWe7VdUcIpsovGo9U/Br4Ie4UVFQTboO2cYnDhAGU6Fp+UxSiT+pMft0SMHfuq6w==", + "version": "7.23.6", + "resolved": "https://registry.npmjs.org/@babel/types/-/types-7.23.6.tgz", + "integrity": "sha512-+uarb83brBzPKN38NX1MkB6vb6+mwvR6amUulqAE7ccQw1pEl+bCia9TbdG1lsnFP7lZySvUn37CHyXQdfTwzg==", "dev": true, "dependencies": { "@babel/helper-string-parser": "^7.23.4", @@ -739,51 +748,6 @@ "url": "https://opencollective.com/eslint" } }, - "node_modules/@eslint/eslintrc/node_modules/argparse": { - "version": "2.0.1", - "resolved": "https://registry.npmjs.org/argparse/-/argparse-2.0.1.tgz", - "integrity": "sha512-8+9WqebbFzpX9OR+Wa6O29asIogeRMzcGtAINdpMHHyAg10f05aSFVBbcEqGf/PXw1EjAZ+q2/bEBg3DvurK3Q==", - "dev": true - }, - "node_modules/@eslint/eslintrc/node_modules/globals": { - "version": "13.24.0", - "resolved": "https://registry.npmjs.org/globals/-/globals-13.24.0.tgz", - "integrity": "sha512-AhO5QUcj8llrbG09iWhPU2B204J1xnPeL8kQmVorSsy+Sjj1sk8gIyh6cUocGmH4L0UuhAJy+hJMRA4mgA4mFQ==", - "dev": true, - "dependencies": { - "type-fest": "^0.20.2" - }, - "engines": { - "node": ">=8" - }, - "funding": { - "url": "https://github.com/sponsors/sindresorhus" - } - }, - "node_modules/@eslint/eslintrc/node_modules/js-yaml": { - "version": "4.1.0", - "resolved": "https://registry.npmjs.org/js-yaml/-/js-yaml-4.1.0.tgz", - "integrity": "sha512-wpxZs9NoxZaJESJGIZTyDEaYpl0FKSA+FB9aJiyemKhMwkxQg63h4T1KJgUGHpTqPDNRcmmYLugrRjJlBtWvRA==", - "dev": true, - "dependencies": { - "argparse": "^2.0.1" - }, - "bin": { - "js-yaml": "bin/js-yaml.js" - } - }, - "node_modules/@eslint/eslintrc/node_modules/type-fest": { - "version": "0.20.2", - "resolved": "https://registry.npmjs.org/type-fest/-/type-fest-0.20.2.tgz", - "integrity": "sha512-Ne+eE4r0/iWnpAxD852z3A+N0Bt5RN//NjJwRd2VFHEmrywxf5vsZlh4R6lixl6B+wz/8d+maTSAkN1FIkI3LQ==", - "dev": true, - "engines": { - "node": ">=10" - }, - "funding": { - "url": "https://github.com/sponsors/sindresorhus" - } - }, "node_modules/@eslint/js": { "version": "8.54.0", "resolved": "https://registry.npmjs.org/@eslint/js/-/js-8.54.0.tgz", @@ -848,6 +812,89 @@ "node": ">=8" } }, + "node_modules/@istanbuljs/load-nyc-config/node_modules/argparse": { + "version": "1.0.10", + "resolved": "https://registry.npmjs.org/argparse/-/argparse-1.0.10.tgz", + "integrity": "sha512-o5Roy6tNG4SL/FOkCAN6RzjiakZS25RLYFrcMttJqbdd8BWrnA+fGz57iN5Pb06pvBGvl5gQ0B48dJlslXvoTg==", + "dev": true, + "dependencies": { + "sprintf-js": "~1.0.2" + } + }, + "node_modules/@istanbuljs/load-nyc-config/node_modules/find-up": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/find-up/-/find-up-4.1.0.tgz", + "integrity": "sha512-PpOwAdQ/YlXQ2vj8a3h8IipDuYRi3wceVQQGYWxNINccq40Anw7BlsEXCMbt1Zt+OLA6Fq9suIpIWD0OsnISlw==", + "dev": true, + "dependencies": { + "locate-path": "^5.0.0", + "path-exists": "^4.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/@istanbuljs/load-nyc-config/node_modules/js-yaml": { + "version": "3.14.1", + "resolved": "https://registry.npmjs.org/js-yaml/-/js-yaml-3.14.1.tgz", + "integrity": "sha512-okMH7OXXJ7YrN9Ok3/SXrnu4iX9yOk+25nqX4imS2npuvTYDmo/QEZoqwZkYaIDk3jVvBOTOIEgEhaLOynBS9g==", + "dev": true, + "dependencies": { + "argparse": "^1.0.7", + "esprima": "^4.0.0" + }, + "bin": { + "js-yaml": "bin/js-yaml.js" + } + }, + "node_modules/@istanbuljs/load-nyc-config/node_modules/locate-path": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/locate-path/-/locate-path-5.0.0.tgz", + "integrity": "sha512-t7hw9pI+WvuwNJXwk5zVHpyhIqzg2qTlklJOf0mVxGSbe3Fp2VieZcduNYjaLDoy6p9uGpQEGWG87WpMKlNq8g==", + "dev": true, + "dependencies": { + "p-locate": "^4.1.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/@istanbuljs/load-nyc-config/node_modules/p-limit": { + "version": "2.3.0", + "resolved": "https://registry.npmjs.org/p-limit/-/p-limit-2.3.0.tgz", + "integrity": "sha512-//88mFWSJx8lxCzwdAABTJL2MyWB12+eIY7MDL2SqLmAkeKU9qxRvWuSyTjm3FUmpBEMuFfckAIqEaVGUDxb6w==", + "dev": true, + "dependencies": { + "p-try": "^2.0.0" + }, + "engines": { + "node": ">=6" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/@istanbuljs/load-nyc-config/node_modules/p-locate": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/p-locate/-/p-locate-4.1.0.tgz", + "integrity": "sha512-R79ZZ/0wAxKGu3oYMlz8jy/kbhsNrS7SKZ7PxEHBgJ5+F2mtFW2fK2cOtBh1cHYkQsbzFV7I+EoRKe6Yt0oK7A==", + "dev": true, + "dependencies": { + "p-limit": "^2.2.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/@istanbuljs/load-nyc-config/node_modules/resolve-from": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/resolve-from/-/resolve-from-5.0.0.tgz", + "integrity": "sha512-qYg9KP24dD5qka9J47d0aVky0N+b4fTU89LN9iDnjB5waksiC49rvMB0PrUJQGoTmH50XPiqOvAjDfaijGxYZw==", + "dev": true, + "engines": { + "node": ">=8" + } + }, "node_modules/@istanbuljs/schema": { "version": "0.1.3", "resolved": "https://registry.npmjs.org/@istanbuljs/schema/-/schema-0.1.3.tgz", @@ -1531,6 +1578,18 @@ "url": "https://github.com/sponsors/sindresorhus" } }, + "node_modules/ansi-escapes/node_modules/type-fest": { + "version": "0.21.3", + "resolved": "https://registry.npmjs.org/type-fest/-/type-fest-0.21.3.tgz", + "integrity": "sha512-t0rzBq87m3fVcduHDUFhKmyyX+9eo6WQjZvf51Ea/M0Q7+T374Jp1aUiyUl0GKxp8M/OETVHSDvmkyPgvX+X2w==", + "dev": true, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, "node_modules/ansi-regex": { "version": "5.0.1", "resolved": "https://registry.npmjs.org/ansi-regex/-/ansi-regex-5.0.1.tgz", @@ -1606,13 +1665,10 @@ } }, "node_modules/argparse": { - "version": "1.0.10", - "resolved": "https://registry.npmjs.org/argparse/-/argparse-1.0.10.tgz", - "integrity": "sha512-o5Roy6tNG4SL/FOkCAN6RzjiakZS25RLYFrcMttJqbdd8BWrnA+fGz57iN5Pb06pvBGvl5gQ0B48dJlslXvoTg==", - "dev": true, - "dependencies": { - "sprintf-js": "~1.0.2" - } + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/argparse/-/argparse-2.0.1.tgz", + "integrity": "sha512-8+9WqebbFzpX9OR+Wa6O29asIogeRMzcGtAINdpMHHyAg10f05aSFVBbcEqGf/PXw1EjAZ+q2/bEBg3DvurK3Q==", + "dev": true }, "node_modules/babel-jest": { "version": "29.7.0", @@ -1929,9 +1985,9 @@ } }, "node_modules/caniuse-lite": { - "version": "1.0.30001566", - "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001566.tgz", - "integrity": "sha512-ggIhCsTxmITBAMmK8yZjEhCO5/47jKXPu6Dha/wuCS4JePVL+3uiDEBuhu2aIoT+bqTOR8L76Ip1ARL9xYsEJA==", + "version": "1.0.30001570", + "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001570.tgz", + "integrity": "sha512-+3e0ASu4sw1SWaoCtvPeyXp+5PsjigkSt8OXZbF9StH5pQWbxEjLAZE3n8Aup5udop1uRiKA7a4utUk/uoSpUw==", "dev": true, "funding": [ { @@ -2012,6 +2068,18 @@ "fsevents": "~2.3.2" } }, + "node_modules/chokidar/node_modules/glob-parent": { + "version": "5.1.2", + "resolved": "https://registry.npmjs.org/glob-parent/-/glob-parent-5.1.2.tgz", + "integrity": "sha512-AOIgSQCepiJYwP3ARnGx+5VnTu2HBYdzbGP45eLw1vr3zB3vZLeyed1sC9hnbcOc9/SrMyM5RPQrkGz4aS9Zow==", + "dev": true, + "dependencies": { + "is-glob": "^4.0.1" + }, + "engines": { + "node": ">= 6" + } + }, "node_modules/chownr": { "version": "2.0.0", "resolved": "https://registry.npmjs.org/chownr/-/chownr-2.0.0.tgz", @@ -2357,9 +2425,9 @@ } }, "node_modules/electron-to-chromium": { - "version": "1.4.609", - "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.609.tgz", - "integrity": "sha512-ihiCP7PJmjoGNuLpl7TjNA8pCQWu09vGyjlPYw1Rqww4gvNuCcmvl+44G+2QyJ6S2K4o+wbTS++Xz0YN8Q9ERw==", + "version": "1.4.612", + "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.612.tgz", + "integrity": "sha512-dM8BMtXtlH237ecSMnYdYuCkib2QHq0kpWfUnavjdYsyr/6OsAwg5ZGUfnQ9KD1Ga4QgB2sqXlB2NT8zy2GnVg==", "dev": true }, "node_modules/emittery": { @@ -2428,12 +2496,15 @@ } }, "node_modules/escape-string-regexp": { - "version": "2.0.0", - "resolved": "https://registry.npmjs.org/escape-string-regexp/-/escape-string-regexp-2.0.0.tgz", - "integrity": "sha512-UpzcLCXolUWcNu5HtVMHYdXJjArjsF9C0aNnquZYY4uW/Vu0miy5YoWvbV345HauVvcAUnpRuhMMcqTcGOY2+w==", + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/escape-string-regexp/-/escape-string-regexp-4.0.0.tgz", + "integrity": "sha512-TtpcNJ3XAzx3Gq8sWRzJaVajRs0uVxA2YAkdb1jm2YkPz4G6egUFAyA3n5vtEIZefPk5Wa4UXbKuS5fKkJWdgA==", "dev": true, "engines": { - "node": ">=8" + "node": ">=10" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" } }, "node_modules/eslint": { @@ -2509,129 +2580,14 @@ }, "node_modules/eslint-visitor-keys": { "version": "3.4.3", - "resolved": "https://registry.npmjs.org/eslint-visitor-keys/-/eslint-visitor-keys-3.4.3.tgz", - "integrity": "sha512-wpc+LXeiyiisxPlEkUzU6svyS1frIO3Mgxj1fdy7Pm8Ygzguax2N3Fa/D/ag1WqbOprdI+uY6wMUl8/a2G+iag==", - "dev": true, - "engines": { - "node": "^12.22.0 || ^14.17.0 || >=16.0.0" - }, - "funding": { - "url": "https://opencollective.com/eslint" - } - }, - "node_modules/eslint/node_modules/argparse": { - "version": "2.0.1", - "resolved": "https://registry.npmjs.org/argparse/-/argparse-2.0.1.tgz", - "integrity": "sha512-8+9WqebbFzpX9OR+Wa6O29asIogeRMzcGtAINdpMHHyAg10f05aSFVBbcEqGf/PXw1EjAZ+q2/bEBg3DvurK3Q==", - "dev": true - }, - "node_modules/eslint/node_modules/escape-string-regexp": { - "version": "4.0.0", - "resolved": "https://registry.npmjs.org/escape-string-regexp/-/escape-string-regexp-4.0.0.tgz", - "integrity": "sha512-TtpcNJ3XAzx3Gq8sWRzJaVajRs0uVxA2YAkdb1jm2YkPz4G6egUFAyA3n5vtEIZefPk5Wa4UXbKuS5fKkJWdgA==", - "dev": true, - "engines": { - "node": ">=10" - }, - "funding": { - "url": "https://github.com/sponsors/sindresorhus" - } - }, - "node_modules/eslint/node_modules/find-up": { - "version": "5.0.0", - "resolved": "https://registry.npmjs.org/find-up/-/find-up-5.0.0.tgz", - "integrity": "sha512-78/PXT1wlLLDgTzDs7sjq9hzz0vXD+zn+7wypEe4fXQxCmdmqfGsEPQxmiCSQI3ajFV91bVSsvNtrJRiW6nGng==", - "dev": true, - "dependencies": { - "locate-path": "^6.0.0", - "path-exists": "^4.0.0" - }, - "engines": { - "node": ">=10" - }, - "funding": { - "url": "https://github.com/sponsors/sindresorhus" - } - }, - "node_modules/eslint/node_modules/glob-parent": { - "version": "6.0.2", - "resolved": "https://registry.npmjs.org/glob-parent/-/glob-parent-6.0.2.tgz", - "integrity": "sha512-XxwI8EOhVQgWp6iDL+3b0r86f4d6AX6zSU55HfB4ydCEuXLXc5FcYeOu+nnGftS4TEju/11rt4KJPTMgbfmv4A==", - "dev": true, - "dependencies": { - "is-glob": "^4.0.3" - }, - "engines": { - "node": ">=10.13.0" - } - }, - "node_modules/eslint/node_modules/globals": { - "version": "13.24.0", - "resolved": "https://registry.npmjs.org/globals/-/globals-13.24.0.tgz", - "integrity": "sha512-AhO5QUcj8llrbG09iWhPU2B204J1xnPeL8kQmVorSsy+Sjj1sk8gIyh6cUocGmH4L0UuhAJy+hJMRA4mgA4mFQ==", - "dev": true, - "dependencies": { - "type-fest": "^0.20.2" - }, - "engines": { - "node": ">=8" - }, - "funding": { - "url": "https://github.com/sponsors/sindresorhus" - } - }, - "node_modules/eslint/node_modules/js-yaml": { - "version": "4.1.0", - "resolved": "https://registry.npmjs.org/js-yaml/-/js-yaml-4.1.0.tgz", - "integrity": "sha512-wpxZs9NoxZaJESJGIZTyDEaYpl0FKSA+FB9aJiyemKhMwkxQg63h4T1KJgUGHpTqPDNRcmmYLugrRjJlBtWvRA==", - "dev": true, - "dependencies": { - "argparse": "^2.0.1" - }, - "bin": { - "js-yaml": "bin/js-yaml.js" - } - }, - "node_modules/eslint/node_modules/locate-path": { - "version": "6.0.0", - "resolved": "https://registry.npmjs.org/locate-path/-/locate-path-6.0.0.tgz", - "integrity": "sha512-iPZK6eYjbxRu3uB4/WZ3EsEIMJFMqAoopl3R+zuq0UjcAm/MO6KCweDgPfP3elTztoKP3KtnVHxTn2NHBSDVUw==", - "dev": true, - "dependencies": { - "p-locate": "^5.0.0" - }, - "engines": { - "node": ">=10" - }, - "funding": { - "url": "https://github.com/sponsors/sindresorhus" - } - }, - "node_modules/eslint/node_modules/p-locate": { - "version": "5.0.0", - "resolved": "https://registry.npmjs.org/p-locate/-/p-locate-5.0.0.tgz", - "integrity": "sha512-LaNjtRWUBY++zB5nE/NwcaoMylSPk+S+ZHNB1TzdbMJMny6dynpAGt7X/tl/QYq3TIeE6nxHppbo2LGymrG5Pw==", - "dev": true, - "dependencies": { - "p-limit": "^3.0.2" - }, - "engines": { - "node": ">=10" - }, - "funding": { - "url": "https://github.com/sponsors/sindresorhus" - } - }, - "node_modules/eslint/node_modules/type-fest": { - "version": "0.20.2", - "resolved": "https://registry.npmjs.org/type-fest/-/type-fest-0.20.2.tgz", - "integrity": "sha512-Ne+eE4r0/iWnpAxD852z3A+N0Bt5RN//NjJwRd2VFHEmrywxf5vsZlh4R6lixl6B+wz/8d+maTSAkN1FIkI3LQ==", + "resolved": "https://registry.npmjs.org/eslint-visitor-keys/-/eslint-visitor-keys-3.4.3.tgz", + "integrity": "sha512-wpc+LXeiyiisxPlEkUzU6svyS1frIO3Mgxj1fdy7Pm8Ygzguax2N3Fa/D/ag1WqbOprdI+uY6wMUl8/a2G+iag==", "dev": true, "engines": { - "node": ">=10" + "node": "^12.22.0 || ^14.17.0 || >=16.0.0" }, "funding": { - "url": "https://github.com/sponsors/sindresorhus" + "url": "https://opencollective.com/eslint" } }, "node_modules/espree": { @@ -2826,16 +2782,19 @@ } }, "node_modules/find-up": { - "version": "4.1.0", - "resolved": "https://registry.npmjs.org/find-up/-/find-up-4.1.0.tgz", - "integrity": "sha512-PpOwAdQ/YlXQ2vj8a3h8IipDuYRi3wceVQQGYWxNINccq40Anw7BlsEXCMbt1Zt+OLA6Fq9suIpIWD0OsnISlw==", + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/find-up/-/find-up-5.0.0.tgz", + "integrity": "sha512-78/PXT1wlLLDgTzDs7sjq9hzz0vXD+zn+7wypEe4fXQxCmdmqfGsEPQxmiCSQI3ajFV91bVSsvNtrJRiW6nGng==", "dev": true, "dependencies": { - "locate-path": "^5.0.0", + "locate-path": "^6.0.0", "path-exists": "^4.0.0" }, "engines": { - "node": ">=8" + "node": ">=10" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" } }, "node_modules/flat": { @@ -2883,6 +2842,20 @@ "resolved": "https://registry.npmjs.org/fs.realpath/-/fs.realpath-1.0.0.tgz", "integrity": "sha512-OO0pH2lK6a0hZnAdau5ItzHPI6pUlvI7jMVnxUQRtw4owF2wk8lOSabtGDCTP4Ggrg2MbGnWO9X8K1t4+fGMDw==" }, + "node_modules/fsevents": { + "version": "2.3.3", + "resolved": "https://registry.npmjs.org/fsevents/-/fsevents-2.3.3.tgz", + "integrity": "sha512-5xoDfX+fL7faATnagmWPpbFtwh/R77WmMMqqHGS65C3vvB0YHrgF+B1YmZ3441tMj5n63k0212XNoJwzlhffQw==", + "dev": true, + "hasInstallScript": true, + "optional": true, + "os": [ + "darwin" + ], + "engines": { + "node": "^8.16.0 || ^10.6.0 || >=11.0.0" + } + }, "node_modules/function-bind": { "version": "1.1.2", "resolved": "https://registry.npmjs.org/function-bind/-/function-bind-1.1.2.tgz", @@ -2970,24 +2943,30 @@ } }, "node_modules/glob-parent": { - "version": "5.1.2", - "resolved": "https://registry.npmjs.org/glob-parent/-/glob-parent-5.1.2.tgz", - "integrity": "sha512-AOIgSQCepiJYwP3ARnGx+5VnTu2HBYdzbGP45eLw1vr3zB3vZLeyed1sC9hnbcOc9/SrMyM5RPQrkGz4aS9Zow==", + "version": "6.0.2", + "resolved": "https://registry.npmjs.org/glob-parent/-/glob-parent-6.0.2.tgz", + "integrity": "sha512-XxwI8EOhVQgWp6iDL+3b0r86f4d6AX6zSU55HfB4ydCEuXLXc5FcYeOu+nnGftS4TEju/11rt4KJPTMgbfmv4A==", "dev": true, "dependencies": { - "is-glob": "^4.0.1" + "is-glob": "^4.0.3" }, "engines": { - "node": ">= 6" + "node": ">=10.13.0" } }, "node_modules/globals": { - "version": "11.12.0", - "resolved": "https://registry.npmjs.org/globals/-/globals-11.12.0.tgz", - "integrity": "sha512-WOBp/EEGUiIsJSp7wcv/y6MO+lV9UoncWqxuFfm8eBwzWNgyfBd6Gz+IeKQ9jCmyhoH99g15M3T+QaVHFjizVA==", + "version": "13.24.0", + "resolved": "https://registry.npmjs.org/globals/-/globals-13.24.0.tgz", + "integrity": "sha512-AhO5QUcj8llrbG09iWhPU2B204J1xnPeL8kQmVorSsy+Sjj1sk8gIyh6cUocGmH4L0UuhAJy+hJMRA4mgA4mFQ==", "dev": true, + "dependencies": { + "type-fest": "^0.20.2" + }, "engines": { - "node": ">=4" + "node": ">=8" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" } }, "node_modules/graceful-fs": { @@ -3143,15 +3122,6 @@ "url": "https://github.com/sponsors/sindresorhus" } }, - "node_modules/import-fresh/node_modules/resolve-from": { - "version": "4.0.0", - "resolved": "https://registry.npmjs.org/resolve-from/-/resolve-from-4.0.0.tgz", - "integrity": "sha512-pb/MYmXstAkysRFx8piNI1tGFNQIFA3vkE3Gq4EuA1dF6gHp/+vgZqsCGJapvy8N3Q+4o7FwvquPJcnZ7RYy4g==", - "dev": true, - "engines": { - "node": ">=4" - } - }, "node_modules/import-local": { "version": "3.1.0", "resolved": "https://registry.npmjs.org/import-local/-/import-local-3.1.0.tgz", @@ -3995,13 +3965,12 @@ "dev": true }, "node_modules/js-yaml": { - "version": "3.14.1", - "resolved": "https://registry.npmjs.org/js-yaml/-/js-yaml-3.14.1.tgz", - "integrity": "sha512-okMH7OXXJ7YrN9Ok3/SXrnu4iX9yOk+25nqX4imS2npuvTYDmo/QEZoqwZkYaIDk3jVvBOTOIEgEhaLOynBS9g==", + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/js-yaml/-/js-yaml-4.1.0.tgz", + "integrity": "sha512-wpxZs9NoxZaJESJGIZTyDEaYpl0FKSA+FB9aJiyemKhMwkxQg63h4T1KJgUGHpTqPDNRcmmYLugrRjJlBtWvRA==", "dev": true, "dependencies": { - "argparse": "^1.0.7", - "esprima": "^4.0.0" + "argparse": "^2.0.1" }, "bin": { "js-yaml": "bin/js-yaml.js" @@ -4045,6 +4014,15 @@ "node": ">=12.0.0" } }, + "node_modules/jsdoc/node_modules/escape-string-regexp": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/escape-string-regexp/-/escape-string-regexp-2.0.0.tgz", + "integrity": "sha512-UpzcLCXolUWcNu5HtVMHYdXJjArjsF9C0aNnquZYY4uW/Vu0miy5YoWvbV345HauVvcAUnpRuhMMcqTcGOY2+w==", + "dev": true, + "engines": { + "node": ">=8" + } + }, "node_modules/jsesc": { "version": "2.5.2", "resolved": "https://registry.npmjs.org/jsesc/-/jsesc-2.5.2.tgz", @@ -4200,15 +4178,18 @@ } }, "node_modules/locate-path": { - "version": "5.0.0", - "resolved": "https://registry.npmjs.org/locate-path/-/locate-path-5.0.0.tgz", - "integrity": "sha512-t7hw9pI+WvuwNJXwk5zVHpyhIqzg2qTlklJOf0mVxGSbe3Fp2VieZcduNYjaLDoy6p9uGpQEGWG87WpMKlNq8g==", + "version": "6.0.0", + "resolved": "https://registry.npmjs.org/locate-path/-/locate-path-6.0.0.tgz", + "integrity": "sha512-iPZK6eYjbxRu3uB4/WZ3EsEIMJFMqAoopl3R+zuq0UjcAm/MO6KCweDgPfP3elTztoKP3KtnVHxTn2NHBSDVUw==", "dev": true, "dependencies": { - "p-locate": "^4.1.0" + "p-locate": "^5.0.0" }, "engines": { - "node": ">=8" + "node": ">=10" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" } }, "node_modules/lodash": { @@ -4341,12 +4322,6 @@ "markdown-it": "*" } }, - "node_modules/markdown-it/node_modules/argparse": { - "version": "2.0.1", - "resolved": "https://registry.npmjs.org/argparse/-/argparse-2.0.1.tgz", - "integrity": "sha512-8+9WqebbFzpX9OR+Wa6O29asIogeRMzcGtAINdpMHHyAg10f05aSFVBbcEqGf/PXw1EjAZ+q2/bEBg3DvurK3Q==", - "dev": true - }, "node_modules/markdown-it/node_modules/entities": { "version": "2.1.0", "resolved": "https://registry.npmjs.org/entities/-/entities-2.1.0.tgz", @@ -4562,12 +4537,6 @@ "url": "https://opencollective.com/mochajs" } }, - "node_modules/mocha/node_modules/argparse": { - "version": "2.0.1", - "resolved": "https://registry.npmjs.org/argparse/-/argparse-2.0.1.tgz", - "integrity": "sha512-8+9WqebbFzpX9OR+Wa6O29asIogeRMzcGtAINdpMHHyAg10f05aSFVBbcEqGf/PXw1EjAZ+q2/bEBg3DvurK3Q==", - "dev": true - }, "node_modules/mocha/node_modules/cliui": { "version": "7.0.4", "resolved": "https://registry.npmjs.org/cliui/-/cliui-7.0.4.tgz", @@ -4579,34 +4548,6 @@ "wrap-ansi": "^7.0.0" } }, - "node_modules/mocha/node_modules/escape-string-regexp": { - "version": "4.0.0", - "resolved": "https://registry.npmjs.org/escape-string-regexp/-/escape-string-regexp-4.0.0.tgz", - "integrity": "sha512-TtpcNJ3XAzx3Gq8sWRzJaVajRs0uVxA2YAkdb1jm2YkPz4G6egUFAyA3n5vtEIZefPk5Wa4UXbKuS5fKkJWdgA==", - "dev": true, - "engines": { - "node": ">=10" - }, - "funding": { - "url": "https://github.com/sponsors/sindresorhus" - } - }, - "node_modules/mocha/node_modules/find-up": { - "version": "5.0.0", - "resolved": "https://registry.npmjs.org/find-up/-/find-up-5.0.0.tgz", - "integrity": "sha512-78/PXT1wlLLDgTzDs7sjq9hzz0vXD+zn+7wypEe4fXQxCmdmqfGsEPQxmiCSQI3ajFV91bVSsvNtrJRiW6nGng==", - "dev": true, - "dependencies": { - "locate-path": "^6.0.0", - "path-exists": "^4.0.0" - }, - "engines": { - "node": ">=10" - }, - "funding": { - "url": "https://github.com/sponsors/sindresorhus" - } - }, "node_modules/mocha/node_modules/glob": { "version": "7.2.0", "resolved": "https://registry.npmjs.org/glob/-/glob-7.2.0.tgz", @@ -4639,33 +4580,6 @@ "node": "*" } }, - "node_modules/mocha/node_modules/js-yaml": { - "version": "4.1.0", - "resolved": "https://registry.npmjs.org/js-yaml/-/js-yaml-4.1.0.tgz", - "integrity": "sha512-wpxZs9NoxZaJESJGIZTyDEaYpl0FKSA+FB9aJiyemKhMwkxQg63h4T1KJgUGHpTqPDNRcmmYLugrRjJlBtWvRA==", - "dev": true, - "dependencies": { - "argparse": "^2.0.1" - }, - "bin": { - "js-yaml": "bin/js-yaml.js" - } - }, - "node_modules/mocha/node_modules/locate-path": { - "version": "6.0.0", - "resolved": "https://registry.npmjs.org/locate-path/-/locate-path-6.0.0.tgz", - "integrity": "sha512-iPZK6eYjbxRu3uB4/WZ3EsEIMJFMqAoopl3R+zuq0UjcAm/MO6KCweDgPfP3elTztoKP3KtnVHxTn2NHBSDVUw==", - "dev": true, - "dependencies": { - "p-locate": "^5.0.0" - }, - "engines": { - "node": ">=10" - }, - "funding": { - "url": "https://github.com/sponsors/sindresorhus" - } - }, "node_modules/mocha/node_modules/minimatch": { "version": "5.0.1", "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-5.0.1.tgz", @@ -4693,21 +4607,6 @@ "integrity": "sha512-6FlzubTLZG3J2a/NVCAleEhjzq5oxgHyaCU9yYXvcLsvoVaHJq/s5xXI6/XXP6tz7R9xAOtHnSO/tXtF3WRTlA==", "dev": true }, - "node_modules/mocha/node_modules/p-locate": { - "version": "5.0.0", - "resolved": "https://registry.npmjs.org/p-locate/-/p-locate-5.0.0.tgz", - "integrity": "sha512-LaNjtRWUBY++zB5nE/NwcaoMylSPk+S+ZHNB1TzdbMJMny6dynpAGt7X/tl/QYq3TIeE6nxHppbo2LGymrG5Pw==", - "dev": true, - "dependencies": { - "p-limit": "^3.0.2" - }, - "engines": { - "node": ">=10" - }, - "funding": { - "url": "https://github.com/sponsors/sindresorhus" - } - }, "node_modules/mocha/node_modules/supports-color": { "version": "8.1.1", "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-8.1.1.tgz", @@ -5029,27 +4928,15 @@ } }, "node_modules/p-locate": { - "version": "4.1.0", - "resolved": "https://registry.npmjs.org/p-locate/-/p-locate-4.1.0.tgz", - "integrity": "sha512-R79ZZ/0wAxKGu3oYMlz8jy/kbhsNrS7SKZ7PxEHBgJ5+F2mtFW2fK2cOtBh1cHYkQsbzFV7I+EoRKe6Yt0oK7A==", - "dev": true, - "dependencies": { - "p-limit": "^2.2.0" - }, - "engines": { - "node": ">=8" - } - }, - "node_modules/p-locate/node_modules/p-limit": { - "version": "2.3.0", - "resolved": "https://registry.npmjs.org/p-limit/-/p-limit-2.3.0.tgz", - "integrity": "sha512-//88mFWSJx8lxCzwdAABTJL2MyWB12+eIY7MDL2SqLmAkeKU9qxRvWuSyTjm3FUmpBEMuFfckAIqEaVGUDxb6w==", + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/p-locate/-/p-locate-5.0.0.tgz", + "integrity": "sha512-LaNjtRWUBY++zB5nE/NwcaoMylSPk+S+ZHNB1TzdbMJMny6dynpAGt7X/tl/QYq3TIeE6nxHppbo2LGymrG5Pw==", "dev": true, "dependencies": { - "p-try": "^2.0.0" + "p-limit": "^3.0.2" }, "engines": { - "node": ">=6" + "node": ">=10" }, "funding": { "url": "https://github.com/sponsors/sindresorhus" @@ -5180,6 +5067,58 @@ "node": ">=8" } }, + "node_modules/pkg-dir/node_modules/find-up": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/find-up/-/find-up-4.1.0.tgz", + "integrity": "sha512-PpOwAdQ/YlXQ2vj8a3h8IipDuYRi3wceVQQGYWxNINccq40Anw7BlsEXCMbt1Zt+OLA6Fq9suIpIWD0OsnISlw==", + "dev": true, + "dependencies": { + "locate-path": "^5.0.0", + "path-exists": "^4.0.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/pkg-dir/node_modules/locate-path": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/locate-path/-/locate-path-5.0.0.tgz", + "integrity": "sha512-t7hw9pI+WvuwNJXwk5zVHpyhIqzg2qTlklJOf0mVxGSbe3Fp2VieZcduNYjaLDoy6p9uGpQEGWG87WpMKlNq8g==", + "dev": true, + "dependencies": { + "p-locate": "^4.1.0" + }, + "engines": { + "node": ">=8" + } + }, + "node_modules/pkg-dir/node_modules/p-limit": { + "version": "2.3.0", + "resolved": "https://registry.npmjs.org/p-limit/-/p-limit-2.3.0.tgz", + "integrity": "sha512-//88mFWSJx8lxCzwdAABTJL2MyWB12+eIY7MDL2SqLmAkeKU9qxRvWuSyTjm3FUmpBEMuFfckAIqEaVGUDxb6w==", + "dev": true, + "dependencies": { + "p-try": "^2.0.0" + }, + "engines": { + "node": ">=6" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/pkg-dir/node_modules/p-locate": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/p-locate/-/p-locate-4.1.0.tgz", + "integrity": "sha512-R79ZZ/0wAxKGu3oYMlz8jy/kbhsNrS7SKZ7PxEHBgJ5+F2mtFW2fK2cOtBh1cHYkQsbzFV7I+EoRKe6Yt0oK7A==", + "dev": true, + "dependencies": { + "p-limit": "^2.2.0" + }, + "engines": { + "node": ">=8" + } + }, "node_modules/prelude-ls": { "version": "1.2.1", "resolved": "https://registry.npmjs.org/prelude-ls/-/prelude-ls-1.2.1.tgz", @@ -5378,7 +5317,7 @@ "node": ">=8" } }, - "node_modules/resolve-from": { + "node_modules/resolve-cwd/node_modules/resolve-from": { "version": "5.0.0", "resolved": "https://registry.npmjs.org/resolve-from/-/resolve-from-5.0.0.tgz", "integrity": "sha512-qYg9KP24dD5qka9J47d0aVky0N+b4fTU89LN9iDnjB5waksiC49rvMB0PrUJQGoTmH50XPiqOvAjDfaijGxYZw==", @@ -5387,6 +5326,15 @@ "node": ">=8" } }, + "node_modules/resolve-from": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/resolve-from/-/resolve-from-4.0.0.tgz", + "integrity": "sha512-pb/MYmXstAkysRFx8piNI1tGFNQIFA3vkE3Gq4EuA1dF6gHp/+vgZqsCGJapvy8N3Q+4o7FwvquPJcnZ7RYy4g==", + "dev": true, + "engines": { + "node": ">=4" + } + }, "node_modules/resolve.exports": { "version": "2.0.2", "resolved": "https://registry.npmjs.org/resolve.exports/-/resolve.exports-2.0.2.tgz", @@ -5649,6 +5597,15 @@ "node": ">=10" } }, + "node_modules/stack-utils/node_modules/escape-string-regexp": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/escape-string-regexp/-/escape-string-regexp-2.0.0.tgz", + "integrity": "sha512-UpzcLCXolUWcNu5HtVMHYdXJjArjsF9C0aNnquZYY4uW/Vu0miy5YoWvbV345HauVvcAUnpRuhMMcqTcGOY2+w==", + "dev": true, + "engines": { + "node": ">=8" + } + }, "node_modules/string_decoder": { "version": "0.10.31", "resolved": "https://registry.npmjs.org/string_decoder/-/string_decoder-0.10.31.tgz", @@ -5861,9 +5818,9 @@ } }, "node_modules/type-fest": { - "version": "0.21.3", - "resolved": "https://registry.npmjs.org/type-fest/-/type-fest-0.21.3.tgz", - "integrity": "sha512-t0rzBq87m3fVcduHDUFhKmyyX+9eo6WQjZvf51Ea/M0Q7+T374Jp1aUiyUl0GKxp8M/OETVHSDvmkyPgvX+X2w==", + "version": "0.20.2", + "resolved": "https://registry.npmjs.org/type-fest/-/type-fest-0.20.2.tgz", + "integrity": "sha512-Ne+eE4r0/iWnpAxD852z3A+N0Bt5RN//NjJwRd2VFHEmrywxf5vsZlh4R6lixl6B+wz/8d+maTSAkN1FIkI3LQ==", "dev": true, "engines": { "node": ">=10" diff --git a/package.json b/package.json index 8265bf00..9b755823 100644 --- a/package.json +++ b/package.json @@ -17,7 +17,7 @@ "module_name": "confluent-kafka-js", "module_path": "./build/{configuration}/", "package_name": "{module_name}-v{version}-{node_abi}-{platform}-{libc}-{arch}.tar.gz", - "host": "https://github.com/confluentinc/confluent-kafka-js/releases/download/", + "host": "https://confluent-kafka-js.s3.us-west-1.amazonaws.com/releases/download/", "remote_path": "v{version}" }, "keywords": [